You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by zu...@apache.org on 2016/06/14 22:20:05 UTC

[01/13] incubator-quickstep git commit: QUICKSTEP-10: Serialized WorkOrders as proto. [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/travis-grpc 14ad61c0f -> 1dae00663 (forced update)


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SortMergeRunOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.cpp b/relational_operators/SortMergeRunOperator.cpp
index 6bf5719..e398d62 100644
--- a/relational_operators/SortMergeRunOperator.cpp
+++ b/relational_operators/SortMergeRunOperator.cpp
@@ -23,9 +23,11 @@
 #include <vector>
 
 #include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
 #include "relational_operators/SortMergeRunOperator.pb.h"
 #include "relational_operators/SortMergeRunOperatorHelpers.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "threading/ThreadIDBasedMap.hpp"
 
 #include "glog/logging.h"
@@ -69,6 +71,72 @@ bool SortMergeRunOperator::getAllWorkOrders(
   return generateWorkOrders(container, query_context, storage_manager, scheduler_client_id, bus);
 }
 
+bool SortMergeRunOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (input_relation_is_stored_) {
+    // Input blocks (or runs) are from base relation. Only possible when base
+    // relation is stored sorted.
+    if (!started_) {
+      // Initialize merge tree completely, since all input runs are known.
+      merge_tree_.initializeTree(input_relation_block_ids_.size());
+      started_ = true;
+      initializeInputRuns();
+    }
+  } else {
+    // Input blocks (or runs) are pipelined from the sorted run generation
+    // operator.
+    if (!started_ && !input_stream_done_) {
+      // Initialize merge tree for first pipeline mode.
+      merge_tree_.initializeForPipeline();
+      started_ = true;
+      initializeInputRuns();
+    }
+  }
+
+  // Get merge jobs from merge tree.
+  std::vector<MergeTree::MergeJob> jobs;
+  const bool done_generating = merge_tree_.getMergeJobs(&jobs);
+
+  for (std::vector<MergeTree::MergeJob>::size_type job_id = 0;
+       job_id < jobs.size();
+       ++job_id) {
+    // Add work order for each merge job.
+    container->addWorkOrderProto(createWorkOrderProto(&jobs[job_id]), op_index_);
+  }
+
+  return done_generating;
+}
+
+serialization::WorkOrder* SortMergeRunOperator::createWorkOrderProto(
+    merge_run_operator::MergeTree::MergeJob *job) {
+  DCHECK(job != nullptr);
+  DCHECK(!job->runs.empty());
+
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::SORT_MERGE_RUN);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::SortMergeRunWorkOrder::operator_index, op_index_);
+  proto->SetExtension(serialization::SortMergeRunWorkOrder::sort_config_index, sort_config_index_);
+
+  for (const merge_run_operator::Run &run : job->runs) {
+    serialization::Run *run_proto = proto->AddExtension(serialization::SortMergeRunWorkOrder::runs);
+    for (const block_id block : run) {
+      run_proto->add_blocks(block);
+    }
+  }
+
+  proto->SetExtension(serialization::SortMergeRunWorkOrder::top_k, top_k_);
+  proto->SetExtension(serialization::SortMergeRunWorkOrder::merge_level, job->level);
+  proto->SetExtension(serialization::SortMergeRunWorkOrder::relation_id,
+                      job->level > 0 ? run_relation_.getID()
+                                     : input_relation_.getID());
+  proto->SetExtension(serialization::SortMergeRunWorkOrder::insert_destination_index,
+                      job->is_final_level ? output_destination_index_
+                                          : run_block_destination_index_);
+
+  return proto;
+}
+
 WorkOrder *SortMergeRunOperator::createWorkOrder(
     merge_run_operator::MergeTree::MergeJob *job,
     QueryContext *query_context,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SortMergeRunOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.hpp b/relational_operators/SortMergeRunOperator.hpp
index cfff8b9..177836f 100644
--- a/relational_operators/SortMergeRunOperator.hpp
+++ b/relational_operators/SortMergeRunOperator.hpp
@@ -44,8 +44,11 @@ namespace quickstep {
 class CatalogRelationSchema;
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /**
  * @defgroup SortMergeRun Merging Sorted Runs
  * @ingroup Sort
@@ -132,6 +135,8 @@ class SortMergeRunOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   void feedInputBlock(const block_id input_block_id,
                       const relation_id input_relation_id) override {
     input_relation_block_ids_.push_back(input_block_id);
@@ -182,6 +187,13 @@ class SortMergeRunOperator : public RelationalOperator {
                              const tmb::client_id scheduler_client_id,
                              tmb::MessageBus *bus);
 
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param job The merge job.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(merge_run_operator::MergeTree::MergeJob *job);
+
   const CatalogRelation &input_relation_;
 
   const CatalogRelation &output_relation_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SortRunGenerationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.cpp b/relational_operators/SortRunGenerationOperator.cpp
index 37b8fb8..d7362db 100644
--- a/relational_operators/SortRunGenerationOperator.cpp
+++ b/relational_operators/SortRunGenerationOperator.cpp
@@ -21,7 +21,9 @@
 
 #include "catalog/CatalogRelation.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageManager.hpp"
@@ -80,6 +82,43 @@ bool SortRunGenerationOperator::getAllWorkOrders(
   }
 }
 
+bool SortRunGenerationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (input_relation_is_stored_) {
+    // Input blocks are from a base relation.
+    if (!started_) {
+      for (const block_id input_block_id : input_relation_block_ids_) {
+        container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    // Input blocks are pipelined.
+    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+      container->addWorkOrderProto(
+          createWorkOrderProto(input_relation_block_ids_[num_workorders_generated_]),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* SortRunGenerationOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::SORT_RUN_GENERATION);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::SortRunGenerationWorkOrder::sort_config_index, sort_config_index_);
+  proto->SetExtension(serialization::SortRunGenerationWorkOrder::relation_id, input_relation_.getID());
+  proto->SetExtension(serialization::SortRunGenerationWorkOrder::insert_destination_index,
+                      output_destination_index_);
+  proto->SetExtension(serialization::SortRunGenerationWorkOrder::block_id, block);
+
+  return proto;
+}
+
+
 void SortRunGenerationWorkOrder::execute() {
   BlockReference block(
       storage_manager_->getBlock(input_block_id_, input_relation_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index f96e6a6..96a3ce1 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -40,8 +40,11 @@ namespace quickstep {
 class CatalogRelationSchema;
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /**
  * \defgroup Sort Sorting
  * \ingroup RelationalOperators
@@ -112,6 +115,8 @@ class SortRunGenerationOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
     DCHECK(input_relation_id == input_relation_.getID());
     input_relation_block_ids_.push_back(input_block_id);
@@ -133,6 +138,13 @@ class SortRunGenerationOperator : public RelationalOperator {
   }
 
  private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
   const CatalogRelation &input_relation_;
 
   const CatalogRelation &output_relation_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/TableGeneratorOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.cpp b/relational_operators/TableGeneratorOperator.cpp
index a3f9340..d5a08ec 100644
--- a/relational_operators/TableGeneratorOperator.cpp
+++ b/relational_operators/TableGeneratorOperator.cpp
@@ -1,6 +1,6 @@
 /**
  *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ *     University of Wisconsin\u2014Madison.
  *   Copyright 2016 Pivotal Software, Inc.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
@@ -20,7 +20,9 @@
 
 #include "expressions/table_generator/GeneratorFunctionHandle.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
 
@@ -53,6 +55,22 @@ bool TableGeneratorOperator::getAllWorkOrders(
   return started_;
 }
 
+bool TableGeneratorOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (!started_) {
+    serialization::WorkOrder *proto = new serialization::WorkOrder;
+    proto->set_work_order_type(serialization::TABLE_GENERATOR);
+    proto->set_query_id(query_id_);
+
+    proto->SetExtension(serialization::TableGeneratorWorkOrder::generator_function_index, generator_function_index_);
+    proto->SetExtension(serialization::TableGeneratorWorkOrder::insert_destination_index, output_destination_index_);
+
+    container->addWorkOrderProto(proto, op_index_);
+    started_ = true;
+  }
+  return true;
+}
+
+
 void TableGeneratorWorkOrder::execute() {
   ColumnVectorsValueAccessor temp_result;
   function_handle_.populateColumns(&temp_result);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/TableGeneratorOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.hpp b/relational_operators/TableGeneratorOperator.hpp
index 6a6af4b..1b791a6 100644
--- a/relational_operators/TableGeneratorOperator.hpp
+++ b/relational_operators/TableGeneratorOperator.hpp
@@ -1,6 +1,6 @@
 /**
  *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
- *   University of Wisconsin\u2014Madison.
+ *     University of Wisconsin\u2014Madison.
  *   Copyright 2016 Pivotal Software, Inc.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
@@ -40,6 +40,7 @@ namespace quickstep {
 class GeneratorFunctionHandle;
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -81,6 +82,8 @@ class TableGeneratorOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index d2fd0cd..49c9150 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -22,6 +22,7 @@
 #include <algorithm>
 #include <cctype>
 #include <cstddef>
+#include <cstdint>
 #include <cstdio>
 #include <cstdlib>
 #include <memory>
@@ -31,21 +32,46 @@
 
 #include "catalog/CatalogAttribute.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "types/Type.hpp"
 #include "types/TypedValue.hpp"
-#include "types/containers/Tuple.hpp"
 #include "types/containers/ColumnVector.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "types/containers/Tuple.hpp"
 #include "utility/Glob.hpp"
 
+#include "gflags/gflags.h"
 #include "glog/logging.h"
 
 #include "tmb/id_typedefs.h"
 
+using std::size_t;
+using std::string;
+
 namespace quickstep {
 
+// Text segment size set to 256KB.
+DEFINE_uint64(textscan_text_segment_size, 0x40000,
+              "Size of text segment in bytes the input text files "
+              "are split into in the TextScanOperator.");
+
+// Check if the segment size is positive.
+static bool ValidateTextScanTextSegmentSize(const char *flagname,
+                                            std::uint64_t text_segment_size) {
+  if (text_segment_size == 0) {
+    LOG(ERROR) << "--" << flagname << " must be greater than 0";
+    return false;
+  }
+
+  return true;
+}
+
+static const volatile bool text_scan_text_segment_size_dummy = gflags::RegisterFlagValidator(
+    &FLAGS_textscan_text_segment_size, &ValidateTextScanTextSegmentSize);
+
 bool TextScanOperator::getAllWorkOrders(
     WorkOrdersContainer *container,
     QueryContext *query_context,
@@ -56,16 +82,12 @@ bool TextScanOperator::getAllWorkOrders(
 
   const std::vector<std::string> files = utility::file::GlobExpand(file_pattern_);
 
-  if (files.size() == 0) {
-    LOG(FATAL) << "No files matched '" << file_pattern_ << "'. Exiting.";
-  }
+  CHECK_NE(files.size(), 0u)
+      << "No files matched '" << file_pattern_ << "'. Exiting.";
 
   InsertDestination *output_destination =
       query_context->getInsertDestination(output_destination_index_);
 
-  // Text segment size set to 256KB.
-  constexpr std::size_t kTextSegmentSize = 0x40000u;
-
   if (blocking_dependencies_met_ && !work_generated_) {
     for (const std::string &file : files) {
       // Use standard C libary to retrieve the file size.
@@ -75,18 +97,32 @@ bool TextScanOperator::getAllWorkOrders(
       std::fclose(fp);
 
       std::size_t text_offset = 0;
-      while (text_offset < file_size) {
+      for (size_t num_full_segments = file_size / FLAGS_textscan_text_segment_size;
+           num_full_segments > 0;
+           --num_full_segments, text_offset += FLAGS_textscan_text_segment_size) {
+        container->addNormalWorkOrder(
+            new TextScanWorkOrder(query_id_,
+                                  file,
+                                  text_offset,
+                                  FLAGS_textscan_text_segment_size,
+                                  field_terminator_,
+                                  process_escape_sequences_,
+                                  output_destination),
+            op_index_);
+      }
+
+      // Deal with the residual partial segment whose size is less than
+      // 'FLAGS_textscan_text_segment_size'.
+      if (text_offset < file_size) {
         container->addNormalWorkOrder(
             new TextScanWorkOrder(query_id_,
                                   file,
                                   text_offset,
-                                  std::min(kTextSegmentSize, file_size - text_offset),
+                                  file_size - text_offset,
                                   field_terminator_,
                                   process_escape_sequences_,
-                                  output_destination,
-                                  storage_manager),
+                                  output_destination),
             op_index_);
-        text_offset += kTextSegmentSize;
       }
     }
     work_generated_ = true;
@@ -94,24 +130,53 @@ bool TextScanOperator::getAllWorkOrders(
   return work_generated_;
 }
 
-TextScanWorkOrder::TextScanWorkOrder(const std::size_t query_id,
-                                     const std::string &filename,
-                                     const std::size_t text_offset,
-                                     const std::size_t text_segment_size,
-                                     const char field_terminator,
-                                     const bool process_escape_sequences,
-                                     InsertDestination *output_destination,
-                                     StorageManager *storage_manager)
-    : WorkOrder(query_id),
-      filename_(filename),
-      text_offset_(text_offset),
-      text_segment_size_(text_segment_size),
-      field_terminator_(field_terminator),
-      process_escape_sequences_(process_escape_sequences),
-      output_destination_(output_destination),
-      storage_manager_(storage_manager) {
-  DCHECK(output_destination_ != nullptr);
-  DCHECK(storage_manager_ != nullptr);
+bool TextScanOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  const std::vector<std::string> files = utility::file::GlobExpand(file_pattern_);
+  if (blocking_dependencies_met_ && !work_generated_) {
+    for (const string &file : files) {
+      // Use standard C libary to retrieve the file size.
+      FILE *fp = std::fopen(file.c_str(), "rb");
+      std::fseek(fp, 0, SEEK_END);
+      const std::size_t file_size = std::ftell(fp);
+      std::fclose(fp);
+
+      size_t text_offset = 0;
+      for (size_t num_full_segments = file_size / FLAGS_textscan_text_segment_size;
+           num_full_segments > 0;
+           --num_full_segments, text_offset += FLAGS_textscan_text_segment_size) {
+        container->addWorkOrderProto(createWorkOrderProto(file, text_offset, FLAGS_textscan_text_segment_size),
+                                     op_index_);
+      }
+
+      // Deal with the residual partial segment whose size is less than
+      // 'FLAGS_textscan_text_segment_size'.
+      if (text_offset < file_size) {
+        container->addWorkOrderProto(createWorkOrderProto(file, text_offset, file_size - text_offset),
+                                     op_index_);
+      }
+    }
+    work_generated_ = true;
+  }
+  return work_generated_;
+}
+
+serialization::WorkOrder* TextScanOperator::createWorkOrderProto(const string &filename,
+                                                                 const size_t text_offset,
+                                                                 const size_t text_segment_size) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::TEXT_SCAN);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::TextScanWorkOrder::filename, filename);
+  proto->SetExtension(serialization::TextScanWorkOrder::text_offset, text_offset);
+  proto->SetExtension(serialization::TextScanWorkOrder::text_segment_size, text_segment_size);
+  proto->SetExtension(serialization::TextScanWorkOrder::field_terminator, field_terminator_);
+  proto->SetExtension(serialization::TextScanWorkOrder::process_escape_sequences,
+                      process_escape_sequences_);
+  proto->SetExtension(serialization::TextScanWorkOrder::insert_destination_index,
+                      output_destination_index_);
+
+  return proto;
 }
 
 void TextScanWorkOrder::execute() {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index d73e7dd..1a62ded 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -33,6 +33,8 @@
 #include "types/containers/Tuple.hpp"
 #include "utility/Macros.hpp"
 
+#include "glog/logging.h"
+
 #include "tmb/id_typedefs.h"
 
 namespace tmb { class MessageBus; }
@@ -42,8 +44,11 @@ namespace quickstep {
 class CatalogRelationSchema;
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -135,6 +140,8 @@ class TextScanOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   QueryContext::insert_destination_id getInsertDestinationID() const override {
     return output_destination_index_;
   }
@@ -144,6 +151,10 @@ class TextScanOperator : public RelationalOperator {
   }
 
  private:
+  serialization::WorkOrder* createWorkOrderProto(const std::string &filename,
+                                                 const std::size_t text_offset,
+                                                 const std::size_t text_segment_size);
+
   const std::string file_pattern_;
   const char field_terminator_;
   const bool process_escape_sequences_;
@@ -173,7 +184,6 @@ class TextScanWorkOrder : public WorkOrder {
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
    * @param output_destination The InsertDestination to insert tuples.
-   * @param storage_manager The StorageManager to use.
    **/
   TextScanWorkOrder(
       const std::size_t query_id,
@@ -182,8 +192,14 @@ class TextScanWorkOrder : public WorkOrder {
       const std::size_t text_segment_size,
       const char field_terminator,
       const bool process_escape_sequences,
-      InsertDestination *output_destination,
-      StorageManager *storage_manager);
+      InsertDestination *output_destination)
+      : WorkOrder(query_id),
+        filename_(filename),
+        text_offset_(text_offset),
+        text_segment_size_(text_segment_size),
+        field_terminator_(field_terminator),
+        process_escape_sequences_(process_escape_sequences),
+        output_destination_(DCHECK_NOTNULL(output_destination)) {}
 
   ~TextScanWorkOrder() override {}
 
@@ -233,7 +249,6 @@ class TextScanWorkOrder : public WorkOrder {
   Tuple parseRow(const char **row_ptr,
                  const CatalogRelationSchema &relation) const;
 
-
   /**
    * @brief Parse up to three octal digits (0-7) starting at \p *literal_ptr as
    *        a char literal. \p *literal_ptr will be modified to the last position
@@ -297,7 +312,6 @@ class TextScanWorkOrder : public WorkOrder {
   const bool process_escape_sequences_;
 
   InsertDestination *output_destination_;
-  StorageManager *storage_manager_;
 
   DISALLOW_COPY_AND_ASSIGN(TextScanWorkOrder);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/TextScanOperator.proto
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.proto b/relational_operators/TextScanOperator.proto
deleted file mode 100644
index 8ead3f3..0000000
--- a/relational_operators/TextScanOperator.proto
+++ /dev/null
@@ -1,22 +0,0 @@
-//   Copyright 2015 Pivotal Software, Inc.
-//
-//   Licensed under the Apache License, Version 2.0 (the "License");
-//   you may not use this file except in compliance with the License.
-//   You may obtain a copy of the License at
-//
-//       http://www.apache.org/licenses/LICENSE-2.0
-//
-//   Unless required by applicable law or agreed to in writing, software
-//   distributed under the License is distributed on an "AS IS" BASIS,
-//   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-//   See the License for the specific language governing permissions and
-//   limitations under the License.
-
-syntax = "proto2";
-
-package quickstep.serialization;
-
-message TextBlob {
-  required fixed64 blob_id = 1;
-  required uint64 size = 2;
-}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.cpp b/relational_operators/UpdateOperator.cpp
index 2130563..1b2979e 100644
--- a/relational_operators/UpdateOperator.cpp
+++ b/relational_operators/UpdateOperator.cpp
@@ -26,7 +26,9 @@
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/QueryExecutionMessages.pb.h"
 #include "query_execution/QueryExecutionUtil.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -72,6 +74,27 @@ bool UpdateOperator::getAllWorkOrders(
   return started_;
 }
 
+bool UpdateOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (blocking_dependencies_met_ && !started_) {
+    for (const block_id input_block_id : input_blocks_) {
+      serialization::WorkOrder *proto = new serialization::WorkOrder;
+      proto->set_work_order_type(serialization::UPDATE);
+      proto->set_query_id(query_id_);
+
+      proto->SetExtension(serialization::UpdateWorkOrder::operator_index, op_index_);
+      proto->SetExtension(serialization::UpdateWorkOrder::relation_id, relation_.getID());
+      proto->SetExtension(serialization::UpdateWorkOrder::insert_destination_index, relocation_destination_index_);
+      proto->SetExtension(serialization::UpdateWorkOrder::predicate_index, predicate_index_);
+      proto->SetExtension(serialization::UpdateWorkOrder::update_group_index, update_group_index_);
+      proto->SetExtension(serialization::UpdateWorkOrder::block_id, input_block_id);
+
+      container->addWorkOrderProto(proto, op_index_);
+    }
+    started_ = true;
+  }
+  return started_;
+}
+
 void UpdateWorkOrder::execute() {
   MutableBlockReference block(
       storage_manager_->getBlockMutable(input_block_id_, relation_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index cebb9b5..f6c5053 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -45,6 +45,7 @@ class InsertDestination;
 class Predicate;
 class Scalar;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -99,6 +100,8 @@ class UpdateOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   QueryContext::insert_destination_id getInsertDestinationID() const override {
     return relocation_destination_index_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index 60d4c8f..3ed065a 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -232,23 +232,14 @@ message TextScanWorkOrder {
   }
 }
 
-message TextSplitWorkOrder {
-  extend WorkOrder {
-    // All required.
-    optional uint64 operator_index = 320;
-    optional string filename = 321;
-    optional bool process_escape_sequences = 322;
-  }
-}
-
 message UpdateWorkOrder {
   extend WorkOrder {
     // All required.
-    optional uint64 operator_index = 336;
-    optional int32 relation_id = 337;
-    optional int32 insert_destination_index = 338;
-    optional int32 predicate_index = 339;
-    optional uint32 update_group_index = 340;
-    optional fixed64 block_id = 341;
+    optional uint64 operator_index = 320;
+    optional int32 relation_id = 321;
+    optional int32 insert_destination_index = 322;
+    optional int32 predicate_index = 323;
+    optional uint32 update_group_index = 324;
+    optional fixed64 block_id = 325;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index da42b4d..e078b84 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -396,8 +396,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
           proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
           query_context->getInsertDestination(
-              proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
-          storage_manager);
+              proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)));
     }
     case serialization::UPDATE: {
       LOG(INFO) << "Creating UpdateWorkOrder";
@@ -425,6 +424,10 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
 bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
                                     const CatalogDatabaseLite &catalog_database,
                                     const QueryContext &query_context) {
+  if (!proto.IsInitialized()) {
+    return false;
+  }
+
   switch (proto.work_order_type()) {
     case serialization::AGGREGATION: {
       return proto.HasExtension(serialization::AggregationWorkOrder::block_id) &&


[10/13] incubator-quickstep git commit: Fix bug: specify log file name correctly

Posted by zu...@apache.org.
Fix bug: specify log file name correctly


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

Branch: refs/heads/travis-grpc
Commit: 31e9e4cefb2a9fffd42138fcd40543f915ed07bb
Parents: 07c011d
Author: Navneet <na...@cs.wisc.edu>
Authored: Mon Jun 13 18:34:40 2016 -0500
Committer: Navneet <na...@cs.wisc.edu>
Committed: Mon Jun 13 18:39:29 2016 -0500

----------------------------------------------------------------------
 build/profile_build.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/31e9e4ce/build/profile_build.sh
----------------------------------------------------------------------
diff --git a/build/profile_build.sh b/build/profile_build.sh
index 5f2fc33..b7164ad 100755
--- a/build/profile_build.sh
+++ b/build/profile_build.sh
@@ -37,7 +37,7 @@ CMAKE_COMMAND="cmake \
 -D CMAKE_BUILD_TYPE=Debug \
 .. "
 MAKE_COMMAND="make VERBOSE=1"
-LOG_FILENAME=$LOG_FILENAME
+LOG_FILENAME=build.log
 
 # Continuously dump memory usage and cpu load info to files for later analysis
 function start_stat_collectors {


[03/13] incubator-quickstep git commit: Configure libtcmalloc_minimal based on shared libs setting

Posted by zu...@apache.org.
Configure libtcmalloc_minimal based on shared libs setting

- Added a switch to change the way we build libtcmalloc_minimal i.e.
  either a static library or shared library.


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

Branch: refs/heads/travis-grpc
Commit: e40e3ba30fccd4230f9255e65dd5e76e84605860
Parents: c9214ec
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Fri Jun 10 10:55:49 2016 -0500
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Fri Jun 10 10:19:25 2016 -0700

----------------------------------------------------------------------
 CMakeLists.txt | 10 +++++++---
 1 file changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e40e3ba3/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 2d10a78..207f313 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -500,9 +500,13 @@ if(USE_TCMALLOC)
     #   as some generators, e.g. Ninja, may need it to build properly
     # BUILD_BYPRODUCTS <INSTALL_DIR>/lib/libtcmalloc_minimal.a
   )
-  # Static libtcmalloc_minimal.a
-  add_library(libtcmalloc_minimal STATIC IMPORTED)
-  set_property(TARGET libtcmalloc_minimal PROPERTY IMPORTED_LOCATION ${CMAKE_CURRENT_BINARY_DIR}/third_party/gperftools/lib/libtcmalloc_minimal.a)
+  if (BUILD_SHARED_LIBS)
+    add_library(libtcmalloc_minimal SHARED IMPORTED)
+    set_property(TARGET libtcmalloc_minimal PROPERTY IMPORTED_LOCATION ${CMAKE_CURRENT_BINARY_DIR}/third_party/gperftools/lib/libtcmalloc_minimal.so)
+  else()
+    add_library(libtcmalloc_minimal STATIC IMPORTED)
+    set_property(TARGET libtcmalloc_minimal PROPERTY IMPORTED_LOCATION ${CMAKE_CURRENT_BINARY_DIR}/third_party/gperftools/lib/libtcmalloc_minimal.a)
+  endif()
   # Linking against tcmalloc also requires linking against the system threading
   # library.
   set_property(TARGET libtcmalloc_minimal PROPERTY IMPORTED_LINK_INTERFACE_LIBRARIES ${CMAKE_THREAD_LIBS_INIT})


[13/13] incubator-quickstep git commit: CI w/ gRPC to build the distributed version.

Posted by zu...@apache.org.
CI w/ gRPC to build 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/1dae0066
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/1dae0066
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/1dae0066

Branch: refs/heads/travis-grpc
Commit: 1dae00663e35684cc83abb2a48ca94c45cf25895
Parents: dd44958
Author: Zuyu Zhang <zz...@pivotal.io>
Authored: Mon May 30 11:57:22 2016 -0700
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jun 14 15:20:24 2016 -0700

----------------------------------------------------------------------
 .travis.yml    | 28 +++++++++++++++++++---------
 CMakeLists.txt |  8 ++++++++
 2 files changed, 27 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1dae0066/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index fce3ba0..7e2a6ad 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -1,4 +1,4 @@
-# NOTE(quickstep-team): In Travis-CI, jobs timeout if they take more than 120
+# NOTE(quickstep-team): In Travis-CI, jobs timeout if they take more than 50
 # mins or if there is no log output for more than 10 mins. Hence, we use -O0 to
 # speed up compilation in release build. Also, jobs can only use upto 20GB of
 # disk space. Hence, we minimize the amount of debug symbol using -g0 (DEBUG
@@ -15,10 +15,10 @@ compiler:
   - clang
 
 env:
-  - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=joinwithbinaryexpressions
-  - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=joinwithbinaryexpressions
   - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=selection
   - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=selection
+  - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=none
+  - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=none
 
 before_install:
   - LLVM_VERSION=3.7.1
@@ -29,12 +29,12 @@ before_install:
       tar xf $LLVM_ARCHIVE_PATH -C $HOME/clang-$LLVM_VERSION --strip-components 1;
       ln -sf $HOME/clang-$LLVM_VERSION/bin/clang++ $HOME/clang-$LLVM_VERSION/bin/clang++-3.7;
       export PATH=$HOME/clang-$LLVM_VERSION/bin:$PATH;
-      export CPPFLAGS="-I $HOME/clang-$LLVM_VERSION/include/c++/v1";
+      export CPPFLAGS="-I $HOME/clang-$LLVM_VERSION/include/c++/v1 -L$HOME/clang-$LLVM_VERSION/lib -stdlib=libc++";
       echo "Using clang at " `which $CC-3.7` " and $CXX at " `which $CXX-3.7`;
     fi
 
 install:
-  - if [ "$CC" = "gcc" ]; then
+  - if [ "$VECTOR_COPY_ELISION_LEVEL" = "selection" ] && [ "$CC" = "gcc" ]; then
       export MAKE_JOBS=1;
     else
       export MAKE_JOBS=2;
@@ -51,11 +51,22 @@ install:
   - export DEBUG_FLAGS="-g0";
   - export RELEASE_FLAGS="-O0 -DNDEBUG";
   - export LINKER_FLAGS="-s"
+  # Protobuf 3 beta.
+  - wget https://github.com/google/protobuf/releases/download/v3.0.0-beta-3/protobuf-cpp-3.0.0-beta-3.tar.gz
+  - tar -xzvf protobuf-cpp-3.0.0-beta-3.tar.gz
+  - pushd protobuf-3.0.0-beta-3 && ./configure --prefix=/usr && make && sudo make install && popd
+  - rm -rf protobuf*
+  - export PROTOC=`which protoc`
+  # gRPC.
+  - git clone https://github.com/grpc/grpc.git
+  - pushd grpc && git checkout release-0_14_1 && git submodule update --init && make && sudo make install && popd
+  - rm -rf grpc
 
 before_script:
   - $CC --version
   - $CXX --version
   - $CLINKER --version
+  - $PROTOC --version
   - (cd build &&
      cmake -D CMAKE_BUILD_TYPE=$BUILD_TYPE
            -D BUILD_SHARED_LIBS=On
@@ -68,14 +79,15 @@ before_script:
            -D CMAKE_CXX_COMPILER=$CXX
            -D CMAKE_LINKER=$CLINKER
            -D USE_TCMALLOC=0
-           -D VECTOR_COPY_ELISION_LEVEL=$VECTOR_COPY_ELISION_LEVEL ..)
+           -D VECTOR_COPY_ELISION_LEVEL=$VECTOR_COPY_ELISION_LEVEL
+           -D ENABLE_DISTRIBUTED=ON ..)
 
 script:
   - ./third_party/cpplint/lint_everything.py
   - ./validate_cmakelists.py
   - ./cyclic_dependency.py
   - (cd build && make -j$MAKE_JOBS)
-  - (cd build && ctest --output-on-failure -j$TEST_JOBS)
+  - (cd build && ctest -E native_net_client_message_bus_unittest --output-on-failure -j$TEST_JOBS)
 
 after_failure:
   - df -h
@@ -95,8 +107,6 @@ addons:
       - gcc-5
       - g++-5
       - binutils-gold
-      - libprotobuf-dev
-      - protobuf-compiler
       - libgtest-dev
       - python-networkx
       - libnuma-dev

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1dae0066/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 20e1fb9..de6754a 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -281,6 +281,14 @@ else()
     set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror")
   endif()
 
+  # Clang reports such warning when using Protoc 3.0 beta.
+  if(${CMAKE_CXX_COMPILER_ID} MATCHES "Clang")
+    CHECK_CXX_COMPILER_FLAG("-Wno-extended-offsetof" COMPILER_HAS_WNO_EXTENDED_OFFSETOF)
+    if (COMPILER_HAS_WNO_EXTENDED_OFFSETOF)
+      set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-extended-offsetof")
+    endif()
+  endif()
+
   # One of the protobuf headers includes a nested anonymous union within
   # another anonymous type. Ordinarily we work around this by compiling the
   # protobuf libraries themselves with "-Wno-nested-anon-types" and including


[08/13] incubator-quickstep git commit: Add option to enable Google Profiler.

Posted by zu...@apache.org.
Add option to enable Google Profiler.


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

Branch: refs/heads/travis-grpc
Commit: 540c09e623744c5ebf24f845372baaddca4265d1
Parents: 8230b12
Author: Navneet Potti <na...@apache.org>
Authored: Mon Jun 13 15:17:06 2016 -0500
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Jun 13 14:57:16 2016 -0700

----------------------------------------------------------------------
 CMakeLists.txt       | 26 ++++++++++++++++++++++++++
 cli/CMakeLists.txt   |  6 +++++-
 cli/CliConfig.h.in   |  1 +
 cli/QuickstepCli.cpp | 47 ++++++++++++++++++++++++++++++++++++++++++++++-
 4 files changed, 78 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/540c09e6/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 9e445f0..20e1fb9 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -159,6 +159,32 @@ else()
   )
 endif()
 
+# Include Google Perftools CPU Profiler. You probably want to
+# use this option with CMAKE_BUILD_TYPE=RelWithDebInfo.
+# Pass profiler_file_name to quickstep_cli_shell to actually
+# run the profiler. The profiler only starts collecting
+# samples after the first query, so that it runs against a
+# warm buffer pool and caches. If you want to profile everything,
+# including the first query run, set the environment variable
+# CPUPROFILE instead of passing the flag profile_file_name
+# Use google-pprof on the output file to convert it into a useful
+# format like graphviz (dot).
+option(ENABLE_GOOGLE_PROFILER "Include Google Perftools CPU Profiler." OFF)
+
+if (ENABLE_GOOGLE_PROFILER)
+  set_property(
+    DIRECTORY
+    APPEND PROPERTY COMPILE_DEFINITIONS QUICKSTEP_ENABLE_PROFILER
+  )
+
+  # TODO(navsan) Add a FindGperftools.cmake module and use that here.
+  check_include_files("gperftools/profiler.h" HAVE_GPERFTOOLS_PROFILER)
+  if (NOT HAVE_GPERFTOOLS_PROFILER)
+    message(FATAL_ERROR "Could not find gperftools. Ensure that it is installed.")
+  endif()
+  set(LIBS ${LIBS} profiler)
+endif()
+
 # Link against the system's threading library.
 find_package(Threads REQUIRED)
 set(LIBS ${LIBS} ${CMAKE_THREAD_LIBS_INIT})

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/540c09e6/cli/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/CMakeLists.txt b/cli/CMakeLists.txt
index faf5040..44ec223 100644
--- a/cli/CMakeLists.txt
+++ b/cli/CMakeLists.txt
@@ -36,6 +36,10 @@ else()
   set(GFLAGS_LIB_NAME gflags_nothreads-static)
 endif()
 
+if (ENABLE_GOOGLE_PROFILER)
+  set(QUICKSTEP_ENABLE_GOOGLE_PROFILER TRUE)
+endif()
+
 configure_file (
   "${CMAKE_CURRENT_SOURCE_DIR}/CliConfig.h.in"
   "${CMAKE_CURRENT_BINARY_DIR}/CliConfig.h"
@@ -106,7 +110,7 @@ target_link_libraries(quickstep_cli_DefaultsConfigurator
 if(QUICKSTEP_HAVE_LIBNUMA)
   target_link_libraries(quickstep_cli_DefaultsConfigurator
                       ${LIBNUMA_LIBRARY})
-endif() 
+endif()
 target_link_libraries(quickstep_cli_InputParserUtil
                       glog
                       quickstep_utility_Macros

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/540c09e6/cli/CliConfig.h.in
----------------------------------------------------------------------
diff --git a/cli/CliConfig.h.in b/cli/CliConfig.h.in
index b44dfb7..69f4d8a 100644
--- a/cli/CliConfig.h.in
+++ b/cli/CliConfig.h.in
@@ -17,3 +17,4 @@
 
 #cmakedefine QUICKSTEP_USE_LINENOISE
 #cmakedefine QUICKSTEP_OS_WINDOWS
+#cmakedefine QUICKSTEP_ENABLE_GOOGLE_PROFILER

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/540c09e6/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index fbe7e3b..0b64fda 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -36,7 +36,7 @@
 #include <stdlib.h>
 #endif
 
-#include "cli/CliConfig.h"  // For QUICKSTEP_USE_LINENOISE.
+#include "cli/CliConfig.h"  // For QUICKSTEP_USE_LINENOISE, QUICKSTEP_ENABLE_GOOGLE_PROFILER.
 #include "cli/CommandExecutor.hpp"
 #include "cli/DropRelation.hpp"
 
@@ -48,6 +48,10 @@ typedef quickstep::LineReaderLineNoise LineReaderImpl;
 typedef quickstep::LineReaderDumb LineReaderImpl;
 #endif
 
+#ifdef QUICKSTEP_ENABLE_GOOGLE_PROFILER
+#include <gperftools/profiler.h>
+#endif
+
 #include "cli/DefaultsConfigurator.hpp"
 #include "cli/InputParserUtil.hpp"
 #include "cli/PrintToScreen.hpp"
@@ -157,6 +161,30 @@ DEFINE_bool(initialize_db, false, "If true, initialize a database.");
 DEFINE_bool(print_query, false,
             "Print each input query statement. This is useful when running a "
             "large number of queries in a batch.");
+DEFINE_string(profile_file_name, "",
+              "If nonempty, enable profiling using GOOGLE CPU Profiler, and write "
+              "its output to the given file name. This flag has no effect if "
+              "ENABLE_GOOGLE_PROFILER CMake flag was not set during build. "
+              "The profiler only starts collecting samples after the first query, "
+              "so that it runs against a warm buffer pool and caches. If you want to profile "
+              "everything, including the first query run, set the "
+              "environment variable CPUPROFILE instead of passing this flag.");
+              // Here's a detailed explanation of why we skip the first query run
+              // during profiling:
+              // Unless you\u2019ve preloaded the buffer pool (which is not always a good
+              // idea), the first run of the query results in disk I/O and other overhead
+              // that significantly skews the profiling results. It\u2019s the same reason we don\u2019t
+              // include the first run time in our benchmarking: when profiling query
+              // execution, it makes more sense to get numbers using a warm buffer pool and
+              // warm caches. This is not *always* the right thing to do: it\u2019s obviously
+              // wrong for profiling the TextScan operator. In those cases, you might want
+              // to put in your own Profiler probes (just follow the start/stop pattern used
+              // in this file) or just run quickstep with the CPUPROFILE environment variable
+              // set (as per gperftools documentation) to get the full profile for the
+              // entire execution.
+              // To put things in perspective, the first run is, in my experiments, about 5-10
+              // times more expensive than the average run. That means the query needs to be
+              // run at least a hundred times to make the impact of the first run small (< 5 %).
 
 }  // namespace quickstep
 
@@ -345,6 +373,9 @@ int main(int argc, char* argv[]) {
   std::unique_ptr<SqlParserWrapper> parser_wrapper(new SqlParserWrapper());
   std::chrono::time_point<std::chrono::steady_clock> start, end;
 
+#ifdef QUICKSTEP_ENABLE_GOOGLE_PROFILER
+  bool started_profiling = false;
+#endif
   for (;;) {
     string *command_string = new string();
     *command_string = line_reader.getNextCommand();
@@ -446,6 +477,13 @@ int main(int argc, char* argv[]) {
         reset_parser = true;
         break;
       }
+#ifdef QUICKSTEP_ENABLE_GOOGLE_PROFILER
+      // Profile only if profile_file_name flag is set
+      if (!started_profiling && !quickstep::FLAGS_profile_file_name.empty()) {
+        started_profiling = true;
+        ProfilerStart(quickstep::FLAGS_profile_file_name.c_str());
+      }
+#endif
     }
 
     if (quitting) {
@@ -456,6 +494,13 @@ int main(int argc, char* argv[]) {
     }
   }
 
+#ifdef QUICKSTEP_ENABLE_GOOGLE_PROFILER
+  if (started_profiling) {
+    ProfilerStop();
+    ProfilerFlush();
+  }
+#endif
+
   // Kill the foreman and workers.
   QueryExecutionUtil::BroadcastPoisonMessage(main_thread_client_id, &bus);
 


[11/13] incubator-quickstep git commit: Fix conditional per-target flags for lexer

Posted by zu...@apache.org.
Fix conditional per-target flags for lexer


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

Branch: refs/heads/travis-grpc
Commit: 29768a7297746606089c6b2ebca1a9bf44180614
Parents: 31e9e4c
Author: Craig Chasseur <sp...@gmail.com>
Authored: Tue Jun 14 00:51:52 2016 -0700
Committer: Craig Chasseur <sp...@gmail.com>
Committed: Tue Jun 14 00:51:52 2016 -0700

----------------------------------------------------------------------
 parser/CMakeLists.txt | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/29768a72/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index 9738c2c..2488d30 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -374,12 +374,15 @@ set_property(SOURCE ${CMAKE_CURRENT_BINARY_DIR}/SqlParser_gen.cpp APPEND PROPERT
 include(CheckCXXCompilerFlag)
 CHECK_CXX_COMPILER_FLAG("-Wno-deprecated-register" COMPILER_HAS_WNO_DEPRECATED_REGISTER)
 if (COMPILER_HAS_WNO_DEPRECATED_REGISTER)
-  set_target_properties(quickstep_parser_SqlLexer PROPERTIES COMPILE_FLAGS "-Wno-deprecated-register")
+  set_property(TARGET quickstep_parser_SqlLexer APPEND_STRING PROPERTY COMPILE_FLAGS " -Wno-deprecated-register")
 endif()
 
 # GCC will make a warning for unsigned-signed comparisons which are inherent
 # in the lexer. For this, we turn off the sign compare.
-set_target_properties(quickstep_parser_SqlLexer PROPERTIES COMPILE_FLAGS "-Wno-sign-compare")
+CHECK_CXX_COMPILER_FLAG("-Wno-sign-compare" COMPILER_HAS_WNO_SIGN_COMPARE)
+if (COMPILER_HAS_WNO_SIGN_COMPARE)
+  set_property(TARGET quickstep_parser_SqlLexer APPEND_STRING PROPERTY COMPILE_FLAGS " -Wno-sign-compare")
+endif()
 
 add_subdirectory(tests)
 


[07/13] incubator-quickstep git commit: Long lived Foreman thread

Posted by zu...@apache.org.
Long lived Foreman thread

- Foreman thread lives through the lifetime of the Quickstep process.
- Foreman and main thread communicate through TMB messages.
- Foreman admits queries and routes them to PolicyEnforcer.
- Foreman relays messages to policy enforcer which in turn processes it,
  based on the query ID of the message.
- All the tests modified accordingly.

Created PolicyEnforcer class.

- First point of entry for queries in the scheduler.
- Can perform admission control.
- Can talk to the QueryManagers of the active queries to provide them
  messages to process and collect work orders for execution from them.
- Support for admitting multiple queries to the PolicyEnforcer.


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

Branch: refs/heads/travis-grpc
Commit: 8230b12495297b6837a66485119da44d8fb95a26
Parents: 659967a
Author: Harshad Deshmukh <ha...@cs.wisc.edu>
Authored: Sat Apr 9 15:08:40 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Sun Jun 12 09:18:14 2016 -0500

----------------------------------------------------------------------
 CMakeLists.txt                                  |   2 +
 catalog/CatalogTypedefs.hpp                     |   3 +
 cli/CommandExecutor.cpp                         |  53 +-
 cli/CommandExecutor.hpp                         |  11 +-
 cli/QuickstepCli.cpp                            |  66 +-
 cli/tests/CMakeLists.txt                        |   2 +
 cli/tests/CommandExecutorTestRunner.cpp         |  29 +-
 cli/tests/CommandExecutorTestRunner.hpp         |  37 +-
 query_execution/AdmitRequestMessage.hpp         |  73 ++
 query_execution/CMakeLists.txt                  |  70 +-
 query_execution/Foreman.cpp                     | 578 +++--------
 query_execution/Foreman.hpp                     | 393 +-------
 query_execution/PolicyEnforcer.cpp              | 183 ++++
 query_execution/PolicyEnforcer.hpp              | 167 ++++
 query_execution/QueryContext.cpp                |  16 +-
 query_execution/QueryContext.proto              |   2 +
 query_execution/QueryExecutionMessages.proto    |   4 +
 query_execution/QueryExecutionTypedefs.hpp      |   5 +-
 query_execution/QueryExecutionUtil.hpp          |  52 +
 query_execution/QueryManager.hpp                |   5 +-
 query_execution/WorkOrdersContainer.hpp         |  70 +-
 query_execution/Worker.cpp                      |  11 +-
 query_execution/Worker.hpp                      |   2 +
 query_execution/WorkerMessage.hpp               |  24 +-
 query_execution/tests/Foreman_unittest.cpp      | 952 -------------------
 query_execution/tests/QueryManager_unittest.cpp |   7 +-
 .../tests/WorkOrdersContainer_unittest.cpp      |  26 +
 query_optimizer/ExecutionGenerator.hpp          |   1 +
 query_optimizer/tests/CMakeLists.txt            |   2 +
 .../tests/ExecutionGeneratorTestRunner.cpp      |  21 +-
 .../tests/ExecutionGeneratorTestRunner.hpp      |  33 +-
 query_optimizer/tests/TestDatabaseLoader.cpp    |   1 +
 relational_operators/DeleteOperator.cpp         |   1 +
 relational_operators/DeleteOperator.hpp         |   1 +
 relational_operators/HashJoinOperator.hpp       |   4 +-
 relational_operators/RebuildWorkOrder.hpp       |   1 +
 relational_operators/SortMergeRunOperator.cpp   |   1 +
 relational_operators/UpdateOperator.cpp         |   1 +
 relational_operators/UpdateOperator.hpp         |   1 +
 relational_operators/WorkOrder.hpp              |  20 +-
 .../tests/AggregationOperator_unittest.cpp      |   4 +
 .../tests/HashJoinOperator_unittest.cpp         |   6 +
 .../tests/SortMergeRunOperator_unittest.cpp     |   4 +
 .../SortRunGenerationOperator_unittest.cpp      |   1 +
 .../tests/TextScanOperator_unittest.cpp         |   1 +
 storage/InsertDestination.cpp                   |  50 +-
 storage/InsertDestination.hpp                   |  71 +-
 47 files changed, 1064 insertions(+), 2004 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 207f313..9e445f0 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -721,9 +721,11 @@ target_link_libraries(quickstep_cli_shell
                       quickstep_cli_PrintToScreen
                       quickstep_parser_ParseStatement
                       quickstep_parser_SqlParserWrapper
+                      quickstep_queryexecution_AdmitRequestMessage
                       quickstep_queryexecution_Foreman
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_QueryExecutionUtil
                       quickstep_queryexecution_Worker
                       quickstep_queryexecution_WorkerDirectory
                       quickstep_queryexecution_WorkerMessage

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/catalog/CatalogTypedefs.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogTypedefs.hpp b/catalog/CatalogTypedefs.hpp
index 213d91d..44832b6 100644
--- a/catalog/CatalogTypedefs.hpp
+++ b/catalog/CatalogTypedefs.hpp
@@ -46,6 +46,9 @@ const int kCatalogMaxID = INT_MAX;
 // id for the catalog ids.
 constexpr int kInvalidCatalogId = -1;
 
+// Used to indicate no preference for a NUMA Node ID.
+constexpr numa_node_id kAnyNUMANodeID = -1;
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index ddcd38f..dc14741 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -51,6 +51,8 @@
 
 #include "glog/logging.h"
 
+#include "tmb/id_typedefs.h"
+
 using std::fprintf;
 using std::fputc;
 using std::fputs;
@@ -58,6 +60,8 @@ using std::size_t;
 using std::string;
 using std::vector;
 
+namespace tmb { class MessageBus; }
+
 namespace quickstep {
 namespace cli {
 namespace {
@@ -194,11 +198,14 @@ void executeDescribeTable(
 /**
  * @brief A helper function that executes a SQL query to obtain a scalar result.
  */
-inline TypedValue executeQueryForSingleResult(const std::string &query_string,
-                                               StorageManager *storage_manager,
-                                               QueryProcessor *query_processor,
-                                               SqlParserWrapper *parser_wrapper,
-                                               Foreman *foreman) {
+inline TypedValue executeQueryForSingleResult(
+    const tmb::client_id main_thread_client_id,
+    const tmb::client_id foreman_client_id,
+    const std::string &query_string,
+    tmb::MessageBus *bus,
+    StorageManager *storage_manager,
+    QueryProcessor *query_processor,
+    SqlParserWrapper *parser_wrapper) {
   parser_wrapper->feedNextBuffer(new std::string(query_string));
 
   ParseResult result = parser_wrapper->getNextStatement();
@@ -210,11 +217,8 @@ inline TypedValue executeQueryForSingleResult(const std::string &query_string,
   DCHECK(query_handle->getQueryPlanMutable() != nullptr);
 
   // Use foreman to execute the query plan.
-  foreman->setQueryPlan(query_handle->getQueryPlanMutable()->getQueryPlanDAGMutable());
-  foreman->reconstructQueryContextFromProto(query_handle->getQueryContextProto());
-
-  foreman->start();
-  foreman->join();
+  QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
+      main_thread_client_id, foreman_client_id, query_handle.get(), bus);
 
   // Retrieve the scalar result from the result relation.
   const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
@@ -246,8 +250,10 @@ inline TypedValue executeQueryForSingleResult(const std::string &query_string,
   return value;
 }
 
-void executeAnalyze(QueryProcessor *query_processor,
-                    Foreman *foreman,
+void executeAnalyze(const tmb::client_id main_thread_client_id,
+                    const tmb::client_id foreman_client_id,
+                    MessageBus *bus,
+                    QueryProcessor *query_processor,
                     FILE *out) {
   const CatalogDatabase &database = *query_processor->getDefaultDatabase();
   StorageManager *storage_manager = query_processor->getStorageManager();
@@ -273,11 +279,13 @@ void executeAnalyze(QueryProcessor *query_processor,
       query_string.append(";");
 
       TypedValue num_distinct_values =
-          executeQueryForSingleResult(query_string,
+          executeQueryForSingleResult(main_thread_client_id,
+                                      foreman_client_id,
+                                      query_string,
+                                      bus,
                                       storage_manager,
                                       query_processor,
-                                      parser_wrapper.get(),
-                                      foreman);
+                                      parser_wrapper.get());
 
       DCHECK(num_distinct_values.getTypeID() == TypeID::kLong);
       mutable_relation->getStatisticsMutable()->setNumDistinctValues(
@@ -291,11 +299,13 @@ void executeAnalyze(QueryProcessor *query_processor,
     query_string.append(";");
 
     TypedValue num_tuples =
-        executeQueryForSingleResult(query_string,
+        executeQueryForSingleResult(main_thread_client_id,
+                                    foreman_client_id,
+                                    query_string,
+                                    bus,
                                     storage_manager,
                                     query_processor,
-                                    parser_wrapper.get(),
-                                    foreman);
+                                    parser_wrapper.get());
 
     DCHECK(num_tuples.getTypeID() == TypeID::kLong);
     mutable_relation->getStatisticsMutable()->setNumTuples(
@@ -312,9 +322,11 @@ void executeAnalyze(QueryProcessor *query_processor,
 
 void executeCommand(const ParseStatement &statement,
                     const CatalogDatabase &catalog_database,
+                    const tmb::client_id main_thread_client_id,
+                    const tmb::client_id foreman_client_id,
+                    MessageBus *bus,
                     StorageManager *storage_manager,
                     QueryProcessor *query_processor,
-                    Foreman *foreman,
                     FILE *out) {
   const ParseCommand &command = static_cast<const ParseCommand &>(statement);
   const PtrVector<ParseString> *arguments = command.arguments();
@@ -328,7 +340,8 @@ void executeCommand(const ParseStatement &statement,
       executeDescribeTable(arguments, catalog_database, out);
     }
   } else if (command_str == C::kAnalyzeCommand) {
-    executeAnalyze(query_processor, foreman, out);
+    executeAnalyze(
+        main_thread_client_id, foreman_client_id, bus, query_processor, out);
   } else {
     THROW_SQL_ERROR_AT(command.command()) << "Invalid Command";
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/cli/CommandExecutor.hpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.hpp b/cli/CommandExecutor.hpp
index c819981..3435aeb 100644
--- a/cli/CommandExecutor.hpp
+++ b/cli/CommandExecutor.hpp
@@ -21,10 +21,14 @@
 #include <cstdio>
 #include <string>
 
+#include "tmb/id_typedefs.h"
+
 using std::fprintf;
 using std::fputc;
 using std::string;
 
+namespace tmb { class MessageBus; }
+
 namespace quickstep {
 
 class CatalogDatabase;
@@ -53,6 +57,9 @@ constexpr char kAnalyzeCommand[] = "\\analyze";
   *
   * @param statement The parsed statement from the cli.
   * @param catalog_database The catalog information about the current database.
+  * @param main_thread_client_id The TMB client ID of the main thread.
+  * @param foreman_client_id The TMB client ID of the Foreman thread.
+  * @param bus A pointer to the TMB.
   * @param storage_manager The current StorageManager.
   * @param query_processor The query processor to generate plans for SQL queries.
   * @param foreman The foreman to execute query plans.
@@ -60,9 +67,11 @@ constexpr char kAnalyzeCommand[] = "\\analyze";
 */
 void executeCommand(const ParseStatement &statement,
                     const CatalogDatabase &catalog_database,
+                    const tmb::client_id main_thread_client_id,
+                    const tmb::client_id foreman_client_id,
+                    tmb::MessageBus *bus,
                     StorageManager *storage_manager,
                     QueryProcessor *query_processor,
-                    Foreman *foreman,
                     FILE *out);
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 558d6eb..fbe7e3b 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -53,8 +53,10 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 #include "cli/PrintToScreen.hpp"
 #include "parser/ParseStatement.hpp"
 #include "parser/SqlParserWrapper.hpp"
+#include "query_execution/AdmitRequestMessage.hpp"
 #include "query_execution/Foreman.hpp"
 #include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/QueryExecutionUtil.hpp"
 #include "query_execution/Worker.hpp"
 #include "query_execution/WorkerDirectory.hpp"
 #include "query_execution/WorkerMessage.hpp"
@@ -95,6 +97,7 @@ using std::string;
 using std::vector;
 
 using quickstep::Address;
+using quickstep::AdmitRequestMessage;
 using quickstep::CatalogRelation;
 using quickstep::DefaultsConfigurator;
 using quickstep::DropRelation;
@@ -107,6 +110,7 @@ using quickstep::ParseResult;
 using quickstep::ParseStatement;
 using quickstep::PrintToScreen;
 using quickstep::PtrVector;
+using quickstep::QueryExecutionUtil;
 using quickstep::QueryHandle;
 using quickstep::QueryPlan;
 using quickstep::QueryProcessor;
@@ -115,9 +119,12 @@ using quickstep::TaggedMessage;
 using quickstep::Worker;
 using quickstep::WorkerDirectory;
 using quickstep::WorkerMessage;
+using quickstep::kAdmitRequestMessage;
 using quickstep::kPoisonMessage;
+using quickstep::kWorkloadCompletionMessage;
 
 using tmb::client_id;
+using tmb::AnnotatedMessage;
 
 namespace quickstep {
 
@@ -197,7 +204,9 @@ int main(int argc, char* argv[]) {
 
   // The TMB client id for the main thread, used to kill workers at the end.
   const client_id main_thread_client_id = bus.Connect();
+  bus.RegisterClientAsSender(main_thread_client_id, kAdmitRequestMessage);
   bus.RegisterClientAsSender(main_thread_client_id, kPoisonMessage);
+  bus.RegisterClientAsReceiver(main_thread_client_id, kWorkloadCompletionMessage);
 
   // Setup the paths used by StorageManager.
   string fixed_storage_path(quickstep::FLAGS_storage_path);
@@ -283,12 +292,6 @@ int main(int argc, char* argv[]) {
            std::chrono::duration<double>(preload_end - preload_start).count());
   }
 
-  Foreman foreman(&bus,
-                  query_processor->getDefaultDatabase(),
-                  query_processor->getStorageManager(),
-                  -1, /* CPU id to bind foreman. -1 is unbound. */
-                  num_numa_nodes_system);
-
   // Get the NUMA affinities for workers.
   vector<int> cpu_numa_nodes = InputParserUtil::GetNUMANodesForCPUs();
   if (cpu_numa_nodes.empty()) {
@@ -323,13 +326,20 @@ int main(int argc, char* argv[]) {
                                    worker_client_ids,
                                    worker_numa_nodes);
 
-  foreman.setWorkerDirectory(&worker_directory);
+  Foreman foreman(main_thread_client_id,
+                  &worker_directory,
+                  &bus,
+                  query_processor->getDefaultDatabase(),
+                  query_processor->getStorageManager(),
+                  num_numa_nodes_system);
 
   // Start the worker threads.
   for (Worker &worker : workers) {
     worker.start();
   }
 
+  foreman.start();
+
   LineReaderImpl line_reader("quickstep> ",
                              "      ...> ");
   std::unique_ptr<SqlParserWrapper> parser_wrapper(new SqlParserWrapper());
@@ -366,9 +376,11 @@ int main(int argc, char* argv[]) {
             quickstep::cli::executeCommand(
                 *result.parsed_statement,
                 *(query_processor->getDefaultDatabase()),
+                main_thread_client_id,
+                foreman.getBusClientID(),
+                &bus,
                 query_processor->getStorageManager(),
                 query_processor.get(),
-                &foreman,
                 stdout);
           } catch (const quickstep::SqlError &sql_error) {
             fprintf(stderr, "%s",
@@ -389,14 +401,18 @@ int main(int argc, char* argv[]) {
         }
 
         DCHECK(query_handle->getQueryPlanMutable() != nullptr);
-        foreman.setQueryPlan(query_handle->getQueryPlanMutable()->getQueryPlanDAGMutable());
-
-        foreman.reconstructQueryContextFromProto(query_handle->getQueryContextProto());
+        start = std::chrono::steady_clock::now();
+        QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
+            main_thread_client_id,
+            foreman.getBusClientID(),
+            query_handle.get(),
+            &bus);
 
         try {
-          start = std::chrono::steady_clock::now();
-          foreman.start();
-          foreman.join();
+          const AnnotatedMessage annotated_msg =
+              bus.Receive(main_thread_client_id, 0, true);
+          const TaggedMessage &tagged_message = annotated_msg.tagged_message;
+          DCHECK_EQ(kWorkloadCompletionMessage, tagged_message.message_type());
           end = std::chrono::steady_clock::now();
 
           const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
@@ -440,29 +456,13 @@ int main(int argc, char* argv[]) {
     }
   }
 
-  // Terminate all workers before exiting.
-  // The main thread broadcasts poison message to the workers. Each worker dies
-  // after receiving poison message. The order of workers' death is irrelavant.
-  MessageStyle style;
-  style.Broadcast(true);
-  Address address;
-  address.All(true);
-  std::unique_ptr<WorkerMessage> poison_message(WorkerMessage::PoisonMessage());
-  TaggedMessage poison_tagged_message(poison_message.get(),
-                                      sizeof(*poison_message),
-                                      kPoisonMessage);
-
-  const tmb::MessageBus::SendStatus send_status =
-      bus.Send(main_thread_client_id,
-               address,
-               style,
-               std::move(poison_tagged_message));
-  CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
-     "Broadcast message from Foreman to workers failed";
+  // Kill the foreman and workers.
+  QueryExecutionUtil::BroadcastPoisonMessage(main_thread_client_id, &bus);
 
   for (Worker &worker : workers) {
     worker.join();
   }
 
+  foreman.join();
   return 0;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/cli/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/tests/CMakeLists.txt b/cli/tests/CMakeLists.txt
index ca37e4a..d177d6c 100644
--- a/cli/tests/CMakeLists.txt
+++ b/cli/tests/CMakeLists.txt
@@ -32,9 +32,11 @@ target_link_libraries(quickstep_cli_tests_CommandExecutorTest
                       quickstep_cli_PrintToScreen
                       quickstep_parser_ParseStatement
                       quickstep_parser_SqlParserWrapper
+                      quickstep_queryexecution_AdmitRequestMessage
                       quickstep_queryexecution_Foreman
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_QueryExecutionUtil
                       quickstep_queryexecution_Worker
                       quickstep_queryexecution_WorkerDirectory
                       quickstep_queryexecution_WorkerMessage

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/cli/tests/CommandExecutorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/cli/tests/CommandExecutorTestRunner.cpp b/cli/tests/CommandExecutorTestRunner.cpp
index 9cd493e..794f7e1 100644
--- a/cli/tests/CommandExecutorTestRunner.cpp
+++ b/cli/tests/CommandExecutorTestRunner.cpp
@@ -20,12 +20,15 @@
 #include <cstdio>
 #include <set>
 #include <string>
+#include <utility>
 
 #include "cli/CommandExecutor.hpp"
 #include "cli/DropRelation.hpp"
 #include "cli/PrintToScreen.hpp"
 #include "parser/ParseStatement.hpp"
+#include "query_execution/AdmitRequestMessage.hpp"
 #include "query_execution/Foreman.hpp"
+#include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/Worker.hpp"
 #include "query_optimizer/ExecutionGenerator.hpp"
 #include "query_optimizer/LogicalGenerator.hpp"
@@ -41,6 +44,8 @@
 
 #include "glog/logging.h"
 
+#include "tmb/tagged_message.h"
+
 namespace quickstep {
 
 class CatalogRelation;
@@ -87,9 +92,11 @@ void CommandExecutorTestRunner::runTestCase(
           quickstep::cli::executeCommand(
               *result.parsed_statement,
               *(test_database_loader_.catalog_database()),
+              main_thread_client_id_,
+              foreman_->getBusClientID(),
+              &bus_,
               test_database_loader_.storage_manager(),
               nullptr,
-              nullptr,
               output_stream.file());
         } else  {
           QueryHandle query_handle(optimizer_context.query_id());
@@ -100,14 +107,20 @@ void CommandExecutorTestRunner::runTestCase(
               physical_generator.generatePlan(
                   logical_generator.generatePlan(*result.parsed_statement));
           execution_generator.generatePlan(physical_plan);
-          foreman_->setQueryPlan(
-              query_handle.getQueryPlanMutable()->getQueryPlanDAGMutable());
-
-          foreman_->reconstructQueryContextFromProto(query_handle.getQueryContextProto());
-
-          foreman_->start();
-          foreman_->join();
 
+          AdmitRequestMessage request_message(&query_handle);
+          TaggedMessage admit_tagged_message(
+              &request_message, sizeof(request_message), kAdmitRequestMessage);
+          QueryExecutionUtil::SendTMBMessage(&bus_,
+                                             main_thread_client_id_,
+                                             foreman_->getBusClientID(),
+                                             std::move(admit_tagged_message));
+
+          // Receive workload completion message from Foreman.
+          const AnnotatedMessage annotated_msg =
+              bus_.Receive(main_thread_client_id_, 0, true);
+          const TaggedMessage &tagged_message = annotated_msg.tagged_message;
+          DCHECK_EQ(kWorkloadCompletionMessage, tagged_message.message_type());
           const CatalogRelation *query_result_relation = query_handle.getQueryResultRelation();
           if (query_result_relation) {
             PrintToScreen::PrintRelation(*query_result_relation,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/cli/tests/CommandExecutorTestRunner.hpp
----------------------------------------------------------------------
diff --git a/cli/tests/CommandExecutorTestRunner.hpp b/cli/tests/CommandExecutorTestRunner.hpp
index 94b1d6a..8fb5b65 100644
--- a/cli/tests/CommandExecutorTestRunner.hpp
+++ b/cli/tests/CommandExecutorTestRunner.hpp
@@ -27,6 +27,7 @@
 #include "parser/SqlParserWrapper.hpp"
 #include "query_execution/Foreman.hpp"
 #include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/QueryExecutionUtil.hpp"
 #include "query_execution/Worker.hpp"
 #include "query_execution/WorkerDirectory.hpp"
 #include "query_execution/WorkerMessage.hpp"
@@ -34,6 +35,9 @@
 #include "utility/Macros.hpp"
 #include "utility/textbased_test/TextBasedTestDriver.hpp"
 
+#include "tmb/id_typedefs.h"
+#include "tmb/message_bus.h"
+
 namespace quickstep {
 
 /**
@@ -57,9 +61,11 @@ class CommandExecutorTestRunner : public TextBasedTestRunner {
 
     bus_.Initialize();
 
-    foreman_.reset(new Foreman(&bus_,
-                               test_database_loader_.catalog_database(),
-                               test_database_loader_.storage_manager()));
+    main_thread_client_id_ = bus_.Connect();
+    bus_.RegisterClientAsSender(main_thread_client_id_, kAdmitRequestMessage);
+    bus_.RegisterClientAsSender(main_thread_client_id_, kPoisonMessage);
+    bus_.RegisterClientAsReceiver(main_thread_client_id_, kWorkloadCompletionMessage);
+
     worker_.reset(new Worker(0, &bus_));
 
     std::vector<client_id> worker_client_ids;
@@ -71,27 +77,20 @@ class CommandExecutorTestRunner : public TextBasedTestRunner {
 
     workers_.reset(new WorkerDirectory(1 /* number of workers */,
                                        worker_client_ids, numa_nodes));
-    foreman_->setWorkerDirectory(workers_.get());
+    foreman_.reset(new Foreman(main_thread_client_id_,
+                               workers_.get(),
+                               &bus_,
+                               test_database_loader_.catalog_database(),
+                               test_database_loader_.storage_manager()));
 
+    foreman_->start();
     worker_->start();
   }
 
   ~CommandExecutorTestRunner() {
-    std::unique_ptr<WorkerMessage> poison_message(WorkerMessage::PoisonMessage());
-    TaggedMessage poison_tagged_message(poison_message.get(),
-                                        sizeof(*poison_message),
-                                        quickstep::kPoisonMessage);
-
-    Address worker_address;
-    MessageStyle single_receiver_style;
-
-    worker_address.AddRecipient(worker_->getBusClientID());
-    bus_.Send(foreman_->getBusClientID(),
-              worker_address,
-              single_receiver_style,
-              std::move(poison_tagged_message));
-
+    QueryExecutionUtil::BroadcastPoisonMessage(main_thread_client_id_, &bus_);
     worker_->join();
+    foreman_->join();
   }
 
   void runTestCase(const std::string &input,
@@ -102,6 +101,8 @@ class CommandExecutorTestRunner : public TextBasedTestRunner {
   SqlParserWrapper sql_parser_;
   optimizer::TestDatabaseLoader test_database_loader_;
 
+  tmb::client_id main_thread_client_id_;
+
   MessageBusImpl bus_;
   std::unique_ptr<Foreman> foreman_;
   std::unique_ptr<Worker> worker_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/AdmitRequestMessage.hpp
----------------------------------------------------------------------
diff --git a/query_execution/AdmitRequestMessage.hpp b/query_execution/AdmitRequestMessage.hpp
new file mode 100644
index 0000000..e33b354
--- /dev/null
+++ b/query_execution/AdmitRequestMessage.hpp
@@ -0,0 +1,73 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_EXECUTION_ADMIT_REQUEST_MESSAGE_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_ADMIT_REQUEST_MESSAGE_HPP_
+
+#include <vector>
+
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class QueryHandle;
+
+/** \addtogroup QueryExecution
+ *  @{
+ */
+
+/**
+ * @brief A message requesting a query or queries to be admitted to the system.
+ **/
+class AdmitRequestMessage {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param query_handles The handles of the queries requesting to be admitted
+   *        to the system.
+   **/
+  explicit AdmitRequestMessage(const std::vector<QueryHandle*> &query_handles)
+      : query_handles_(query_handles) {}
+
+  /**
+   * @brief Constructor for requesting single query admission.
+   *
+   * @param query_handle The handle of the query requesting to be admitted.
+   **/
+  explicit AdmitRequestMessage(QueryHandle *query_handle) {
+    query_handles_.push_back(query_handle);
+  }
+
+  /**
+   * @brief Get the query handles from this message.
+   **/
+  const std::vector<QueryHandle*>& getQueryHandles() const {
+    return query_handles_;
+  }
+
+ private:
+  std::vector<QueryHandle*> query_handles_;
+
+  DISALLOW_COPY_AND_ASSIGN(AdmitRequestMessage);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_EXECUTION_ADMIT_REQUEST_MESSAGE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 95bc0d6..323e4a9 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -29,8 +29,10 @@ endif()
 if (ENABLE_DISTRIBUTED)
   add_library(quickstep_queryexecution_BlockLocator BlockLocator.cpp BlockLocator.hpp)
 endif()
+add_library(quickstep_queryexecution_AdmitRequestMessage ../empty_src.cpp AdmitRequestMessage.hpp)
 add_library(quickstep_queryexecution_Foreman Foreman.cpp Foreman.hpp)
 add_library(quickstep_queryexecution_ForemanLite ../empty_src.cpp ForemanLite.hpp)
+add_library(quickstep_queryexecution_PolicyEnforcer PolicyEnforcer.cpp PolicyEnforcer.hpp)
 add_library(quickstep_queryexecution_QueryContext QueryContext.cpp QueryContext.hpp)
 add_library(quickstep_queryexecution_QueryContext_proto
             ${queryexecution_QueryContext_proto_srcs}
@@ -50,6 +52,8 @@ add_library(quickstep_queryexecution_WorkerMessage ../empty_src.cpp WorkerMessag
 add_library(quickstep_queryexecution_WorkerSelectionPolicy ../empty_src.cpp WorkerSelectionPolicy.hpp)
 
 # Link dependencies:
+target_link_libraries(quickstep_queryexecution_AdmitRequestMessage
+                      quickstep_utility_Macros)
 if (ENABLE_DISTRIBUTED)
   target_link_libraries(quickstep_queryexecution_BlockLocator
                         glog
@@ -64,29 +68,17 @@ if (ENABLE_DISTRIBUTED)
                         tmb)
 endif()
 target_link_libraries(quickstep_queryexecution_Foreman
+                      ${GFLAGS_LIB_NAME} 
                       glog
-                      gtest
-                      quickstep_catalog_CatalogDatabase
-                      quickstep_catalog_CatalogRelation
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_catalog_PartitionScheme
+                      quickstep_queryexecution_AdmitRequestMessage
                       quickstep_queryexecution_ForemanLite
-                      quickstep_queryexecution_QueryContext
-                      quickstep_queryexecution_QueryExecutionMessages_proto
-                      quickstep_queryexecution_QueryExecutionState
+                      quickstep_queryexecution_PolicyEnforcer
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_QueryExecutionUtil
-                      quickstep_queryexecution_WorkOrdersContainer
                       quickstep_queryexecution_WorkerDirectory
                       quickstep_queryexecution_WorkerMessage
-                      quickstep_relationaloperators_RebuildWorkOrder
-                      quickstep_relationaloperators_RelationalOperator
-                      quickstep_relationaloperators_WorkOrder
-                      quickstep_storage_InsertDestination
-                      quickstep_storage_StorageBlock
-                      quickstep_storage_StorageBlockInfo
                       quickstep_threading_ThreadUtil
-                      quickstep_utility_DAG
+                      quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros
                       tmb)
 target_link_libraries(quickstep_queryexecution_ForemanLite
@@ -94,6 +86,18 @@ target_link_libraries(quickstep_queryexecution_ForemanLite
                       quickstep_threading_Thread
                       quickstep_utility_Macros
                       tmb)
+target_link_libraries(quickstep_queryexecution_PolicyEnforcer
+                      ${GFLAGS_LIB_NAME}
+                      glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_queryexecution_QueryExecutionMessages_proto
+                      quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_QueryManager
+                      quickstep_queryexecution_WorkerMessage
+                      quickstep_queryoptimizer_QueryHandle
+                      quickstep_relationaloperators_WorkOrder
+                      quickstep_utility_Macros
+                      tmb)
 target_link_libraries(quickstep_queryexecution_QueryContext
                       glog
                       quickstep_catalog_CatalogDatabaseLite
@@ -135,7 +139,9 @@ target_link_libraries(quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_threading_ThreadIDBasedMap
                       tmb)
 target_link_libraries(quickstep_queryexecution_QueryExecutionUtil
+                      quickstep_queryexecution_AdmitRequestMessage
                       quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_WorkerMessage
                       quickstep_utility_Macros
                       tmb)
 target_link_libraries(quickstep_queryexecution_QueryManager
@@ -189,8 +195,10 @@ target_link_libraries(quickstep_queryexecution_WorkerSelectionPolicy
 # Module all-in-one library:
 add_library(quickstep_queryexecution ../empty_src.cpp QueryExecutionModule.hpp)
 target_link_libraries(quickstep_queryexecution
+                      quickstep_queryexecution_AdmitRequestMessage
                       quickstep_queryexecution_Foreman
                       quickstep_queryexecution_ForemanLite
+                      quickstep_queryexecution_PolicyEnforcer
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryContext_proto
                       quickstep_queryexecution_QueryExecutionMessages_proto
@@ -235,36 +243,6 @@ if (ENABLE_DISTRIBUTED)
   add_test(BlockLocator_unittest BlockLocator_unittest)
 endif()
 
-add_executable(Foreman_unittest
-  "${CMAKE_CURRENT_SOURCE_DIR}/tests/Foreman_unittest.cpp")
-target_link_libraries(Foreman_unittest
-                      glog
-                      gtest
-                      gtest_main
-                      quickstep_catalog_CatalogDatabase
-                      quickstep_catalog_CatalogRelation
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_queryexecution_Foreman
-                      quickstep_queryexecution_QueryContext
-                      quickstep_queryexecution_QueryContext_proto
-                      quickstep_queryexecution_QueryExecutionState
-                      quickstep_queryexecution_QueryExecutionTypedefs
-                      quickstep_queryexecution_WorkOrdersContainer
-                      quickstep_queryexecution_WorkerDirectory
-                      quickstep_queryexecution_WorkerMessage
-                      quickstep_queryoptimizer_QueryPlan
-                      quickstep_relationaloperators_RelationalOperator
-                      quickstep_relationaloperators_WorkOrder
-                      quickstep_storage_InsertDestination
-                      quickstep_storage_InsertDestination_proto
-                      quickstep_storage_StorageBlock
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_StorageManager
-                      quickstep_utility_DAG
-                      quickstep_utility_Macros
-                      tmb)
-add_test(Foreman_unittest Foreman_unittest)
-
 add_executable(QueryManager_unittest
   "${CMAKE_CURRENT_SOURCE_DIR}/tests/QueryManager_unittest.cpp")
 target_link_libraries(QueryManager_unittest

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/Foreman.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.cpp b/query_execution/Foreman.cpp
index 7705819..0577e20 100644
--- a/query_execution/Foreman.cpp
+++ b/query_execution/Foreman.cpp
@@ -22,355 +22,189 @@
 #include <utility>
 #include <vector>
 
-#include "catalog/CatalogDatabase.hpp"
-#include "catalog/CatalogRelation.hpp"
-#include "catalog/CatalogTypedefs.hpp"
-#include "catalog/PartitionScheme.hpp"
-#include "query_execution/QueryContext.hpp"
-#include "query_execution/QueryExecutionMessages.pb.h"
+#include "query_execution/AdmitRequestMessage.hpp"
 #include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/QueryExecutionUtil.hpp"
 #include "query_execution/WorkerDirectory.hpp"
 #include "query_execution/WorkerMessage.hpp"
-#include "relational_operators/RebuildWorkOrder.hpp"
-#include "relational_operators/RelationalOperator.hpp"
-#include "relational_operators/WorkOrder.hpp"
-#include "storage/InsertDestination.hpp"
-#include "storage/StorageBlock.hpp"
-#include "storage/StorageBlockInfo.hpp"
 #include "threading/ThreadUtil.hpp"
+#include "utility/EqualsAnyConstant.hpp"
 #include "utility/Macros.hpp"
 
+#include "gflags/gflags.h"
 #include "glog/logging.h"
 
 #include "tmb/message_bus.h"
 #include "tmb/tagged_message.h"
 
 using std::move;
-using std::pair;
 using std::size_t;
+using std::unique_ptr;
 using std::vector;
 
 namespace quickstep {
 
-void Foreman::initialize() {
+DEFINE_uint64(min_load_per_worker, 2, "The minimum load defined as the number "
+              "of pending work orders for the worker. This information is used "
+              "by the Foreman to assign work orders to worker threads");
+
+Foreman::Foreman(const tmb::client_id main_thread_client_id,
+                 WorkerDirectory *worker_directory,
+                 tmb::MessageBus *bus,
+                 CatalogDatabaseLite *catalog_database,
+                 StorageManager *storage_manager,
+                 const int cpu_id,
+                 const size_t num_numa_nodes)
+    : ForemanLite(bus, cpu_id),
+      main_thread_client_id_(main_thread_client_id),
+      worker_directory_(DCHECK_NOTNULL(worker_directory)),
+      catalog_database_(DCHECK_NOTNULL(catalog_database)),
+      storage_manager_(DCHECK_NOTNULL(storage_manager)) {
+  const std::vector<QueryExecutionMessageType> sender_message_types{
+      kPoisonMessage,
+      kRebuildWorkOrderMessage,
+      kWorkOrderMessage,
+      kWorkloadCompletionMessage};
+
+  for (const auto message_type : sender_message_types) {
+    bus_->RegisterClientAsSender(foreman_client_id_, message_type);
+  }
+
+  const std::vector<QueryExecutionMessageType> receiver_message_types{
+      kAdmitRequestMessage,
+      kCatalogRelationNewBlockMessage,
+      kDataPipelineMessage,
+      kPoisonMessage,
+      kRebuildWorkOrderCompleteMessage,
+      kWorkOrderFeedbackMessage,
+      kWorkOrdersAvailableMessage,
+      kWorkOrderCompleteMessage};
+
+  for (const auto message_type : receiver_message_types) {
+    bus_->RegisterClientAsReceiver(foreman_client_id_, message_type);
+  }
+
+  policy_enforcer_.reset(new PolicyEnforcer(
+      foreman_client_id_,
+      num_numa_nodes,
+      catalog_database_,
+      storage_manager_,
+      bus_));
+}
+
+void Foreman::run() {
   if (cpu_id_ >= 0) {
     // We can pin the foreman thread to a CPU if specified.
     ThreadUtil::BindToCPU(cpu_id_);
   }
-  initializeState();
-
-  DEBUG_ASSERT(query_dag_ != nullptr);
-  const dag_node_index dag_size = query_dag_->size();
-
-  // Collect all the workorders from all the relational operators in the DAG.
-  for (dag_node_index index = 0; index < dag_size; ++index) {
-    if (checkAllBlockingDependenciesMet(index)) {
-      query_dag_->getNodePayloadMutable(index)->informAllBlockingDependenciesMet();
-      processOperator(index, false);
-    }
-  }
-
-  // Dispatch the WorkOrders generated so far.
-  dispatchWorkerMessages(0, 0);
-}
-
-void Foreman::processWorkOrderCompleteMessage(const dag_node_index op_index,
-                                              const size_t worker_thread_index) {
-  query_exec_state_->decrementNumQueuedWorkOrders(op_index);
-
-  // As the given worker finished executing a WorkOrder, decrement its number
-  // of queued WorkOrders.
-  workers_->decrementNumQueuedWorkOrders(worker_thread_index);
-
-  // Check if new work orders are available and fetch them if so.
-  fetchNormalWorkOrders(op_index);
-
-  if (checkRebuildRequired(op_index)) {
-    if (checkNormalExecutionOver(op_index)) {
-      if (!checkRebuildInitiated(op_index)) {
-        if (initiateRebuild(op_index)) {
-          // Rebuild initiated and completed right away.
-          markOperatorFinished(op_index);
-        } else {
-          // Rebuild under progress.
-        }
-      } else if (checkRebuildOver(op_index)) {
-        // Rebuild was under progress and now it is over.
-        markOperatorFinished(op_index);
-      }
-    } else {
-      // Normal execution under progress for this operator.
-    }
-  } else if (checkOperatorExecutionOver(op_index)) {
-    // Rebuild not required for this operator and its normal execution is
-    // complete.
-    markOperatorFinished(op_index);
-  }
-
-  for (const pair<dag_node_index, bool> &dependent_link :
-       query_dag_->getDependents(op_index)) {
-    const dag_node_index dependent_op_index = dependent_link.first;
-    if (checkAllBlockingDependenciesMet(dependent_op_index)) {
-      // Process the dependent operator (of the operator whose WorkOrder
-      // was just executed) for which all the dependencies have been met.
-      processOperator(dependent_op_index, true);
-    }
-  }
-
-  // Dispatch the WorkerMessages to the workers. We prefer to start the search
-  // for the schedulable WorkOrders beginning from 'op_index'. The first
-  // candidate worker to receive the next WorkOrder is the one that sent the
-  // response message to Foreman.
-  dispatchWorkerMessages(worker_thread_index, op_index);
-}
-
-void Foreman::processRebuildWorkOrderCompleteMessage(const dag_node_index op_index,
-                                                     const size_t worker_thread_index) {
-  query_exec_state_->decrementNumRebuildWorkOrders(op_index);
-  workers_->decrementNumQueuedWorkOrders(worker_thread_index);
-
-  if (checkRebuildOver(op_index)) {
-    markOperatorFinished(op_index);
-
-    for (const pair<dag_node_index, bool> &dependent_link :
-         query_dag_->getDependents(op_index)) {
-      const dag_node_index dependent_op_index = dependent_link.first;
-      if (checkAllBlockingDependenciesMet(dependent_op_index)) {
-        processOperator(dependent_op_index, true);
-      }
-    }
-  }
-
-  // Dispatch the WorkerMessages to the workers. We prefer to start the search
-  // for the schedulable WorkOrders beginning from 'op_index'. The first
-  // candidate worker to receive the next WorkOrder is the one that sent the
-  // response message to Foreman.
-  dispatchWorkerMessages(worker_thread_index, op_index);
-}
-
-void Foreman::processDataPipelineMessage(const dag_node_index op_index,
-                                         const block_id block,
-                                         const relation_id rel_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);
-    // Because of the streamed input just fed, check if there are any new
-    // WorkOrders available and if so, fetch them.
-    fetchNormalWorkOrders(consumer_index);
-  }
-
-  // Dispatch the WorkerMessages to the workers. We prefer to start the search
-  // for the schedulable WorkOrders beginning from 'op_index'. The first
-  // candidate worker to receive the next WorkOrder is the one that sent the
-  // response message to Foreman.
-  // TODO(zuyu): Improve the data locality for the next WorkOrder.
-  dispatchWorkerMessages(0, op_index);
-}
-
-void Foreman::processFeedbackMessage(const WorkOrder::FeedbackMessage &msg) {
-  RelationalOperator *op =
-      query_dag_->getNodePayloadMutable(msg.header().rel_op_index);
-  op->receiveFeedbackMessage(msg);
-}
-
-void Foreman::run() {
-  // Initialize before for Foreman eventloop.
-  initialize();
 
   // Event loop
-  while (!query_exec_state_->hasQueryExecutionFinished()) {
+  for (;;) {
     // Receive() causes this thread to sleep until next message is received.
-    AnnotatedMessage annotated_msg = bus_->Receive(foreman_client_id_, 0, true);
+    const AnnotatedMessage annotated_msg =
+        bus_->Receive(foreman_client_id_, 0, true);
     const TaggedMessage &tagged_message = annotated_msg.tagged_message;
-    switch (tagged_message.message_type()) {
-      case kWorkOrderCompleteMessage: {
-        serialization::WorkOrderCompletionMessage proto;
-        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
-
-        processWorkOrderCompleteMessage(proto.operator_index(), proto.worker_thread_index());
-        break;
-      }
-      case kRebuildWorkOrderCompleteMessage: {
-        serialization::WorkOrderCompletionMessage proto;
-        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
-
-        processRebuildWorkOrderCompleteMessage(proto.operator_index(), proto.worker_thread_index());
+    const tmb::message_type_id message_type = tagged_message.message_type();
+    switch (message_type) {
+      case kCatalogRelationNewBlockMessage:  // Fall through
+      case kDataPipelineMessage:
+      case kRebuildWorkOrderCompleteMessage:
+      case kWorkOrderCompleteMessage:
+      case kWorkOrderFeedbackMessage:
+      case kWorkOrdersAvailableMessage: {
+        policy_enforcer_->processMessage(tagged_message);
         break;
       }
-      case kCatalogRelationNewBlockMessage: {
-        serialization::CatalogRelationNewBlockMessage proto;
-        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
-
-        const block_id block = proto.block_id();
-
-        CatalogRelation *relation =
-            static_cast<CatalogDatabase*>(catalog_database_)->getRelationByIdMutable(proto.relation_id());
-        relation->addBlock(block);
-
-        if (proto.has_partition_id()) {
-          relation->getPartitionSchemeMutable()->addBlockToPartition(proto.partition_id(), block);
+      case kAdmitRequestMessage: {
+        const AdmitRequestMessage *msg =
+            static_cast<const AdmitRequestMessage *>(tagged_message.message());
+        const vector<QueryHandle *> &query_handles = msg->getQueryHandles();
+
+        DCHECK(!query_handles.empty());
+        bool all_queries_admitted = true;
+        if (query_handles.size() == 1u) {
+          all_queries_admitted =
+              policy_enforcer_->admitQuery(query_handles.front());
+        } else {
+          all_queries_admitted = policy_enforcer_->admitQueries(query_handles);
+        }
+        if (!all_queries_admitted) {
+          LOG(WARNING) << "The scheduler could not admit all the queries";
+          // TODO(harshad) - Inform the main thread about the failure.
         }
         break;
       }
-      case kDataPipelineMessage: {
-        // Possible message senders include InsertDestinations and some
-        // operators which modify existing blocks.
-        serialization::DataPipelineMessage proto;
-        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
-
-        processDataPipelineMessage(proto.operator_index(), proto.block_id(), proto.relation_id());
-        break;
-      }
-      case kWorkOrdersAvailableMessage: {
-        serialization::WorkOrdersAvailableMessage proto;
-        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
-
-        const dag_node_index op_index = proto.operator_index();
-
-        // Check if new work orders are available.
-        fetchNormalWorkOrders(op_index);
-
-        // Dispatch the WorkerMessages to the workers. We prefer to start the search
-        // for the schedulable WorkOrders beginning from 'op_index'. The first
-        // candidate worker to receive the next WorkOrder is the one that sent the
-        // response message to Foreman.
-        // TODO(zuyu): Improve the data locality for the next WorkOrder.
-        dispatchWorkerMessages(0, op_index);
-        break;
-      }
-      case kWorkOrderFeedbackMessage: {
-        WorkOrder::FeedbackMessage msg(const_cast<void *>(tagged_message.message()),
-                                       tagged_message.message_bytes());
-        processFeedbackMessage(msg);
-        break;
+      case kPoisonMessage: {
+        if (policy_enforcer_->hasQueries()) {
+          LOG(WARNING) << "Foreman thread exiting while some queries are "
+                          "under execution or waiting to be admitted";
+        }
+        return;
       }
       default:
         LOG(FATAL) << "Unknown message type to Foreman";
     }
-  }
-
-  // Clean up before exiting.
-  cleanUp();
-}
 
-void Foreman::dispatchWorkerMessages(
-    const size_t start_worker_index,
-    const dag_node_index start_operator_index) {
-  // Loop over all workers. Stopping criteria:
-  // 1. Every worker has been assigned exactly max_msgs_per_worker_ workorders.
-  // OR 2. No schedulable workorders at this time.
-  size_t done_workers_count = 0;
-  for (size_t curr_worker = start_worker_index;
-       done_workers_count < workers_->getNumWorkers();
-       curr_worker = (curr_worker + 1) % workers_->getNumWorkers()) {
-    if (workers_->getNumQueuedWorkOrders(curr_worker) < max_msgs_per_worker_) {
-      std::unique_ptr<WorkerMessage> msg;
-      msg.reset(getNextWorkerMessage(
-          start_operator_index, workers_->getNUMANode(curr_worker)));
-      if (msg.get() != nullptr) {
-        sendWorkerMessage(curr_worker, *msg);
-        workers_->incrementNumQueuedWorkOrders(curr_worker);
-      } else {
-        // No schedulable workorder at this point.
-        ++done_workers_count;
-      }
-    } else {
-      // curr_worker already has been assigned max_msgs_per_worker workorders.
-      ++done_workers_count;
+    if (canCollectNewMessages(message_type)) {
+      vector<unique_ptr<WorkerMessage>> new_messages;
+      policy_enforcer_->getWorkerMessages(&new_messages);
+      dispatchWorkerMessages(new_messages);
+    }
+
+    // We check again, as some queries may produce zero work orders and finish
+    // their execution.
+    if (!policy_enforcer_->hasQueries()) {
+      // Signal the main thread that there are no queries to be executed.
+      // Currently the message doesn't have any real content.
+      const int dummy_payload = 0;
+      TaggedMessage completion_tagged_message(
+          &dummy_payload, sizeof(dummy_payload), kWorkloadCompletionMessage);
+      const tmb::MessageBus::SendStatus send_status =
+          QueryExecutionUtil::SendTMBMessage(
+              bus_,
+              foreman_client_id_,
+              main_thread_client_id_,
+              move(completion_tagged_message));
+      CHECK(send_status == tmb::MessageBus::SendStatus::kOK)
+          << "Message could not be sent from Foreman with TMB client ID "
+          << foreman_client_id_ << " to main thread with TMB client ID"
+          << main_thread_client_id_;
     }
   }
 }
 
-void Foreman::initializeState() {
-  const dag_node_index dag_size = query_dag_->size();
-
-  output_consumers_.resize(dag_size);
-  blocking_dependencies_.resize(dag_size);
-
-  query_exec_state_.reset(new QueryExecutionState(dag_size));
-  workorders_container_.reset(new WorkOrdersContainer(dag_size, num_numa_nodes_));
-
-  for (dag_node_index node_index = 0; node_index < dag_size; ++node_index) {
-    const QueryContext::insert_destination_id insert_destination_index =
-        query_dag_->getNodePayload(node_index).getInsertDestinationID();
-    if (insert_destination_index != QueryContext::kInvalidInsertDestinationId) {
-      // Rebuild is necessary whenever InsertDestination is present.
-      query_exec_state_->setRebuildRequired(node_index);
-      query_exec_state_->setRebuildStatus(node_index, 0, false);
-    }
-
-    for (const pair<dag_node_index, bool> &dependent_link :
-         query_dag_->getDependents(node_index)) {
-      const dag_node_index dependent_op_index = dependent_link.first;
-      if (!query_dag_->getLinkMetadata(node_index, dependent_op_index)) {
-        // The link is not a pipeline-breaker. Streaming of blocks is possible
-        // between these two operators.
-        output_consumers_[node_index].push_back(dependent_op_index);
-      } else {
-        // The link is a pipeline-breaker. Streaming of blocks is not possible
-        // between these two operators.
-        blocking_dependencies_[dependent_op_index].push_back(node_index);
-      }
-    }
+bool Foreman::canCollectNewMessages(const tmb::message_type_id message_type) {
+  if (QUICKSTEP_EQUALS_ANY_CONSTANT(message_type,
+                                    kCatalogRelationNewBlockMessage,
+                                    kWorkOrderFeedbackMessage)) {
+    return false;
+  } else if (worker_directory_->getLeastLoadedWorker().second <=
+             FLAGS_min_load_per_worker) {
+    // If the least loaded worker has only one pending work order, we should
+    // collect new messages and dispatch them.
+    return true;
+  } else {
+    return false;
   }
 }
 
-// TODO(harshad) : The default policy may execute remote WorkOrders for an
-// operator with a lower index even when there are local WorkOrders available for
-// an operator with higher index. We should examine if avoiding this behavior
-// has any benefits with respect to execution time and/or memory pressure.
-WorkerMessage* Foreman::getNextWorkerMessage(
-    const dag_node_index start_operator_index, const int numa_node) {
-  // Default policy: Operator with lowest index first.
-  WorkOrder *work_order = nullptr;
-  size_t num_operators_checked = 0;
-  for (dag_node_index index = start_operator_index;
-       num_operators_checked < query_dag_->size();
-       index = (index + 1) % query_dag_->size(), ++num_operators_checked) {
-    if (query_exec_state_->hasExecutionFinished(index)) {
-      continue;
-    }
-    if (numa_node != -1) {
-      // First try to get a normal WorkOrder from the specified NUMA node.
-      work_order = workorders_container_->getNormalWorkOrderForNUMANode(index, numa_node);
-      if (work_order != nullptr) {
-        // A WorkOrder found on the given NUMA node.
-        query_exec_state_->incrementNumQueuedWorkOrders(index);
-        return WorkerMessage::WorkOrderMessage(work_order, index);
-      } else {
-        // Normal workorder not found on this node. Look for a rebuild workorder
-        // on this NUMA node.
-        work_order = workorders_container_->getRebuildWorkOrderForNUMANode(index, numa_node);
-        if (work_order != nullptr) {
-          return WorkerMessage::RebuildWorkOrderMessage(work_order, index);
-        }
-      }
-    }
-    // Either no workorder found on the given NUMA node, or numa_node is -1.
-    // Try to get a normal WorkOrder from other NUMA nodes.
-    work_order = workorders_container_->getNormalWorkOrder(index);
-    if (work_order != nullptr) {
-      query_exec_state_->incrementNumQueuedWorkOrders(index);
-      return WorkerMessage::WorkOrderMessage(work_order, index);
+void Foreman::dispatchWorkerMessages(const vector<unique_ptr<WorkerMessage>> &messages) {
+  for (const auto &message : messages) {
+    DCHECK(message != nullptr);
+    const int recipient_worker_thread_index = message->getRecipientHint();
+    if (recipient_worker_thread_index != WorkerMessage::kInvalidRecipientIndexHint) {
+      sendWorkerMessage(static_cast<size_t>(recipient_worker_thread_index),
+                        *message);
     } else {
-      // Normal WorkOrder not found, look for a RebuildWorkOrder.
-      work_order = workorders_container_->getRebuildWorkOrder(index);
-      if (work_order != nullptr) {
-        return WorkerMessage::RebuildWorkOrderMessage(work_order, index);
-      }
+      sendWorkerMessage(worker_directory_->getLeastLoadedWorker().first,
+                        *message);
     }
   }
-  // No WorkOrders available right now.
-  return nullptr;
 }
 
-void Foreman::sendWorkerMessage(const std::size_t worker_thread_index,
+void Foreman::sendWorkerMessage(const size_t worker_thread_index,
                                 const WorkerMessage &message) {
-  message_type_id type;
+  tmb::message_type_id type;
   if (message.getType() == WorkerMessage::WorkerMessageType::kRebuildWorkOrder) {
     type = kRebuildWorkOrderMessage;
   } else if (message.getType() == WorkerMessage::WorkerMessageType::kWorkOrder) {
@@ -383,152 +217,12 @@ void Foreman::sendWorkerMessage(const std::size_t worker_thread_index,
   const tmb::MessageBus::SendStatus send_status =
       QueryExecutionUtil::SendTMBMessage(bus_,
                                          foreman_client_id_,
-                                         workers_->getClientID(worker_thread_index),
+                                         worker_directory_->getClientID(worker_thread_index),
                                          move(worker_tagged_message));
   CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
       "Message could not be sent from Foreman with TMB client ID "
       << foreman_client_id_ << " to Foreman with TMB client ID "
-      << workers_->getClientID(worker_thread_index);
-}
-
-bool Foreman::fetchNormalWorkOrders(const dag_node_index index) {
-  bool generated_new_workorders = false;
-  if (!query_exec_state_->hasDoneGenerationWorkOrders(index)) {
-    // Do not fetch any work units until all blocking dependencies are met.
-    // The releational operator is not aware of blocking dependencies for
-    // uncorrelated scalar queries.
-    if (!checkAllBlockingDependenciesMet(index)) {
-      return false;
-    }
-    const size_t num_pending_workorders_before =
-        workorders_container_->getNumNormalWorkOrders(index);
-    const bool done_generation =
-        query_dag_->getNodePayloadMutable(index)->getAllWorkOrders(workorders_container_.get(),
-                                                                   query_context_.get(),
-                                                                   storage_manager_,
-                                                                   foreman_client_id_,
-                                                                   bus_);
-    if (done_generation) {
-      query_exec_state_->setDoneGenerationWorkOrders(index);
-    }
-
-    // TODO(shoban): It would be a good check to see if operator is making
-    // useful progress, i.e., the operator either generates work orders to
-    // execute or still has pending work orders executing. However, this will not
-    // work if Foreman polls operators without feeding data. This check can be
-    // enabled, if Foreman is refactored to call getAllWorkOrders() only when
-    // pending work orders are completed or new input blocks feed.
-
-    generated_new_workorders =
-        (num_pending_workorders_before <
-         workorders_container_->getNumNormalWorkOrders(index));
-  }
-  return generated_new_workorders;
-}
-
-void Foreman::processOperator(const dag_node_index index,
-                              const bool recursively_check_dependents) {
-  if (fetchNormalWorkOrders(index)) {
-    // Fetched work orders. Return to wait for the generated work orders to
-    // execute, and skip the execution-finished checks.
-    return;
-  }
-
-  if (checkNormalExecutionOver(index)) {
-    if (checkRebuildRequired(index)) {
-      if (!checkRebuildInitiated(index)) {
-        // Rebuild hasn't started, initiate it.
-        if (initiateRebuild(index)) {
-          // Rebuild initiated and completed right away.
-          markOperatorFinished(index);
-        } else {
-          // Rebuild WorkOrders have been generated.
-          return;
-        }
-      } else if (checkRebuildOver(index)) {
-        // Rebuild had been initiated and it is over.
-        markOperatorFinished(index);
-      }
-    } else {
-      // Rebuild is not required and normal execution over, mark finished.
-      markOperatorFinished(index);
-    }
-    // If we reach here, that means the operator has been marked as finished.
-    if (recursively_check_dependents) {
-      for (const pair<dag_node_index, bool> &dependent_link :
-           query_dag_->getDependents(index)) {
-        const dag_node_index dependent_op_index = dependent_link.first;
-        if (checkAllBlockingDependenciesMet(dependent_op_index)) {
-          processOperator(dependent_op_index, true);
-        }
-      }
-    }
-  }
-}
-
-void Foreman::markOperatorFinished(const dag_node_index index) {
-  query_exec_state_->setExecutionFinished(index);
-
-  RelationalOperator *op = query_dag_->getNodePayloadMutable(index);
-  op->updateCatalogOnCompletion();
-
-  const relation_id output_rel = op->getOutputRelationID();
-  for (const pair<dag_node_index, bool> &dependent_link : query_dag_->getDependents(index)) {
-    const dag_node_index dependent_op_index = dependent_link.first;
-    RelationalOperator *dependent_op = query_dag_->getNodePayloadMutable(dependent_op_index);
-    // Signal dependent operator that current operator is done feeding input blocks.
-    if (output_rel >= 0) {
-      dependent_op->doneFeedingInputBlocks(output_rel);
-    }
-    if (checkAllBlockingDependenciesMet(dependent_op_index)) {
-      dependent_op->informAllBlockingDependenciesMet();
-    }
-  }
-}
-
-bool Foreman::initiateRebuild(const dag_node_index index) {
-  DEBUG_ASSERT(!workorders_container_->hasRebuildWorkOrder(index));
-  DEBUG_ASSERT(checkRebuildRequired(index));
-  DEBUG_ASSERT(!checkRebuildInitiated(index));
-
-  getRebuildWorkOrders(index, workorders_container_.get());
-
-  query_exec_state_->setRebuildStatus(
-      index, workorders_container_->getNumRebuildWorkOrders(index), true);
-
-  return (query_exec_state_->getNumRebuildWorkOrders(index) == 0);
-}
-
-void Foreman::getRebuildWorkOrders(const dag_node_index index, WorkOrdersContainer *container) {
-  const RelationalOperator &op = query_dag_->getNodePayload(index);
-  const QueryContext::insert_destination_id insert_destination_index = op.getInsertDestinationID();
-
-  if (insert_destination_index == QueryContext::kInvalidInsertDestinationId) {
-    return;
-  }
-
-  vector<MutableBlockReference> partially_filled_block_refs;
-
-  DCHECK(query_context_ != nullptr);
-  InsertDestination *insert_destination = query_context_->getInsertDestination(insert_destination_index);
-  DCHECK(insert_destination != nullptr);
-
-  insert_destination->getPartiallyFilledBlocks(&partially_filled_block_refs);
-
-  for (vector<MutableBlockReference>::size_type i = 0;
-       i < partially_filled_block_refs.size();
-       ++i) {
-    // Note: The query ID used below is dummy for now, it will be replaced with
-    // the true query ID when QueryManager gets used in Foreman.
-    container->addRebuildWorkOrder(
-        new RebuildWorkOrder(0,
-                             move(partially_filled_block_refs[i]),
-                             index,
-                             op.getOutputRelationID(),
-                             foreman_client_id_,
-                             bus_),
-        index);
-  }
+      << worker_directory_->getClientID(worker_thread_index);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/Foreman.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.hpp b/query_execution/Foreman.hpp
index 2d6e0d3..94cb9fc 100644
--- a/query_execution/Foreman.hpp
+++ b/query_execution/Foreman.hpp
@@ -22,22 +22,11 @@
 #include <memory>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/ForemanLite.hpp"
-#include "query_execution/QueryContext.hpp"
-#include "query_execution/QueryExecutionState.hpp"
-#include "query_execution/QueryExecutionTypedefs.hpp"
-#include "query_execution/WorkOrdersContainer.hpp"
-#include "query_execution/WorkerMessage.hpp"
-#include "relational_operators/RelationalOperator.hpp"
-#include "relational_operators/WorkOrder.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "utility/DAG.hpp"
+#include "query_execution/PolicyEnforcer.hpp"
 #include "utility/Macros.hpp"
 
-#include "glog/logging.h"
-#include "gtest/gtest_prod.h"
-
+#include "tmb/id_typedefs.h"
 #include "tmb/message_bus.h"
 
 namespace quickstep {
@@ -45,23 +34,24 @@ namespace quickstep {
 class CatalogDatabaseLite;
 class StorageManager;
 class WorkerDirectory;
-
-namespace serialization { class QueryContext; }
+class WorkerMessage;
 
 /** \addtogroup QueryExecution
  *  @{
  */
 
 /**
- * @brief The Foreman scans the query DAG, requests each operator to produce
- *        workorders. It also pipelines the intermediate output it receives to
- *        the relational operators which need it.
+ * @brief The Foreman receives queries from the main thread, messages from the
+ *        policy enforcer and dispatches the work to worker threads. It also
+ *        receives work completion messages from workers.
  **/
 class Foreman final : public ForemanLite {
  public:
   /**
    * @brief Constructor.
    *
+   * @param main_thread_client_id The TMB client ID of the main thread.
+   * @param worker_directory The worker directory.
    * @param bus A pointer to the TMB.
    * @param catalog_database The catalog database where this query is executed.
    * @param storage_manager The StorageManager to use.
@@ -71,233 +61,28 @@ class Foreman final : public ForemanLite {
    * @note If cpu_id is not specified, Foreman thread can be possibly moved
    *       around on different CPUs by the OS.
   **/
-  Foreman(tmb::MessageBus *bus,
+  Foreman(const tmb::client_id main_thread_client_id,
+          WorkerDirectory *worker_directory,
+          tmb::MessageBus *bus,
           CatalogDatabaseLite *catalog_database,
           StorageManager *storage_manager,
           const int cpu_id = -1,
-          const int num_numa_nodes = 1)
-      : ForemanLite(bus, cpu_id),
-        catalog_database_(DCHECK_NOTNULL(catalog_database)),
-        storage_manager_(DCHECK_NOTNULL(storage_manager)),
-        max_msgs_per_worker_(1),
-        num_numa_nodes_(num_numa_nodes) {
-    bus_->RegisterClientAsSender(foreman_client_id_, kWorkOrderMessage);
-    bus_->RegisterClientAsSender(foreman_client_id_, kRebuildWorkOrderMessage);
-    // NOTE : Foreman thread sends poison messages in the optimizer's
-    // ExecutionGeneratorTest.
-    bus_->RegisterClientAsSender(foreman_client_id_, kPoisonMessage);
-
-    bus_->RegisterClientAsReceiver(foreman_client_id_,
-                                   kWorkOrderCompleteMessage);
-    bus_->RegisterClientAsReceiver(foreman_client_id_,
-                                   kRebuildWorkOrderCompleteMessage);
-    bus_->RegisterClientAsReceiver(foreman_client_id_, kCatalogRelationNewBlockMessage);
-    bus_->RegisterClientAsReceiver(foreman_client_id_, kDataPipelineMessage);
-    bus_->RegisterClientAsReceiver(foreman_client_id_,
-                                   kWorkOrdersAvailableMessage);
-    bus_->RegisterClientAsReceiver(foreman_client_id_,
-                                   kWorkOrderFeedbackMessage);
-  }
+          const std::size_t num_numa_nodes = 1);
 
   ~Foreman() override {}
 
-  /**
-   * @brief Set the Query plan DAG for the query to be executed.
-   *
-   * @param query_plan_dag A pointer to the query plan DAG.
-   **/
-  inline void setQueryPlan(DAG<RelationalOperator, bool> *query_plan_dag) {
-    query_dag_ = query_plan_dag;
-  }
-
-  /**
-   * @brief Reconstruct the QueryContext for the query to be executed.
-   *
-   * @param proto The serialized QueryContext.
-   **/
-  inline void reconstructQueryContextFromProto(const serialization::QueryContext &proto) {
-    query_context_.reset(
-        new QueryContext(proto, *catalog_database_, storage_manager_, foreman_client_id_, bus_));
-  }
-
-  /**
-   * @brief Set the WorkerDirectory pointer.
-   *
-   * @param workers A pointer to the WorkerDirectory.
-   **/
-  void setWorkerDirectory(WorkerDirectory *workers) {
-    workers_ = workers;
-  }
-
-  /**
-   * @brief Set the maximum number of messages that should be allocated to each
-   *        worker during a single round of WorkOrder dispatch.
-   *
-   * @param max_msgs_per_worker Maximum number of messages.
-   **/
-  void setMaxMessagesPerWorker(const std::size_t max_msgs_per_worker) {
-    max_msgs_per_worker_ = max_msgs_per_worker;
-  }
-
  protected:
-  /**
-   * @brief The foreman receives a DAG of relational operators, asks relational
-   *        operators to produce the workorders and based on the response it gets
-   *        pipelines the intermediate output to dependent relational operators.
-   *
-   * @note  The workers who get the messages from the Foreman execute and
-   *        subsequently delete the WorkOrder contained in the message.
-   **/
   void run() override;
 
  private:
-  typedef DAG<RelationalOperator, bool>::size_type_nodes dag_node_index;
-
-  /**
-   * @brief Check if all the dependencies of the node at specified index have
-   *        finished their execution.
-   *
-   * @note This function's true return value is a pre-requisite for calling
-   *       getRebuildWorkOrders()
-   *
-   * @param node_index The index of the specified node in the query DAG.
-   *
-   * @return True if all the dependencies have finished their execution. False
-   *         otherwise.
-   **/
-  inline bool checkAllDependenciesMet(const dag_node_index node_index) const {
-    for (const dag_node_index dependency_index : query_dag_->getDependencies(node_index)) {
-      // If at least one of the dependencies is not met, return false.
-      if (!query_exec_state_->hasExecutionFinished(dependency_index)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /**
-   * @brief Check if all the blocking dependencies of the node at specified
-   *        index have finished their execution.
-   *
-   * @note A blocking dependency is the one which is pipeline breaker. Output of
-   *       a dependency can't be streamed to its dependent if the link between
-   *       them is pipeline breaker.
-   *
-   * @param node_index The index of the specified node in the query DAG.
-   *
-   * @return True if all the blocking dependencies have finished their
-   *         execution. False otherwise.
-   **/
-  inline bool checkAllBlockingDependenciesMet(const dag_node_index node_index) const {
-    for (const dag_node_index blocking_dependency_index : blocking_dependencies_[node_index]) {
-      if (!query_exec_state_->hasExecutionFinished(blocking_dependency_index)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
   /**
    * @brief Dispatch schedulable WorkOrders, wrapped in WorkerMessages to the
    *        worker threads.
    *
-   * @param start_worker_index The dispatch of WorkOrders preferably begins with
-   *        the worker at this index.
-   * @param start_operator_index The search for a schedulable WorkOrder
-   *        begins with the WorkOrders generated by this operator.
-   **/
-  void dispatchWorkerMessages(const std::size_t start_worker_index,
-                              const dag_node_index start_operator_index);
-
-  /**
-   * @brief Initialize all the local vectors and maps. If the operator has an
-   *        InsertDestination, pass the bus address and Foreman's TMB client ID
-   *        to it.
-   **/
-  void initializeState();
-
-  /**
-   * @brief Initialize the Foreman before starting the event loop. This binds
-   * the Foreman thread to configured CPU, and does initial processing of
-   * operator before waiting for events from Workers.
-   **/
-  void initialize();
-
-  /**
-   * @brief Process the received WorkOrder complete message.
-   *
-   * @param node_index The index of the specified operator node in the query DAG
-   *        for the completed WorkOrder.
-   * @param worker_thread_index The logical index of the worker thread in
-   *        WorkerDirectory for the completed WorkOrder.
-   **/
-  void processWorkOrderCompleteMessage(const dag_node_index op_index,
-                                       const std::size_t worker_thread_index);
-
-  /**
-   * @brief Process the received RebuildWorkOrder complete message.
-   *
-   * @param node_index The index of the specified operator node in the query DAG
-   *        for the completed RebuildWorkOrder.
-   * @param worker_thread_index The logical index of the worker thread in
-   *        WorkerDirectory for the completed RebuildWorkOrder.
-   **/
-  void processRebuildWorkOrderCompleteMessage(const dag_node_index op_index,
-                                              const std::size_t worker_thread_index);
-
-  /**
-   * @brief Process the received data pipeline message.
-   *
-   * @param node_index The index of the specified operator node in the query DAG
-   *        for the pipelining block.
-   * @param block The block id.
-   * @param rel_id The ID of the relation that produced 'block'.
-   **/
-  void processDataPipelineMessage(const dag_node_index op_index,
-                                  const block_id block,
-                                  const relation_id rel_id);
-
-  /**
-   * @brief Process the received work order feedback message and notify relational
-   *        operator.
-   *
-   * @param message Feedback message from work order.
-   **/
-  void processFeedbackMessage(const WorkOrder::FeedbackMessage &message);
-
-  /**
-   * @brief Clear some of the vectors used for a single run of a query.
-   **/
-  void cleanUp() {
-    output_consumers_.clear();
-    blocking_dependencies_.clear();
-  }
-
-  /**
-   * @brief Process a current relational operator: Get its workorders and store
-   *        them in the WorkOrdersContainer for this query. If the operator can
-   *        be marked as done, do so.
-   *
-   * @param index The index of the relational operator to be processed in the
-   *        query plan DAG.
-   * @param recursively_check_dependents If an operator is done, should we
-   *        call processOperator on its dependents recursively.
+   * @param messages The messages to be dispatched.
    **/
-  void processOperator(const dag_node_index index, const bool recursively_check_dependents);
-
- /**
-   * @brief Get the next workorder to be excuted, wrapped in a WorkerMessage.
-   *
-   * @param start_operator_index Begin the search for the schedulable WorkOrder
-   *        with the operator at this index.
-   * @param numa_node The next WorkOrder should preferably have its input(s)
-   *        from this numa_node. This is a hint and not a binding requirement.
-   *
-   * @return A pointer to the WorkerMessage. If there's no WorkOrder to be
-   *         executed, return NULL.
-   **/
-  WorkerMessage* getNextWorkerMessage(
-      const dag_node_index start_operator_index, const int numa_node = -1);
+  void dispatchWorkerMessages(
+      const std::vector<std::unique_ptr<WorkerMessage>> &messages);
 
   /**
    * @brief Send the given message to the specified worker.
@@ -306,156 +91,24 @@ class Foreman final : public ForemanLite {
    *        in WorkerDirectory.
    * @param message The WorkerMessage to be sent.
    **/
-  void sendWorkerMessage(const std::size_t worker_thread_index, const WorkerMessage &message);
-
-  /**
-   * @brief Fetch all work orders currently available in relational operator and
-   *        store them internally.
-   *
-   * @param index The index of the relational operator to be processed in the
-   *        query plan DAG.
-   *
-   * @return Whether any work order was generated by op.
-   **/
-  bool fetchNormalWorkOrders(const dag_node_index index);
-
-  /**
-   * @brief This function does the following things:
-   *        1. Mark the given relational operator as "done".
-   *        2. For all the dependents of this operator, check if all of their
-   *        blocking dependencies are met. If so inform them that the blocking
-   *        dependencies are met.
-   *        3. Check if the given operator is done producing output. If it's
-   *        done, inform the dependents that they won't receive input anymore
-   *        from the given operator.
-   *
-   * @param index The index of the given relational operator in the DAG.
-   **/
-  void markOperatorFinished(const dag_node_index index);
-
-  /**
-   * @brief Check if the execution of the given operator is over.
-   *
-   * @param index The index of the given operator in the DAG.
-   *
-   * @return True if the execution of the given operator is over, false
-   *         otherwise.
-   **/
-  inline bool checkOperatorExecutionOver(const dag_node_index index) const {
-    if (checkRebuildRequired(index)) {
-      return (checkNormalExecutionOver(index) && checkRebuildOver(index));
-    } else {
-      return checkNormalExecutionOver(index);
-    }
-  }
-
-  /**
-   * @brief Check if the given operator's normal execution is over.
-   *
-   * @note The conditions for a given operator's normal execution to get over:
-   *       1. All of its  normal (i.e. non rebuild) WorkOrders have finished
-   *       execution.
-   *       2. The operator is done generating work orders.
-   *       3. All of the dependencies of the given operator have been met.
-   *
-   * @param index The index of the given operator in the DAG.
-   *
-   * @return True if the normal execution of the given operator is over, false
-   *         otherwise.
-   **/
-  inline bool checkNormalExecutionOver(const dag_node_index index) const {
-    return (checkAllDependenciesMet(index) &&
-            !workorders_container_->hasNormalWorkOrder(index) &&
-            query_exec_state_->getNumQueuedWorkOrders(index) == 0 &&
-            query_exec_state_->hasDoneGenerationWorkOrders(index));
-  }
+  void sendWorkerMessage(const std::size_t worker_thread_index,
+                         const WorkerMessage &message);
 
   /**
-   * @brief Check if the rebuild operation is required for a given operator.
-   *
-   * @param index The index of the given operator in the DAG.
+   * @brief Check if we can collect new messages from the PolicyEnforcer.
    *
-   * @return True if the rebuild operation is required, false otherwise.
+   * @param message_type The type of the last received message.
    **/
-  inline bool checkRebuildRequired(const dag_node_index index) const {
-    return query_exec_state_->isRebuildRequired(index);
-  }
+  bool canCollectNewMessages(const tmb::message_type_id message_type);
 
-  /**
-   * @brief Check if the rebuild operation for a given operator is over.
-   *
-   * @param index The index of the given operator in the DAG.
-   *
-   * @return True if the rebuild operation is over, false otherwise.
-   **/
-  inline bool checkRebuildOver(const dag_node_index index) const {
-    return query_exec_state_->hasRebuildInitiated(index) &&
-           !workorders_container_->hasRebuildWorkOrder(index) &&
-           (query_exec_state_->getNumRebuildWorkOrders(index) == 0);
-  }
+  const tmb::client_id main_thread_client_id_;
 
-  /**
-   * @brief Check if the rebuild operation for a given operator has been
-   *        initiated.
-   *
-   * @param index The index of the given operator in the DAG.
-   *
-   * @return True if the rebuild operation has been initiated, false otherwise.
-   **/
-  inline bool checkRebuildInitiated(const dag_node_index index) const {
-    return query_exec_state_->hasRebuildInitiated(index);
-  }
-
-  /**
-   * @brief Initiate the rebuild process for partially filled blocks generated
-   *        during the execution of the given operator.
-   *
-   * @param index The index of the given operator in the DAG.
-   *
-   * @return True if the rebuild is over immediately, i.e. the operator didn't
-   *         generate any rebuild WorkOrders, false otherwise.
-   **/
-  bool initiateRebuild(const dag_node_index index);
-
-  /**
-   * @brief Get the rebuild WorkOrders for an operator.
-   *
-   * @note This function should be called only once, when all the normal
-   *       WorkOrders generated by an operator finish their execution.
-   *
-   * @param index The index of the operator in the query plan DAG.
-   * @param container A pointer to a WorkOrdersContainer to be used to store the
-   *        generated WorkOrders.
-   **/
-  void getRebuildWorkOrders(const dag_node_index index, WorkOrdersContainer *container);
+  WorkerDirectory *worker_directory_;
 
   CatalogDatabaseLite *catalog_database_;
   StorageManager *storage_manager_;
 
-  DAG<RelationalOperator, bool> *query_dag_;
-
-  std::unique_ptr<QueryContext> query_context_;
-
-  // During a single round of WorkOrder dispatch, a Worker should be allocated
-  // at most these many WorkOrders.
-  std::size_t max_msgs_per_worker_;
-
-  // For all nodes, store their receiving dependents.
-  std::vector<std::vector<dag_node_index>> output_consumers_;
-
-  // For all nodes, store their pipeline breaking dependencies (if any).
-  std::vector<std::vector<dag_node_index>> blocking_dependencies_;
-
-  std::unique_ptr<QueryExecutionState> query_exec_state_;
-
-  std::unique_ptr<WorkOrdersContainer> workorders_container_;
-
-  const int num_numa_nodes_;
-
-  WorkerDirectory *workers_;
-
-  friend class ForemanTest;
-  FRIEND_TEST(ForemanTest, TwoNodesDAGPartiallyFilledBlocksTest);
+  std::unique_ptr<PolicyEnforcer> policy_enforcer_;
 
   DISALLOW_COPY_AND_ASSIGN(Foreman);
 };


[12/13] incubator-quickstep git commit: Part I of a sequence of changes to bring the NOTICE and Copyright in line with ASF guidelines

Posted by zu...@apache.org.
Part I of a sequence of changes to bring the NOTICE and Copyright in line with ASF guidelines


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

Branch: refs/heads/travis-grpc
Commit: dd4495899e22437b9d64bbcb65edcbc7e25ce409
Parents: 29768a7
Author: Jignesh Patel <jm...@hotmail.com>
Authored: Tue Jun 14 12:31:19 2016 -0500
Committer: Jignesh Patel <jm...@hotmail.com>
Committed: Tue Jun 14 12:31:19 2016 -0500

----------------------------------------------------------------------
 NOTICE | 19 ++++++-------------
 1 file changed, 6 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/dd449589/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index 6395645..9cfd585 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,15 +1,8 @@
-QUICKSTEP
+Apache Quickstep (incubating)
+Copyright 2016 The Apache Software Foundation.
 
-Copyright (c) 2011-2016 Pivotal Software, Inc. All Rights Reserved.
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
 
-Licensed under the Apache License, Version 2.0 (the "License");
-you may not use this file except in compliance with the License.
-You may obtain a copy of the License at
-
-   http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
+Portions Copyright (c) 2011-2015, Quickstep Technologies, LLC.
+Portions Copyright (c) 2015-2016, Pivotal Software, Inc.


[05/13] incubator-quickstep git commit: Long lived Foreman thread

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp b/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
index 8352d55..9204073 100644
--- a/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
+++ b/query_optimizer/tests/ExecutionGeneratorTestRunner.hpp
@@ -61,9 +61,11 @@ class ExecutionGeneratorTestRunner : public TextBasedTestRunner {
 
     bus_.Initialize();
 
-    foreman_.reset(new Foreman(&bus_,
-                               test_database_loader_.catalog_database(),
-                               test_database_loader_.storage_manager()));
+    main_thread_client_id_ = bus_.Connect();
+    bus_.RegisterClientAsSender(main_thread_client_id_, kAdmitRequestMessage);
+    bus_.RegisterClientAsSender(main_thread_client_id_, kPoisonMessage);
+    bus_.RegisterClientAsReceiver(main_thread_client_id_, kWorkloadCompletionMessage);
+
     worker_.reset(new Worker(0, &bus_));
 
     std::vector<client_id> worker_client_ids;
@@ -75,27 +77,20 @@ class ExecutionGeneratorTestRunner : public TextBasedTestRunner {
 
     workers_.reset(new WorkerDirectory(1 /* number of workers */,
                                        worker_client_ids, numa_nodes));
-    foreman_->setWorkerDirectory(workers_.get());
+    foreman_.reset(new Foreman(main_thread_client_id_,
+                               workers_.get(),
+                               &bus_,
+                               test_database_loader_.catalog_database(),
+                               test_database_loader_.storage_manager()));
 
+    foreman_->start();
     worker_->start();
   }
 
   ~ExecutionGeneratorTestRunner() {
-    std::unique_ptr<WorkerMessage> poison_message(WorkerMessage::PoisonMessage());
-    TaggedMessage poison_tagged_message(poison_message.get(),
-                                        sizeof(*poison_message),
-                                        quickstep::kPoisonMessage);
-
-    Address worker_address;
-    MessageStyle single_receiver_style;
-
-    worker_address.AddRecipient(worker_->getBusClientID());
-    bus_.Send(foreman_->getBusClientID(),
-              worker_address,
-              single_receiver_style,
-              std::move(poison_tagged_message));
-
+    QueryExecutionUtil::BroadcastPoisonMessage(main_thread_client_id_, &bus_);
     worker_->join();
+    foreman_->join();
   }
 
   void runTestCase(const std::string &input,
@@ -112,6 +107,8 @@ class ExecutionGeneratorTestRunner : public TextBasedTestRunner {
 
   std::unique_ptr<WorkerDirectory> workers_;
 
+  tmb::client_id main_thread_client_id_;
+
   // This map is needed for InsertDestination and some operators that send
   // messages to Foreman directly. To know the reason behind the design of this
   // map, see the note in InsertDestination.hpp.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_optimizer/tests/TestDatabaseLoader.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/TestDatabaseLoader.cpp b/query_optimizer/tests/TestDatabaseLoader.cpp
index 2de69b6..764ff2f 100644
--- a/query_optimizer/tests/TestDatabaseLoader.cpp
+++ b/query_optimizer/tests/TestDatabaseLoader.cpp
@@ -122,6 +122,7 @@ void TestDatabaseLoader::loadTestRelation() {
                                          nullptr,
                                          &storage_manager_,
                                          0 /* dummy op index */,
+                                         0,  // dummy query ID.
                                          scheduler_client_id_,
                                          &bus_);
   int sign = 1;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.cpp b/relational_operators/DeleteOperator.cpp
index 47e36e9..933918b 100644
--- a/relational_operators/DeleteOperator.cpp
+++ b/relational_operators/DeleteOperator.cpp
@@ -132,6 +132,7 @@ void DeleteWorkOrder::execute() {
   proto.set_operator_index(delete_operator_index_);
   proto.set_block_id(input_block_id_);
   proto.set_relation_id(input_relation_.getID());
+  proto.set_query_id(query_id_);
 
   // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
   const std::size_t proto_length = proto.ByteSize();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index aa8a688..74da8c1 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -174,6 +174,7 @@ class DeleteWorkOrder : public WorkOrder {
   StorageManager *storage_manager_;
 
   const std::size_t delete_operator_index_;
+
   const tmb::client_id scheduler_client_id_;
   MessageBus *bus_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 6f4271d..9762f04 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -644,14 +644,14 @@ 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 hash_table The JoinHashTable to use.
+   * @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
    *        tuples, and the resulting value is inserted into the join result.
    * @param is_selection_on_build Whether each Scalar in the \p selection vector
    *        is using attributes from the build relation as input. Note that the
    *        length of this vector should equal the length of \p selection.
-   * @param lookup_block_id The block id of the probe_relation.
+   * @param hash_table The JoinHashTable to use.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index 86f8eaf..3125447 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -85,6 +85,7 @@ class RebuildWorkOrder : public WorkOrder {
     proto.set_operator_index(input_operator_index_);
     proto.set_block_id(block_ref_->getID());
     proto.set_relation_id(input_relation_id_);
+    proto.set_query_id(query_id_);
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const std::size_t proto_length = proto.ByteSize();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/SortMergeRunOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.cpp b/relational_operators/SortMergeRunOperator.cpp
index e398d62..1603b78 100644
--- a/relational_operators/SortMergeRunOperator.cpp
+++ b/relational_operators/SortMergeRunOperator.cpp
@@ -327,6 +327,7 @@ void SortMergeRunWorkOrder::execute() {
 
   // Send completion message to operator.
   FeedbackMessage msg(SortMergeRunOperator::kRunOutputMessage,
+                      getQueryID(),
                       operator_index_,
                       serialized_output.first,
                       serialized_output.second);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.cpp b/relational_operators/UpdateOperator.cpp
index 1b2979e..f103b0e 100644
--- a/relational_operators/UpdateOperator.cpp
+++ b/relational_operators/UpdateOperator.cpp
@@ -114,6 +114,7 @@ void UpdateWorkOrder::execute() {
   proto.set_operator_index(update_operator_index_);
   proto.set_block_id(input_block_id_);
   proto.set_relation_id(relation_.getID());
+  proto.set_query_id(query_id_);
 
   // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
   const std::size_t proto_length = proto.ByteSize();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index f6c5053..4471a17 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -184,6 +184,7 @@ class UpdateWorkOrder : public WorkOrder {
   StorageManager *storage_manager_;
 
   const std::size_t update_operator_index_;
+
   const tmb::client_id scheduler_client_id_;
   MessageBus *bus_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index 059865d..df195cc 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -65,20 +65,25 @@ class WorkOrder {
    *       relational operator.
    */
   struct FeedbackMessageHeader {
+    std::size_t query_id;
     std::size_t rel_op_index;
     std::size_t payload_size;
     FeedbackMessageType payload_type;
 
     /**
      * @brief Header constructor.
+     *
+     * @param query_id The ID of the query.
      * @param relational_op_index Index of the relation operator.
      * @param payload_size Size of the payload of the message.
      * @param payload_type Type of payload.
      */
-    FeedbackMessageHeader(const std::size_t relational_op_index,
+    FeedbackMessageHeader(const std::size_t query_id,
+                          const std::size_t relational_op_index,
                           const std::size_t payload_size,
                           const FeedbackMessageType payload_type)
-        : rel_op_index(relational_op_index),
+        : query_id(query_id),
+          rel_op_index(relational_op_index),
           payload_size(payload_size),
           payload_type(payload_type) {}
   };
@@ -93,17 +98,19 @@ class WorkOrder {
      * @brief Feedback message constructor.
      *
      * @param type Type of the message.
+     * @param query_id The ID of the query.
      * @param rel_op_index Relational operator index.
      * @param payload Blob of payload.
      * @param payload_size Size of the payload blob.
      * @param ownership Whether to take ownership of the payload blob.
      */
     FeedbackMessage(const FeedbackMessageType type,
+                    const std::size_t query_id,
                     const std::size_t rel_op_index,
                     void *payload,
                     const std::size_t payload_size,
                     const bool ownership = true)
-        : header_(rel_op_index, payload_size, type),
+        : header_(query_id, rel_op_index, payload_size, type),
           payload_(payload),
           ownership_(ownership) {}
 
@@ -285,6 +292,13 @@ class WorkOrder {
         " receiver thread with TMB client ID " << receiver_id;
   }
 
+  /**
+   * @brief Get the ID of the query which this WorkOder belongs to.
+   **/
+  inline const std::size_t getQueryID() const {
+    return query_id_;
+  }
+
  protected:
   /**
    * @brief Constructor.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index fdcc54f..fd4692a 100644
--- a/relational_operators/tests/AggregationOperator_unittest.cpp
+++ b/relational_operators/tests/AggregationOperator_unittest.cpp
@@ -228,6 +228,8 @@ class AggregationOperatorTest : public ::testing::Test {
 
     // Setup the aggregation state proto in the query context proto.
     serialization::QueryContext query_context_proto;
+    query_context_proto.set_query_id(0);  // dummy query ID.
+
     const QueryContext::aggregation_state_id aggr_state_index = query_context_proto.aggregation_states_size();
     serialization::AggregationOperationState *aggr_state_proto = query_context_proto.add_aggregation_states();
     aggr_state_proto->set_relation_id(table_->getID());
@@ -319,6 +321,8 @@ class AggregationOperatorTest : public ::testing::Test {
 
     // Setup the aggregation state proto in the query context proto.
     serialization::QueryContext query_context_proto;
+    query_context_proto.set_query_id(0);  // dummy query ID.
+
     const QueryContext::aggregation_state_id aggr_state_index = query_context_proto.aggregation_states_size();
     serialization::AggregationOperationState *aggr_state_proto = query_context_proto.add_aggregation_states();
     aggr_state_proto->set_relation_id(table_->getID());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 074b603..9c34170 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -294,6 +294,7 @@ class HashJoinOperatorTest : public ::testing::TestWithParam<HashTableImplType>
 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.
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -434,6 +435,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.
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -604,6 +606,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.
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -739,6 +742,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.
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -906,6 +910,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.
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();
@@ -1083,6 +1088,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.
 
   const QueryContext::join_hash_table_id join_hash_table_index =
       query_context_proto.join_hash_tables_size();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index fc10671..46bce5b 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -189,6 +189,7 @@ class RunTest : public ::testing::Test {
                                        nullptr,
                                        storage_manager_.get(),
                                        kOpIndex,
+                                       0,  // dummy query ID.
                                        foreman_client_id_,
                                        &bus_));
   }
@@ -433,6 +434,7 @@ class RunMergerTest : public ::testing::Test {
                                        nullptr,
                                        storage_manager_.get(),
                                        kOpIndex,
+                                       0,  // dummy query ID.
                                        foreman_client_id_,
                                        &bus_));
   }
@@ -1269,6 +1271,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
     ASSERT_EQ(null_col3_, result_table_->getAttributeByName("null-col-3")->getID());
     ASSERT_EQ(tid_col_, result_table_->getAttributeByName("tid")->getID());
 
+    query_context_proto_.set_query_id(0);  // dummy query ID.
+
     // Setup the InsertDestination proto in the query context proto.
     insert_destination_index_ = query_context_proto_.insert_destinations_size();
     serialization::InsertDestination *insert_destination_proto = query_context_proto_.add_insert_destinations();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index 71a80e4..bd682c2 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -328,6 +328,7 @@ class SortRunGenerationOperatorTest : public ::testing::Test {
                    const std::vector<bool> &null_ordering) {
     // Setup the InsertDestination proto in the query context proto.
     serialization::QueryContext query_context_proto;
+    query_context_proto.set_query_id(0);  // dummy query ID.
 
     const QueryContext::insert_destination_id insert_destination_index =
         query_context_proto.insert_destinations_size();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/relational_operators/tests/TextScanOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/TextScanOperator_unittest.cpp b/relational_operators/tests/TextScanOperator_unittest.cpp
index 5860745..5bcbee5 100644
--- a/relational_operators/tests/TextScanOperator_unittest.cpp
+++ b/relational_operators/tests/TextScanOperator_unittest.cpp
@@ -180,6 +180,7 @@ TEST_F(TextScanOperatorTest, ScanTest) {
 
   // Setup the InsertDestination proto in the query context proto.
   serialization::QueryContext query_context_proto;
+  query_context_proto.set_query_id(0);  // dummy query ID.
 
   QueryContext::insert_destination_id output_destination_index = query_context_proto.insert_destinations_size();
   serialization::InsertDestination *output_destination_proto = query_context_proto.add_insert_destinations();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index 354bed4..2866c5f 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -60,6 +60,7 @@ InsertDestination::InsertDestination(const CatalogRelationSchema &relation,
                                      const StorageBlockLayout *layout,
                                      StorageManager *storage_manager,
                                      const std::size_t relational_op_index,
+                                     const std::size_t query_id,
                                      const tmb::client_id scheduler_client_id,
                                      tmb::MessageBus *bus)
     : thread_id_map_(*ClientIDMap::Instance()),
@@ -67,6 +68,7 @@ InsertDestination::InsertDestination(const CatalogRelationSchema &relation,
       relation_(relation),
       layout_(layout),
       relational_op_index_(relational_op_index),
+      query_id_(query_id),
       scheduler_client_id_(scheduler_client_id),
       bus_(DCHECK_NOTNULL(bus)) {
   if (layout_ == nullptr) {
@@ -74,11 +76,13 @@ InsertDestination::InsertDestination(const CatalogRelationSchema &relation,
   }
 }
 
-InsertDestination* InsertDestination::ReconstructFromProto(const serialization::InsertDestination &proto,
-                                                           const CatalogRelationSchema &relation,
-                                                           StorageManager *storage_manager,
-                                                           const tmb::client_id scheduler_client_id,
-                                                           tmb::MessageBus *bus) {
+InsertDestination* InsertDestination::ReconstructFromProto(
+    const std::size_t query_id,
+    const serialization::InsertDestination &proto,
+    const CatalogRelationSchema &relation,
+    StorageManager *storage_manager,
+    const tmb::client_id scheduler_client_id,
+    tmb::MessageBus *bus) {
   DCHECK(ProtoIsValid(proto, relation));
 
   StorageBlockLayout *layout = nullptr;
@@ -93,6 +97,7 @@ InsertDestination* InsertDestination::ReconstructFromProto(const serialization::
                                                     layout,
                                                     storage_manager,
                                                     proto.relational_op_index(),
+                                                    query_id,
                                                     scheduler_client_id,
                                                     bus);
     }
@@ -107,6 +112,7 @@ InsertDestination* InsertDestination::ReconstructFromProto(const serialization::
                                             storage_manager,
                                             move(blocks),
                                             proto.relational_op_index(),
+                                            query_id,
                                             scheduler_client_id,
                                             bus);
     }
@@ -134,6 +140,7 @@ InsertDestination* InsertDestination::ReconstructFromProto(const serialization::
           storage_manager,
           move(partitions),
           proto.relational_op_index(),
+          query_id,
           scheduler_client_id,
           bus);
     }
@@ -262,6 +269,7 @@ MutableBlockReference AlwaysCreateBlockInsertDestination::createNewBlock() {
   serialization::CatalogRelationNewBlockMessage proto;
   proto.set_relation_id(relation_.getID());
   proto.set_block_id(new_id);
+  proto.set_query_id(getQueryID());
 
   const size_t proto_length = proto.ByteSize();
   char *proto_bytes = static_cast<char*>(malloc(proto_length));
@@ -309,6 +317,7 @@ MutableBlockReference BlockPoolInsertDestination::createNewBlock() {
   serialization::CatalogRelationNewBlockMessage proto;
   proto.set_relation_id(relation_.getID());
   proto.set_block_id(new_id);
+  proto.set_query_id(getQueryID());
 
   const size_t proto_length = proto.ByteSize();
   char *proto_bytes = static_cast<char*>(malloc(proto_length));
@@ -385,21 +394,29 @@ const std::vector<block_id>& BlockPoolInsertDestination::getTouchedBlocksInterna
   return done_block_ids_;
 }
 
-PartitionAwareInsertDestination::PartitionAwareInsertDestination(PartitionSchemeHeader *partition_scheme_header,
-                                                                 const CatalogRelationSchema &relation,
-                                                                 const StorageBlockLayout *layout,
-                                                                 StorageManager *storage_manager,
-                                                                 vector<vector<block_id>> &&partitions,
-                                                                 const std::size_t relational_op_index,
-                                                                 const tmb::client_id scheduler_client_id,
-                                                                 tmb::MessageBus *bus)
-    : InsertDestination(relation, layout, storage_manager, relational_op_index, scheduler_client_id, bus),
+PartitionAwareInsertDestination::PartitionAwareInsertDestination(
+    PartitionSchemeHeader *partition_scheme_header,
+    const CatalogRelationSchema &relation,
+    const StorageBlockLayout *layout,
+    StorageManager *storage_manager,
+    vector<vector<block_id>> &&partitions,
+    const std::size_t relational_op_index,
+    const std::size_t query_id,
+    const tmb::client_id scheduler_client_id,
+    tmb::MessageBus *bus)
+    : InsertDestination(relation,
+                        layout,
+                        storage_manager,
+                        relational_op_index,
+                        query_id,
+                        scheduler_client_id,
+                        bus),
       partition_scheme_header_(DCHECK_NOTNULL(partition_scheme_header)),
       available_block_refs_(partition_scheme_header_->getNumPartitions()),
       available_block_ids_(move(partitions)),
       done_block_ids_(partition_scheme_header_->getNumPartitions()),
-      mutexes_for_partition_(new SpinMutex[partition_scheme_header_->getNumPartitions()]) {
-}
+      mutexes_for_partition_(
+          new SpinMutex[partition_scheme_header_->getNumPartitions()]) {}
 
 MutableBlockReference PartitionAwareInsertDestination::createNewBlock() {
   FATAL_ERROR("PartitionAwareInsertDestination::createNewBlock needs a partition id as an argument.");
@@ -415,6 +432,7 @@ MutableBlockReference PartitionAwareInsertDestination::createNewBlockInPartition
   proto.set_relation_id(relation_.getID());
   proto.set_block_id(new_id);
   proto.set_partition_id(part_id);
+  proto.set_query_id(getQueryID());
 
   const size_t proto_length = proto.ByteSize();
   char *proto_bytes = static_cast<char*>(malloc(proto_length));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index 670cd6c..5ff33f5 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -78,6 +78,7 @@ class InsertDestination : public InsertDestinationInterface {
    * @param storage_manager The StorageManager to use.
    * @param relational_op_index The index of the relational operator in the
    *        QueryPlan DAG that has outputs.
+   * @param query_id The ID of this query.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
    * @param bus A pointer to the TMB.
    **/
@@ -85,6 +86,7 @@ class InsertDestination : public InsertDestinationInterface {
                     const StorageBlockLayout *layout,
                     StorageManager *storage_manager,
                     const std::size_t relational_op_index,
+                    const std::size_t query_id,
                     const tmb::client_id scheduler_client_id,
                     tmb::MessageBus *bus);
 
@@ -98,6 +100,7 @@ class InsertDestination : public InsertDestinationInterface {
    * @brief A factory method to generate the InsertDestination from the
    *        serialized Protocol Buffer representation.
    *
+   * @param query_id The ID of this query.
    * @param proto A serialized Protocol Buffer representation of an
    *        InsertDestination, originally generated by the optimizer.
    * @param relation The relation to insert tuples into.
@@ -107,11 +110,13 @@ class InsertDestination : public InsertDestinationInterface {
    *
    * @return The constructed InsertDestination.
    */
-  static InsertDestination* ReconstructFromProto(const serialization::InsertDestination &proto,
-                                                 const CatalogRelationSchema &relation,
-                                                 StorageManager *storage_manager,
-                                                 const tmb::client_id scheduler_client_id,
-                                                 tmb::MessageBus *bus);
+  static InsertDestination* ReconstructFromProto(
+      const std::size_t query_id,
+      const serialization::InsertDestination &proto,
+      const CatalogRelationSchema &relation,
+      StorageManager *storage_manager,
+      const tmb::client_id scheduler_client_id,
+      tmb::MessageBus *bus);
 
   /**
    * @brief Check whether a serialized InsertDestination is fully-formed and
@@ -211,6 +216,7 @@ class InsertDestination : public InsertDestinationInterface {
     proto.set_operator_index(relational_op_index_);
     proto.set_block_id(id);
     proto.set_relation_id(relation_.getID());
+    proto.set_query_id(query_id_);
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const std::size_t proto_length = proto.ByteSize();
@@ -253,6 +259,10 @@ class InsertDestination : public InsertDestinationInterface {
         " ID " << scheduler_client_id_;
   }
 
+  inline const std::size_t getQueryID() const {
+    return query_id_;
+  }
+
   const ClientIDMap &thread_id_map_;
 
   StorageManager *storage_manager_;
@@ -260,6 +270,7 @@ class InsertDestination : public InsertDestinationInterface {
 
   std::unique_ptr<const StorageBlockLayout> layout_;
   const std::size_t relational_op_index_;
+  const std::size_t query_id_;
 
   tmb::client_id scheduler_client_id_;
   tmb::MessageBus *bus_;
@@ -288,10 +299,16 @@ class AlwaysCreateBlockInsertDestination : public InsertDestination {
                                      const StorageBlockLayout *layout,
                                      StorageManager *storage_manager,
                                      const std::size_t relational_op_index,
+                                     const std::size_t query_id,
                                      const tmb::client_id scheduler_client_id,
                                      tmb::MessageBus *bus)
-      : InsertDestination(relation, layout, storage_manager, relational_op_index, scheduler_client_id, bus) {
-  }
+      : InsertDestination(relation,
+                          layout,
+                          storage_manager,
+                          relational_op_index,
+                          query_id,
+                          scheduler_client_id,
+                          bus) {}
 
   ~AlwaysCreateBlockInsertDestination() override {
   }
@@ -334,16 +351,23 @@ class BlockPoolInsertDestination : public InsertDestination {
    * @param relational_op_index The index of the relational operator in the
    *        QueryPlan DAG that has outputs.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
+   * @param query_id The ID of the query.
    * @param bus A pointer to the TMB.
    **/
   BlockPoolInsertDestination(const CatalogRelationSchema &relation,
                              const StorageBlockLayout *layout,
                              StorageManager *storage_manager,
                              const std::size_t relational_op_index,
+                             const std::size_t query_id,
                              const tmb::client_id scheduler_client_id,
                              tmb::MessageBus *bus)
-      : InsertDestination(relation, layout, storage_manager, relational_op_index, scheduler_client_id, bus) {
-  }
+      : InsertDestination(relation,
+                          layout,
+                          storage_manager,
+                          relational_op_index,
+                          query_id,
+                          scheduler_client_id,
+                          bus) {}
 
   /**
    * @brief Constructor.
@@ -363,9 +387,16 @@ class BlockPoolInsertDestination : public InsertDestination {
                              StorageManager *storage_manager,
                              std::vector<block_id> &&blocks,
                              const std::size_t relational_op_index,
+                             const std::size_t query_id,
                              const tmb::client_id scheduler_client_id,
                              tmb::MessageBus *bus)
-      : InsertDestination(relation, layout, storage_manager, relational_op_index, scheduler_client_id, bus),
+      : InsertDestination(relation,
+                          layout,
+                          storage_manager,
+                          relational_op_index,
+                          query_id,
+                          scheduler_client_id,
+                          bus),
         available_block_ids_(std::move(blocks)) {
     // TODO(chasseur): Once block fill statistics are available, replace this
     // with something smarter.
@@ -386,7 +417,6 @@ class BlockPoolInsertDestination : public InsertDestination {
   MutableBlockReference createNewBlock() override;
 
  private:
-  FRIEND_TEST(ForemanTest, TwoNodesDAGPartiallyFilledBlocksTest);
   FRIEND_TEST(QueryManagerTest, TwoNodesDAGPartiallyFilledBlocksTest);
 
   // A vector of references to blocks which are loaded in memory.
@@ -416,17 +446,20 @@ class PartitionAwareInsertDestination : public InsertDestination {
    * @param partitions The blocks in partitions.
    * @param relational_op_index The index of the relational operator in the
    *        QueryPlan DAG that has outputs.
+   * @param query_id The ID of the query.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
    * @param bus A pointer to the TMB.
    **/
-  PartitionAwareInsertDestination(PartitionSchemeHeader *partition_scheme_header,
-                                  const CatalogRelationSchema &relation,
-                                  const StorageBlockLayout *layout,
-                                  StorageManager *storage_manager,
-                                  std::vector<std::vector<block_id>> &&partitions,
-                                  const std::size_t relational_op_index,
-                                  const tmb::client_id scheduler_client_id,
-                                  tmb::MessageBus *bus);
+  PartitionAwareInsertDestination(
+      PartitionSchemeHeader *partition_scheme_header,
+      const CatalogRelationSchema &relation,
+      const StorageBlockLayout *layout,
+      StorageManager *storage_manager,
+      std::vector<std::vector<block_id>> &&partitions,
+      const std::size_t relational_op_index,
+      const std::size_t query_id,
+      const tmb::client_id scheduler_client_id,
+      tmb::MessageBus *bus);
 
   ~PartitionAwareInsertDestination() override {
     delete[] mutexes_for_partition_;


[02/13] incubator-quickstep git commit: QUICKSTEP-10: Serialized WorkOrders as proto.

Posted by zu...@apache.org.
QUICKSTEP-10: Serialized WorkOrders as proto.


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

Branch: refs/heads/travis-grpc
Commit: c9214ecb1d481b3d1b02db0ffdf53852b11b540f
Parents: ccd11c0
Author: Zuyu Zhang <zz...@pivotal.io>
Authored: Tue Apr 12 16:55:48 2016 -0700
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Jun 9 17:06:02 2016 -0700

----------------------------------------------------------------------
 query_execution/CMakeLists.txt                  |   6 +
 query_execution/WorkOrderProtosContainer.hpp    | 146 +++++++++++++++++++
 query_execution/tests/Foreman_unittest.cpp      |   6 +
 query_execution/tests/QueryManager_unittest.cpp |   6 +
 relational_operators/AggregationOperator.cpp    |  34 +++++
 relational_operators/AggregationOperator.hpp    |  12 ++
 relational_operators/BuildHashOperator.cpp      |  40 +++++
 relational_operators/BuildHashOperator.hpp      |  12 ++
 relational_operators/CMakeLists.txt             |  36 ++++-
 relational_operators/CreateIndexOperator.hpp    |   9 ++
 relational_operators/CreateTableOperator.hpp    |   8 +
 relational_operators/DeleteOperator.cpp         |  38 ++++-
 relational_operators/DeleteOperator.hpp         |  14 +-
 relational_operators/DestroyHashOperator.cpp    |  17 +++
 relational_operators/DestroyHashOperator.hpp    |   3 +
 relational_operators/DropTableOperator.cpp      |  23 +++
 relational_operators/DropTableOperator.hpp      |   3 +
 .../FinalizeAggregationOperator.cpp             |  20 +++
 .../FinalizeAggregationOperator.hpp             |   3 +
 relational_operators/HashJoinOperator.cpp       | 124 ++++++++++++++++
 relational_operators/HashJoinOperator.hpp       |  21 +++
 relational_operators/InsertOperator.cpp         |  19 +++
 relational_operators/InsertOperator.hpp         |   3 +
 .../NestedLoopsJoinOperator.cpp                 | 142 ++++++++++++++++++
 .../NestedLoopsJoinOperator.hpp                 |  51 +++++++
 relational_operators/RelationalOperator.hpp     |  22 +++
 relational_operators/SampleOperator.cpp         | 101 ++++++++++---
 relational_operators/SampleOperator.hpp         |  12 ++
 relational_operators/SaveBlocksOperator.cpp     |  18 +++
 relational_operators/SaveBlocksOperator.hpp     |   3 +
 relational_operators/SelectOperator.cpp         |  43 ++++++
 relational_operators/SelectOperator.hpp         |  12 ++
 relational_operators/SortMergeRunOperator.cpp   |  68 +++++++++
 relational_operators/SortMergeRunOperator.hpp   |  12 ++
 .../SortRunGenerationOperator.cpp               |  39 +++++
 .../SortRunGenerationOperator.hpp               |  12 ++
 relational_operators/TableGeneratorOperator.cpp |  20 ++-
 relational_operators/TableGeneratorOperator.hpp |   5 +-
 relational_operators/TextScanOperator.cpp       | 125 ++++++++++++----
 relational_operators/TextScanOperator.hpp       |  24 ++-
 relational_operators/TextScanOperator.proto     |  22 ---
 relational_operators/UpdateOperator.cpp         |  23 +++
 relational_operators/UpdateOperator.hpp         |   3 +
 relational_operators/WorkOrder.proto            |  21 +--
 relational_operators/WorkOrderFactory.cpp       |   7 +-
 45 files changed, 1291 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/query_execution/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_execution/CMakeLists.txt b/query_execution/CMakeLists.txt
index 8306f78..95bc0d6 100644
--- a/query_execution/CMakeLists.txt
+++ b/query_execution/CMakeLists.txt
@@ -42,6 +42,7 @@ add_library(quickstep_queryexecution_QueryExecutionState ../empty_src.cpp QueryE
 add_library(quickstep_queryexecution_QueryExecutionTypedefs ../empty_src.cpp QueryExecutionTypedefs.hpp)
 add_library(quickstep_queryexecution_QueryExecutionUtil ../empty_src.cpp QueryExecutionUtil.hpp)
 add_library(quickstep_queryexecution_QueryManager QueryManager.cpp QueryManager.hpp)
+add_library(quickstep_queryexecution_WorkOrderProtosContainer ../empty_src.cpp WorkOrderProtosContainer.hpp)
 add_library(quickstep_queryexecution_WorkOrdersContainer WorkOrdersContainer.cpp WorkOrdersContainer.hpp)
 add_library(quickstep_queryexecution_Worker Worker.cpp Worker.hpp)
 add_library(quickstep_queryexecution_WorkerDirectory ../empty_src.cpp WorkerDirectory.hpp)
@@ -157,6 +158,10 @@ target_link_libraries(quickstep_queryexecution_QueryManager
                       quickstep_utility_DAG
                       quickstep_utility_Macros
                       tmb)
+target_link_libraries(quickstep_queryexecution_WorkOrderProtosContainer
+                      glog
+                      quickstep_relationaloperators_WorkOrder_proto
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryexecution_WorkOrdersContainer
                       glog
                       quickstep_relationaloperators_WorkOrder
@@ -193,6 +198,7 @@ target_link_libraries(quickstep_queryexecution
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_QueryExecutionUtil
                       quickstep_queryexecution_QueryManager
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_queryexecution_Worker
                       quickstep_queryexecution_WorkerDirectory

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/query_execution/WorkOrderProtosContainer.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkOrderProtosContainer.hpp b/query_execution/WorkOrderProtosContainer.hpp
new file mode 100644
index 0000000..5043755
--- /dev/null
+++ b/query_execution/WorkOrderProtosContainer.hpp
@@ -0,0 +1,146 @@
+/**
+ *   Copyright 2015-2016 Pivotal Software, Inc.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_EXECUTION_WORKORDER_PROTOS_CONTAINER_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_WORKORDER_PROTOS_CONTAINER_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <queue>
+#include <vector>
+
+#include "relational_operators/WorkOrder.pb.h"  // IWYU pragma: keep
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup QueryExecution
+ *  @{
+ */
+
+ /**
+  * @brief A container used in the distributed version to hold the normal
+  *        (non-rebuild) WorkOrder protos for a given query.
+  *
+  * @note This container stays alive during the lifetime of the query.
+  **/
+class WorkOrderProtosContainer {
+ public:
+  /**
+   * @brief Constructor
+   *
+   * @param num_operators Number of operators in the query DAG.
+   **/
+  explicit WorkOrderProtosContainer(const std::size_t num_operators)
+      : num_operators_(num_operators),
+        operator_containers_(num_operators_) {
+    DCHECK_NE(num_operators_, 0u);
+  }
+
+  /**
+   * @brief Destructor.
+   *
+   * @note If the query is executed normally, we should never encounter a
+   *       situation where at the time of deletion the WorkOrderProtosContainer has
+   *       pending WorkOrders.
+   **/
+  ~WorkOrderProtosContainer() {
+    for (std::size_t op = 0; op < num_operators_; ++op) {
+      if (hasWorkOrderProto(op)) {
+        LOG(WARNING) << "Destroying a WorkOrderProtosContainer that still has pending WorkOrder protos.";
+        break;
+      }
+    }
+  }
+
+  /**
+   * @brief Check if there are some pending WorkOrders for the given operator.
+   *
+   * @param operator_index Index of the operator.
+   *
+   * @return If there are pending WorkOrders.
+   **/
+  bool hasWorkOrderProto(const std::size_t operator_index) const {
+    DCHECK_LT(operator_index, num_operators_);
+    return !operator_containers_[operator_index].empty();
+  }
+
+  /**
+   * @brief Get a WorkOrder for a given operator.
+   *
+   * @param operator_index The index of the operator.
+   *
+   * @return Release a WorkOrder proto. If no WorkOrder proto is available,
+   *         return nullptr.
+   **/
+  serialization::WorkOrder* getWorkOrderProto(const std::size_t operator_index) {
+    DCHECK_LT(operator_index, num_operators_);
+
+    if (operator_containers_[operator_index].empty()) {
+      return nullptr;
+    }
+
+    serialization::WorkOrder *proto =
+        operator_containers_[operator_index].front().release();
+    operator_containers_[operator_index].pop();
+
+    return proto;
+  }
+
+  /**
+   * @brief Add a WorkOrder generated from a given
+   *        operator.
+   *
+   * @param workorder A pointer to the WorkOrder to be added.
+   * @param operator_index The index of the operator in the query DAG.
+   **/
+  void addWorkOrderProto(serialization::WorkOrder *proto,
+                         const std::size_t operator_index) {
+    DCHECK(proto != nullptr);
+    DCHECK_LT(operator_index, num_operators_);
+
+    operator_containers_[operator_index].emplace(
+        std::unique_ptr<serialization::WorkOrder>(proto));
+  }
+
+  /**
+   * @brief Get the number of all pending WorkOrders
+   *        for a given operator.
+   *
+   * @param operator_index The index of the operator.
+   *
+   * @return The number of pending WorkOrders.
+   **/
+  std::size_t getNumWorkOrderProtos(const std::size_t operator_index) const {
+    DCHECK_LT(operator_index, num_operators_);
+    return operator_containers_[operator_index].size();
+  }
+
+ private:
+  const std::size_t num_operators_;
+
+  std::vector<std::queue<std::unique_ptr<serialization::WorkOrder>>> operator_containers_;
+
+  DISALLOW_COPY_AND_ASSIGN(WorkOrderProtosContainer);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_EXECUTION_WORKORDER_PROTOS_CONTAINER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/query_execution/tests/Foreman_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/Foreman_unittest.cpp b/query_execution/tests/Foreman_unittest.cpp
index 79f8f4a..cbe5088 100644
--- a/query_execution/tests/Foreman_unittest.cpp
+++ b/query_execution/tests/Foreman_unittest.cpp
@@ -58,6 +58,8 @@ using tmb::client_id;
 
 namespace quickstep {
 
+class WorkOrderProtosContainer;
+
 class MockWorkOrder : public WorkOrder {
  public:
   explicit MockWorkOrder(const int op_index)
@@ -168,6 +170,10 @@ class MockOperator: public RelationalOperator {
     return num_calls_get_workorders_ == max_getworkorder_iters_;
   }
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override {
+    return true;
+  }
+
   void feedInputBlock(const block_id input_block_id,
                       const relation_id input_relation_id) override {
     ++num_calls_feedblock_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/query_execution/tests/QueryManager_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManager_unittest.cpp b/query_execution/tests/QueryManager_unittest.cpp
index 308d5ca..9ba5978 100644
--- a/query_execution/tests/QueryManager_unittest.cpp
+++ b/query_execution/tests/QueryManager_unittest.cpp
@@ -59,6 +59,8 @@ using tmb::client_id;
 
 namespace quickstep {
 
+class WorkOrderProtosContainer;
+
 class MockWorkOrder : public WorkOrder {
  public:
   explicit MockWorkOrder(const int op_index)
@@ -169,6 +171,10 @@ class MockOperator: public RelationalOperator {
     return num_calls_get_workorders_ == max_getworkorder_iters_;
   }
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override {
+    return true;
+  }
+
   void feedInputBlock(const block_id input_block_id,
                       const relation_id input_relation_id) override {
     ++num_calls_feedblock_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/AggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.cpp b/relational_operators/AggregationOperator.cpp
index 7252541..6d16930 100644
--- a/relational_operators/AggregationOperator.cpp
+++ b/relational_operators/AggregationOperator.cpp
@@ -20,7 +20,9 @@
 #include <vector>
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/AggregationOperationState.hpp"
 #include "storage/StorageBlockInfo.hpp"
 
@@ -61,6 +63,38 @@ bool AggregationOperator::getAllWorkOrders(
   }
 }
 
+bool AggregationOperator::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_);
+      }
+      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_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* AggregationOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::AGGREGATION);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::AggregationWorkOrder::block_id, block);
+  proto->SetExtension(serialization::AggregationWorkOrder::aggr_state_index, aggr_state_index_);
+
+  return proto;
+}
+
+
 void AggregationWorkOrder::execute() {
   state_->aggregateBlock(input_block_id_);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index f340d4e..4bcbcf6 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -38,8 +38,11 @@ namespace quickstep {
 
 class AggregationOperationState;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -80,6 +83,8 @@ class AggregationOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
     input_relation_block_ids_.push_back(input_block_id);
   }
@@ -91,6 +96,13 @@ class AggregationOperator : public RelationalOperator {
   }
 
  private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
   const bool input_relation_is_stored_;
   std::vector<block_id> input_relation_block_ids_;
   const QueryContext::aggregation_state_id aggr_state_index_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/BuildHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.cpp b/relational_operators/BuildHashOperator.cpp
index 9dc4afe..1c2ff05 100644
--- a/relational_operators/BuildHashOperator.cpp
+++ b/relational_operators/BuildHashOperator.cpp
@@ -22,7 +22,9 @@
 
 #include "catalog/CatalogRelation.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/HashTable.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -99,6 +101,44 @@ 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_);
+      }
+      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_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* BuildHashOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::BUILD_HASH);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::BuildHashWorkOrder::relation_id, input_relation_.getID());
+  for (const attribute_id attr_id : join_key_attributes_) {
+    proto->AddExtension(serialization::BuildHashWorkOrder::join_key_attributes, attr_id);
+  }
+  proto->SetExtension(serialization::BuildHashWorkOrder::any_join_key_attributes_nullable,
+                      any_join_key_attributes_nullable_);
+  proto->SetExtension(serialization::BuildHashWorkOrder::join_hash_table_index, hash_table_index_);
+  proto->SetExtension(serialization::BuildHashWorkOrder::block_id, block);
+
+  return proto;
+}
+
+
 void BuildHashWorkOrder::execute() {
   BlockReference block(
       storage_manager_->getBlock(build_block_id_, input_relation_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 50dd7d6..464bbf8 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -39,6 +39,7 @@ namespace quickstep {
 
 class CatalogRelationSchema;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 struct TupleReference;
@@ -46,6 +47,8 @@ struct TupleReference;
 template <typename, bool, bool, bool, bool> class HashTable;
 typedef HashTable<TupleReference, true, false, false, true> JoinHashTable;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -96,6 +99,8 @@ class BuildHashOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   void feedInputBlock(const block_id input_block_id,
                       const relation_id input_relation_id) override {
     input_relation_block_ids_.push_back(input_block_id);
@@ -109,6 +114,13 @@ class BuildHashOperator : public RelationalOperator {
   }
 
  private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
   const CatalogRelation &input_relation_;
   const bool input_relation_is_stored_;
   const std::vector<attribute_id> join_key_attributes_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index eb73c07..91d1097 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -73,9 +73,11 @@ target_link_libraries(quickstep_relationaloperators_AggregationOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_AggregationOperationState
                       quickstep_storage_StorageBlockInfo
                       quickstep_utility_Macros
@@ -85,9 +87,11 @@ target_link_libraries(quickstep_relationaloperators_BuildHashOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_HashTable
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
@@ -120,9 +124,11 @@ target_link_libraries(quickstep_relationaloperators_DeleteOperator
                       quickstep_queryexecution_QueryExecutionMessages_proto
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_QueryExecutionUtil
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_StorageManager
@@ -132,9 +138,11 @@ target_link_libraries(quickstep_relationaloperators_DeleteOperator
 target_link_libraries(quickstep_relationaloperators_DestroyHashOperator
                       glog
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_utility_Macros
                       tmb)
 target_link_libraries(quickstep_relationaloperators_DropTableOperator
@@ -143,9 +151,11 @@ target_link_libraries(quickstep_relationaloperators_DropTableOperator
                       quickstep_catalog_CatalogDatabaseLite
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_StorageManager
                       quickstep_utility_Macros
@@ -155,9 +165,11 @@ target_link_libraries(quickstep_relationaloperators_FinalizeAggregationOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_AggregationOperationState
                       quickstep_utility_Macros
                       tmb)
@@ -170,9 +182,11 @@ target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_HashTable
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlock
@@ -194,9 +208,11 @@ target_link_libraries(quickstep_relationaloperators_InsertOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_types_containers_Tuple
                       quickstep_utility_Macros
@@ -209,9 +225,11 @@ target_link_libraries(quickstep_relationaloperators_NestedLoopsJoinOperator
                       quickstep_expressions_predicate_Predicate
                       quickstep_expressions_scalar_Scalar
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
@@ -245,9 +263,11 @@ target_link_libraries(quickstep_relationaloperators_SampleOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
-                      quickstep_relationaloperators_RelationalOperator
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
+                      quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
@@ -257,9 +277,11 @@ target_link_libraries(quickstep_relationaloperators_SampleOperator
 target_link_libraries(quickstep_relationaloperators_SaveBlocksOperator
                       glog
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_StorageManager
                       quickstep_utility_Macros
@@ -270,9 +292,11 @@ target_link_libraries(quickstep_relationaloperators_SelectOperator
                       quickstep_catalog_CatalogTypedefs
                       quickstep_catalog_PartitionSchemeHeader
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
@@ -289,11 +313,13 @@ target_link_libraries(quickstep_relationaloperators_SortMergeRunOperator
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_SortMergeRunOperatorHelpers
                       quickstep_relationaloperators_SortMergeRunOperator_proto
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_StorageBlockInfo
                       quickstep_threading_ThreadIDBasedMap
                       quickstep_utility_Macros
@@ -326,9 +352,11 @@ target_link_libraries(quickstep_relationaloperators_SortRunGenerationOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
@@ -343,9 +371,11 @@ target_link_libraries(quickstep_relationaloperators_TableGeneratorOperator
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_tablegenerator_GeneratorFunctionHandle
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlockInfo
                       quickstep_types_containers_ColumnVectorsValueAccessor
@@ -358,9 +388,11 @@ target_link_libraries(quickstep_relationaloperators_TextScanOperator
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_QueryContext
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_types_Type
                       quickstep_types_TypedValue
@@ -379,9 +411,11 @@ target_link_libraries(quickstep_relationaloperators_UpdateOperator
                       quickstep_queryexecution_QueryExecutionMessages_proto
                       quickstep_queryexecution_QueryExecutionTypedefs
                       quickstep_queryexecution_QueryExecutionUtil
+                      quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_relationaloperators_WorkOrder
+                      quickstep_relationaloperators_WorkOrder_proto
                       quickstep_storage_InsertDestination
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index 11a01ae..18ca656 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -1,6 +1,7 @@
 /**
  *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
  *     University of Wisconsin\u2014Madison.
+ *   Copyright 2016 Pivotal Software, Inc.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -37,6 +38,7 @@ namespace quickstep {
 class CatalogRelation;
 class QueryContext;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -76,6 +78,13 @@ class CreateIndexOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  /**
+   * @note no WorkOrder proto generated for this operator.
+   **/
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override {
+    return true;
+  }
+
   void updateCatalogOnCompletion() override;
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/CreateTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.hpp b/relational_operators/CreateTableOperator.hpp
index 60bcef4..6d91142 100644
--- a/relational_operators/CreateTableOperator.hpp
+++ b/relational_operators/CreateTableOperator.hpp
@@ -36,6 +36,7 @@ namespace quickstep {
 class CatalogDatabase;
 class QueryContext;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -74,6 +75,13 @@ class CreateTableOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  /**
+   * @note no WorkOrder proto generated for this operator.
+   **/
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override {
+    return true;
+  }
+
   void updateCatalogOnCompletion() override;
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.cpp b/relational_operators/DeleteOperator.cpp
index 15dc9e3..47e36e9 100644
--- a/relational_operators/DeleteOperator.cpp
+++ b/relational_operators/DeleteOperator.cpp
@@ -26,13 +26,14 @@
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/QueryExecutionMessages.pb.h"
 #include "query_execution/QueryExecutionUtil.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
 #include "threading/ThreadIDBasedMap.hpp"
 
-
 #include "glog/logging.h"
 
 #include "tmb/id_typedefs.h"
@@ -85,6 +86,41 @@ bool DeleteOperator::getAllWorkOrders(
   }
 }
 
+bool DeleteOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (relation_is_stored_) {
+    // If relation_ is stored, iterate over the list of blocks in relation_.
+    if (!started_) {
+      for (const block_id input_block_id : relation_block_ids_) {
+        container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    while (num_workorders_generated_ < relation_block_ids_.size()) {
+      container->addWorkOrderProto(
+          createWorkOrderProto(relation_block_ids_[num_workorders_generated_]),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* DeleteOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::DELETE);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::DeleteWorkOrder::operator_index, op_index_);
+  proto->SetExtension(serialization::DeleteWorkOrder::relation_id, relation_.getID());
+  proto->SetExtension(serialization::DeleteWorkOrder::predicate_index, predicate_index_);
+  proto->SetExtension(serialization::DeleteWorkOrder::block_id, block);
+
+  return proto;
+}
+
+
 void DeleteWorkOrder::execute() {
   MutableBlockReference block(
       storage_manager_->getBlockMutable(input_block_id_, input_relation_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index c55f585..aa8a688 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -41,8 +41,11 @@ namespace quickstep {
 class CatalogRelationSchema;
 class Predicate;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -67,7 +70,7 @@ class DeleteOperator : public RelationalOperator {
                  const CatalogRelation &relation,
                  const QueryContext::predicate_id predicate_index,
                  const bool relation_is_stored)
-     :  RelationalOperator(query_id),
+      : RelationalOperator(query_id),
         relation_(relation),
         predicate_index_(predicate_index),
         relation_is_stored_(relation_is_stored),
@@ -84,6 +87,8 @@ class DeleteOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   const relation_id getOutputRelationID() const override {
     return relation_.getID();
   }
@@ -101,6 +106,13 @@ class DeleteOperator : public RelationalOperator {
   }
 
  private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
   const CatalogRelation &relation_;
   const QueryContext::predicate_id predicate_index_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/DestroyHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.cpp b/relational_operators/DestroyHashOperator.cpp
index 8aa40b4..e748470 100644
--- a/relational_operators/DestroyHashOperator.cpp
+++ b/relational_operators/DestroyHashOperator.cpp
@@ -18,7 +18,9 @@
 #include "relational_operators/DestroyHashOperator.hpp"
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 
 #include "tmb/id_typedefs.h"
 
@@ -39,6 +41,21 @@ bool DestroyHashOperator::getAllWorkOrders(
   return work_generated_;
 }
 
+bool DestroyHashOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (blocking_dependencies_met_ && !work_generated_) {
+    work_generated_ = true;
+
+    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_);
+  }
+  return work_generated_;
+}
+
+
 void DestroyHashWorkOrder::execute() {
   query_context_->destroyJoinHashTable(hash_table_index_);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index 7d8acb7..181386f 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -32,6 +32,7 @@ namespace tmb { class MessageBus; }
 namespace quickstep {
 
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -63,6 +64,8 @@ class DestroyHashOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
  private:
   const QueryContext::join_hash_table_id hash_table_index_;
   bool work_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/DropTableOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.cpp b/relational_operators/DropTableOperator.cpp
index 256f6a1..727aa46 100644
--- a/relational_operators/DropTableOperator.cpp
+++ b/relational_operators/DropTableOperator.cpp
@@ -24,7 +24,9 @@
 #include "catalog/CatalogDatabaseLite.hpp"
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogTypedefs.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
 
@@ -55,6 +57,27 @@ bool DropTableOperator::getAllWorkOrders(
   return work_generated_;
 }
 
+bool DropTableOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (blocking_dependencies_met_ && !work_generated_) {
+    work_generated_ = true;
+
+    serialization::WorkOrder *proto = new serialization::WorkOrder;
+    proto->set_work_order_type(serialization::DROP_TABLE);
+    proto->set_query_id(query_id_);
+
+    std::vector<block_id> relation_blocks(relation_.getBlocksSnapshot());
+    for (const block_id relation_block : relation_blocks) {
+      proto->AddExtension(serialization::DropTableWorkOrder::block_ids, relation_block);
+    }
+
+    container->addWorkOrderProto(proto, op_index_);
+
+    database_->setStatus(CatalogDatabase::Status::kPendingBlockDeletions);
+  }
+
+  return work_generated_;
+}
+
 void DropTableOperator::updateCatalogOnCompletion() {
   const relation_id rel_id = relation_.getID();
   if (only_drop_blocks_) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index a0a8d6e..6c7fca3 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -41,6 +41,7 @@ class CatalogDatabaseLite;
 class CatalogRelation;
 class QueryContext;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -79,6 +80,8 @@ class DropTableOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   void updateCatalogOnCompletion() override;
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/FinalizeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.cpp b/relational_operators/FinalizeAggregationOperator.cpp
index 1dc4188..20d0ee5 100644
--- a/relational_operators/FinalizeAggregationOperator.cpp
+++ b/relational_operators/FinalizeAggregationOperator.cpp
@@ -18,7 +18,9 @@
 #include "relational_operators/FinalizeAggregationOperator.hpp"
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/AggregationOperationState.hpp"
 
 #include "glog/logging.h"
@@ -47,6 +49,24 @@ bool FinalizeAggregationOperator::getAllWorkOrders(
   return started_;
 }
 
+bool FinalizeAggregationOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (blocking_dependencies_met_ && !started_) {
+    started_ = true;
+
+    serialization::WorkOrder *proto = new serialization::WorkOrder;
+    proto->set_work_order_type(serialization::FINALIZE_AGGREGATION);
+    proto->set_query_id(query_id_);
+    proto->SetExtension(serialization::FinalizeAggregationWorkOrder::aggr_state_index,
+                        aggr_state_index_);
+    proto->SetExtension(serialization::FinalizeAggregationWorkOrder::insert_destination_index,
+                        output_destination_index_);
+
+    container->addWorkOrderProto(proto, op_index_);
+  }
+  return started_;
+}
+
+
 void FinalizeAggregationWorkOrder::execute() {
   state_->finalizeAggregate(output_destination_);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index e8a403f..158a637 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -39,6 +39,7 @@ namespace quickstep {
 
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -79,6 +80,8 @@ class FinalizeAggregationOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   QueryContext::insert_destination_id getInsertDestinationID() const override {
     return output_destination_index_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index b89cfb3..5a47b50 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -31,7 +31,9 @@
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/HashTable.hpp"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
@@ -391,6 +393,128 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
   return false;
 }
 
+bool HashJoinOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  switch (join_type_) {
+    case JoinType::kInnerJoin:
+      return getAllNonOuterJoinWorkOrderProtos(container, serialization::HashJoinWorkOrder::HASH_INNER_JOIN);
+    case JoinType::kLeftSemiJoin:
+      return getAllNonOuterJoinWorkOrderProtos(container, serialization::HashJoinWorkOrder::HASH_SEMI_JOIN);
+    case JoinType::kLeftAntiJoin:
+      return getAllNonOuterJoinWorkOrderProtos(container, serialization::HashJoinWorkOrder::HASH_ANTI_JOIN);
+    case JoinType::kLeftOuterJoin:
+      return getAllOuterJoinWorkOrderProtos(container);
+    default:
+      LOG(FATAL) << "Unknown join type in HashJoinOperator::getAllWorkOrderProtos()";
+  }
+}
+
+bool HashJoinOperator::getAllNonOuterJoinWorkOrderProtos(
+    WorkOrderProtosContainer *container,
+    const serialization::HashJoinWorkOrder::HashJoinWorkOrderType hash_join_type) {
+  // We wait until the building of global hash table is complete.
+  if (!blocking_dependencies_met_) {
+    return false;
+  }
+
+  if (probe_relation_is_stored_) {
+    if (!started_) {
+      for (const block_id probe_block_id : probe_relation_block_ids_) {
+        container->addWorkOrderProto(
+            createNonOuterJoinWorkOrderProto(hash_join_type, probe_block_id),
+            op_index_);
+      }
+      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_;
+    }
+
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* HashJoinOperator::createNonOuterJoinWorkOrderProto(
+    const serialization::HashJoinWorkOrder::HashJoinWorkOrderType hash_join_type,
+    const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::HASH_JOIN);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::HashJoinWorkOrder::hash_join_work_order_type, hash_join_type);
+  proto->SetExtension(serialization::HashJoinWorkOrder::build_relation_id, build_relation_.getID());
+  proto->SetExtension(serialization::HashJoinWorkOrder::probe_relation_id, probe_relation_.getID());
+  for (const attribute_id attr_id : join_key_attributes_) {
+    proto->AddExtension(serialization::HashJoinWorkOrder::join_key_attributes, attr_id);
+  }
+  proto->SetExtension(serialization::HashJoinWorkOrder::any_join_key_attributes_nullable,
+                      any_join_key_attributes_nullable_);
+  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::residual_predicate_index, residual_predicate_index_);
+
+  return proto;
+}
+
+bool HashJoinOperator::getAllOuterJoinWorkOrderProtos(WorkOrderProtosContainer *container) {
+  // We wait until the building of global hash table is complete.
+  if (!blocking_dependencies_met_) {
+    return false;
+  }
+
+  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_);
+      }
+      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_;
+    }
+
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* HashJoinOperator::createOuterJoinWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::HASH_JOIN);
+
+  proto->SetExtension(serialization::HashJoinWorkOrder::hash_join_work_order_type,
+                      serialization::HashJoinWorkOrder::HASH_OUTER_JOIN);
+  proto->SetExtension(serialization::HashJoinWorkOrder::build_relation_id, build_relation_.getID());
+  proto->SetExtension(serialization::HashJoinWorkOrder::probe_relation_id, probe_relation_.getID());
+  for (const attribute_id attr_id : join_key_attributes_) {
+    proto->AddExtension(serialization::HashJoinWorkOrder::join_key_attributes, attr_id);
+  }
+  proto->SetExtension(serialization::HashJoinWorkOrder::any_join_key_attributes_nullable,
+                      any_join_key_attributes_nullable_);
+  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);
+
+  for (const bool is_attribute_on_build : is_selection_on_build_) {
+    proto->AddExtension(serialization::HashJoinWorkOrder::is_selection_on_build, is_attribute_on_build);
+  }
+
+  return proto;
+}
+
+
 void HashInnerJoinWorkOrder::execute() {
   if (FLAGS_vector_based_joined_tuple_collector) {
     executeWithCollectorType<VectorBasedJoinedTupleCollector>();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 1d5d4e3..6f4271d 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -30,6 +30,7 @@
 #include "query_execution/QueryContext.hpp"
 #include "relational_operators/RelationalOperator.hpp"
 #include "relational_operators/WorkOrder.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/HashTable.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "utility/Macros.hpp"
@@ -47,6 +48,7 @@ class InsertDestination;
 class Predicate;
 class Scalar;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -161,6 +163,8 @@ class HashJoinOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   void feedInputBlock(const block_id input_block_id,
                       const relation_id input_relation_id) override {
     DCHECK(input_relation_id == probe_relation_.getID());
@@ -202,6 +206,23 @@ class HashJoinOperator : public RelationalOperator {
                                  QueryContext *query_context,
                                  StorageManager *storage_manager);
 
+  bool getAllNonOuterJoinWorkOrderProtos(
+      WorkOrderProtosContainer *container,
+      const serialization::HashJoinWorkOrder::HashJoinWorkOrderType hash_join_type);
+
+  serialization::WorkOrder* createNonOuterJoinWorkOrderProto(
+      const serialization::HashJoinWorkOrder::HashJoinWorkOrderType hash_join_type,
+      const block_id block);
+
+  bool getAllOuterJoinWorkOrderProtos(WorkOrderProtosContainer *container);
+
+  /**
+   * @brief Create HashOuterJoinWorkOrder proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createOuterJoinWorkOrderProto(const block_id block);
+
   const CatalogRelation &build_relation_;
   const CatalogRelation &probe_relation_;
   const bool probe_relation_is_stored_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/InsertOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.cpp b/relational_operators/InsertOperator.cpp
index 3ec9933..963cdcd 100644
--- a/relational_operators/InsertOperator.cpp
+++ b/relational_operators/InsertOperator.cpp
@@ -20,7 +20,9 @@
 #include <memory>
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 
 #include "glog/logging.h"
@@ -49,6 +51,23 @@ bool InsertOperator::getAllWorkOrders(
   return work_generated_;
 }
 
+bool InsertOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (blocking_dependencies_met_ && !work_generated_) {
+    work_generated_ = true;
+
+    serialization::WorkOrder *proto = new serialization::WorkOrder;
+    proto->set_work_order_type(serialization::INSERT);
+    proto->set_query_id(query_id_);
+    proto->SetExtension(serialization::InsertWorkOrder::insert_destination_index, output_destination_index_);
+    proto->SetExtension(serialization::InsertWorkOrder::tuple_index, tuple_index_);
+
+    container->addWorkOrderProto(proto, op_index_);
+  }
+
+  return work_generated_;
+}
+
+
 void InsertWorkOrder::execute() {
   output_destination_->insertTuple(*tuple_);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/InsertOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.hpp b/relational_operators/InsertOperator.hpp
index 51c606d..78f5199 100644
--- a/relational_operators/InsertOperator.hpp
+++ b/relational_operators/InsertOperator.hpp
@@ -39,6 +39,7 @@ namespace quickstep {
 
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -78,6 +79,8 @@ class InsertOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   QueryContext::insert_destination_id getInsertDestinationID() const override {
     return output_destination_index_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/NestedLoopsJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.cpp b/relational_operators/NestedLoopsJoinOperator.cpp
index 5a47fca..43588ee 100644
--- a/relational_operators/NestedLoopsJoinOperator.cpp
+++ b/relational_operators/NestedLoopsJoinOperator.cpp
@@ -26,7 +26,9 @@
 #include "expressions/predicate/Predicate.hpp"
 #include "expressions/scalar/Scalar.hpp"
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -150,6 +152,72 @@ bool NestedLoopsJoinOperator::getAllWorkOrders(
   }
 }
 
+bool NestedLoopsJoinOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  if (left_relation_is_stored_ && right_relation_is_stored_) {
+    // Make sure we generate workorders only once.
+    if (!all_workorders_generated_) {
+      for (const block_id left_block_id : left_relation_block_ids_) {
+        for (const block_id right_block_id : right_relation_block_ids_) {
+          container->addWorkOrderProto(createWorkOrderProto(left_block_id, right_block_id),
+                                       op_index_);
+        }
+      }
+      all_workorders_generated_ = true;
+    }
+    return true;
+  } else if (!(left_relation_is_stored_ || right_relation_is_stored_)) {
+    // Both relations are not stored.
+    const std::vector<block_id>::size_type new_left_blocks
+        = left_relation_block_ids_.size() - num_left_workorders_generated_;
+    const std::vector<block_id>::size_type new_right_blocks
+        = right_relation_block_ids_.size() - num_right_workorders_generated_;
+
+    std::size_t new_workorders = 0;
+    if (new_left_blocks > 0 && new_right_blocks > 0) {
+      // Blocks added to both left and right relations.
+      // First generate (left + new_left_blocks) * (new_right_blocks).
+      new_workorders =
+          getAllWorkOrderProtosHelperBothNotStored(container,
+                                                   0,
+                                                   left_relation_block_ids_.size(),
+                                                   num_right_workorders_generated_,
+                                                   right_relation_block_ids_.size());
+
+      // Now generate new_left_blocks * (right).
+      new_workorders +=
+          getAllWorkOrderProtosHelperBothNotStored(container,
+                                                   num_left_workorders_generated_,
+                                                   left_relation_block_ids_.size(),
+                                                   0,
+                                                   num_right_workorders_generated_);
+    } else if (new_left_blocks == 0 && new_right_blocks > 0) {
+      // Only new right blocks are added. Generate left * new_right_blocks.
+      new_workorders =
+          getAllWorkOrderProtosHelperBothNotStored(container,
+                                                   0,
+                                                   left_relation_block_ids_.size(),
+                                                   num_right_workorders_generated_,
+                                                   right_relation_block_ids_.size());
+    } else if (new_left_blocks > 0 && new_right_blocks == 0) {
+      // Generate new_left_blocks * right
+      new_workorders =
+          getAllWorkOrderProtosHelperBothNotStored(container,
+                                                   num_left_workorders_generated_,
+                                                   left_relation_block_ids_.size(),
+                                                   0,
+                                                   right_relation_block_ids_.size());
+    }
+    if (new_workorders > 0) {
+      num_left_workorders_generated_ = left_relation_block_ids_.size();
+      num_right_workorders_generated_ = right_relation_block_ids_.size();
+    }
+    return done_feeding_left_relation_ && done_feeding_right_relation_;
+  } else {
+    // Only one relation is a stored relation.
+    return getAllWorkOrderProtosHelperOneStored(container);
+  }
+}
+
 std::size_t NestedLoopsJoinOperator::getAllWorkOrdersHelperBothNotStored(WorkOrdersContainer *container,
                                                                          QueryContext *query_context,
                                                                          StorageManager *storage_manager,
@@ -241,6 +309,80 @@ bool NestedLoopsJoinOperator::getAllWorkOrdersHelperOneStored(WorkOrdersContaine
   }
 }
 
+std::size_t NestedLoopsJoinOperator::getAllWorkOrderProtosHelperBothNotStored(
+    WorkOrderProtosContainer *container,
+    const std::vector<block_id>::size_type left_min,
+    const std::vector<block_id>::size_type left_max,
+    const std::vector<block_id>::size_type right_min,
+    const std::vector<block_id>::size_type right_max) {
+  DCHECK(!(left_relation_is_stored_ || right_relation_is_stored_));
+  DCHECK_LE(left_min, left_max);
+  DCHECK_LE(right_min, right_max);
+
+  for (std::vector<block_id>::size_type left_index = left_min;
+       left_index < left_max;
+       ++left_index) {
+    for (std::vector<block_id>::size_type right_index = right_min;
+         right_index < right_max;
+         ++right_index) {
+      container->addWorkOrderProto(
+          createWorkOrderProto(left_relation_block_ids_[left_index], right_relation_block_ids_[right_index]),
+          op_index_);
+    }
+  }
+  // Return the number of workorders produced.
+  return (left_max - left_min) * (right_max - right_min);
+}
+
+bool NestedLoopsJoinOperator::getAllWorkOrderProtosHelperOneStored(WorkOrderProtosContainer *container) {
+  DCHECK(left_relation_is_stored_ ^ right_relation_is_stored_);
+
+  if (left_relation_is_stored_) {
+    for (std::vector<block_id>::size_type right_index = num_right_workorders_generated_;
+         right_index < right_relation_block_ids_.size();
+         ++right_index) {
+      for (const block_id left_block_id : left_relation_block_ids_) {
+        container->addWorkOrderProto(
+            createWorkOrderProto(left_block_id, right_relation_block_ids_[right_index]),
+            op_index_);
+      }
+    }
+    num_right_workorders_generated_ = right_relation_block_ids_.size();
+    return done_feeding_right_relation_;
+  } else {
+    for (std::vector<block_id>::size_type left_index = num_left_workorders_generated_;
+         left_index < left_relation_block_ids_.size();
+         ++left_index) {
+      for (const block_id right_block_id : right_relation_block_ids_) {
+        container->addWorkOrderProto(
+            createWorkOrderProto(left_relation_block_ids_[left_index], right_block_id),
+            op_index_);
+      }
+    }
+    num_left_workorders_generated_ = left_relation_block_ids_.size();
+    return done_feeding_left_relation_;
+  }
+}
+
+serialization::WorkOrder* NestedLoopsJoinOperator::createWorkOrderProto(const block_id left_block,
+                                                                        const block_id right_block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::NESTED_LOOP_JOIN);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::left_relation_id, left_input_relation_.getID());
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::right_relation_id, right_input_relation_.getID());
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::left_block_id, left_block);
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::right_block_id, right_block);
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::insert_destination_index,
+                      output_destination_index_);
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::join_predicate_index, join_predicate_index_);
+  proto->SetExtension(serialization::NestedLoopsJoinWorkOrder::selection_index, selection_index_);
+
+  return proto;
+}
+
+
 template <bool LEFT_PACKED, bool RIGHT_PACKED>
 void NestedLoopsJoinWorkOrder::executeHelper(const TupleStorageSubBlock &left_store,
                                              const TupleStorageSubBlock &right_store) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index 0b13842..992e76d 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -44,8 +44,11 @@ class Predicate;
 class Scalar;
 class StorageManager;
 class TupleStorageSubBlock;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -119,6 +122,8 @@ class NestedLoopsJoinOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   void doneFeedingInputBlocks(const relation_id rel_id) override {
     if (rel_id == left_input_relation_.getID()) {
       done_feeding_left_relation_ = true;
@@ -187,6 +192,52 @@ class NestedLoopsJoinOperator : public RelationalOperator {
                                        QueryContext *query_context,
                                        StorageManager *storage_manager);
 
+  /**
+   * @brief Pairs block IDs from left and right relation block IDs and generates
+   *        NestedLoopsJoinWorkOrder protos and pushes them to the
+   *        WorkOrderProtosContainer when both relations are not stored
+   *        relations.
+   *
+   * @param container A pointer to the WorkOrderProtosContainer to store the
+   *                  resulting WorkOrder protos.
+   * @param left_min The starting index in left_relation_block_ids_ from where
+   *                 we begin generating NestedLoopsJoinWorkOrders.
+   * @param left_max The index in left_relation_block_ids_ until which we
+   *                 generate NestedLoopsJoinWorkOrders (excluding left_max).
+   * @param right_min The starting index in right_relation_block_ids_ from where
+   *                  we begin generating NestedLoopsJoinWorkOrders.
+   * @param right_max The index in right_relation_block_ids_ until which we
+   *                  generate NestedLoopsJoinWorkOrders. (excluding right_max).
+   *
+   * @return The number of workorder protos generated during the execution of this
+   *         function.
+   **/
+  std::size_t getAllWorkOrderProtosHelperBothNotStored(WorkOrderProtosContainer *container,
+                                                       const std::vector<block_id>::size_type left_min,
+                                                       const std::vector<block_id>::size_type left_max,
+                                                       const std::vector<block_id>::size_type right_min,
+                                                       const std::vector<block_id>::size_type right_max);
+
+  /**
+   * @brief Pairs block IDs from left and right relation block IDs and generates
+   *        NestedLoopsJoinWorkOrder protos and pushes them to the
+   *        WorkOrderProtosContainer when only one relation is a stored relation.
+   *
+   * @param container A pointer to the WorkOrderProtosContainer to store the
+   *                  resulting WorkOrder protos.
+   *
+   * @return Whether all work orders have been generated.
+   **/
+  bool getAllWorkOrderProtosHelperOneStored(WorkOrderProtosContainer *container);
+
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id left_block,
+                                                 const block_id right_block);
+
   const CatalogRelation &left_input_relation_;
   const CatalogRelation &right_input_relation_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index c173a0a..116727b 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -36,6 +36,7 @@ namespace tmb { class MessageBus; }
 namespace quickstep {
 
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -82,6 +83,27 @@ class RelationalOperator {
                                 tmb::MessageBus *bus) = 0;
 
   /**
+    * @brief For the distributed version, generate all the next WorkOrder protos
+    *        for this RelationalOperator
+    *
+    * @note If a RelationalOperator has blocking dependencies, it should not
+    *       generate workorders unless all of the blocking dependencies have been
+    *       met.
+    *
+    * @note If a RelationalOperator is not parallelizeable on a block-level, then
+    *       only one WorkOrder consisting of all the work for this
+    *       RelationalOperator should be generated.
+    *
+    * @param container A pointer to a WorkOrderProtosContainer to be used to
+    *        store the generated WorkOrder protos.
+    *
+    * @return Whether the operator has finished generating work order protos. If
+    *         \c false, the execution engine will invoke this method after at
+    *         least one pending work order has finished executing.
+    **/
+  virtual bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) = 0;
+
+  /**
    * @brief Update Catalog upon the completion of this RelationalOperator, if
    *        necessary.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SampleOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.cpp b/relational_operators/SampleOperator.cpp
index 8d5fade..5e5a417 100644
--- a/relational_operators/SampleOperator.cpp
+++ b/relational_operators/SampleOperator.cpp
@@ -1,6 +1,7 @@
 /**
  *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
  *     University of Wisconsin\u2014Madison.
+ *   Copyright 2016 Pivotal Software, Inc.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -18,10 +19,13 @@
 #include "relational_operators/SampleOperator.hpp"
 
 #include <memory>
+#include <random>
 #include <vector>
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -82,24 +86,9 @@ bool SampleOperator::getAllWorkOrders(
     }
     return started_;
   } else {
-      if (is_block_sample_) {
-          while (num_workorders_generated_ < input_relation_block_ids_.size()) {
-            if (distribution(generator) <= probability) {
-              container->addNormalWorkOrder(
-                  new SampleWorkOrder(
-                      query_id_,
-                      input_relation_,
-                      input_relation_block_ids_[num_workorders_generated_],
-                      is_block_sample_,
-                      percentage_,
-                      output_destination,
-                      storage_manager),
-                  op_index_);
-            ++num_workorders_generated_;
-          }
-        }
-      } else  {
-        while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+    if (is_block_sample_) {
+      while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+        if (distribution(generator) <= probability) {
           container->addNormalWorkOrder(
               new SampleWorkOrder(
                   query_id_,
@@ -113,10 +102,86 @@ bool SampleOperator::getAllWorkOrders(
           ++num_workorders_generated_;
         }
       }
+    } else {
+      while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+        container->addNormalWorkOrder(
+            new SampleWorkOrder(
+                query_id_,
+                input_relation_,
+                input_relation_block_ids_[num_workorders_generated_],
+                is_block_sample_,
+                percentage_,
+                output_destination,
+                storage_manager),
+            op_index_);
+        ++num_workorders_generated_;
+      }
+    }
     return done_feeding_input_relation_;
   }
 }
 
+bool SampleOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  std::random_device random_device;
+  std::mt19937 generator(random_device());
+  std::uniform_real_distribution<> distribution(0, 1);
+  const double probability = static_cast<double>(percentage_) / 100;
+
+  if (input_relation_is_stored_) {
+    if (!started_) {
+      // If the sampling is by block choose blocks randomly
+      if (is_block_sample_) {
+        for (const block_id input_block_id : input_relation_block_ids_) {
+          if (distribution(generator) <= probability) {
+            container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
+          }
+        }
+      } else {
+        // Add all the blocks for tuple sampling which would handle
+        // the sampling from each block
+        for (const block_id input_block_id : input_relation_block_ids_) {
+          container->addWorkOrderProto(createWorkOrderProto(input_block_id), op_index_);
+        }
+      }
+      started_ = true;
+    }
+    return true;
+  } else {
+    if (is_block_sample_) {
+      while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+        if (distribution(generator) <= probability) {
+          container->addWorkOrderProto(
+              createWorkOrderProto(input_relation_block_ids_[num_workorders_generated_]),
+              op_index_);
+          ++num_workorders_generated_;
+        }
+      }
+    } 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_;
+      }
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* SampleOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::SAMPLE);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::SampleWorkOrder::relation_id, input_relation_.getID());
+  proto->SetExtension(serialization::SampleWorkOrder::block_id, block);
+  proto->SetExtension(serialization::SampleWorkOrder::is_block_sample, is_block_sample_);
+  proto->SetExtension(serialization::SampleWorkOrder::percentage, percentage_);
+  proto->SetExtension(serialization::SampleWorkOrder::insert_destination_index, output_destination_index_);
+
+  return proto;
+}
+
 void SampleWorkOrder::execute() {
   BlockReference block(
       storage_manager_->getBlock(input_block_id_, input_relation_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index 505daa2..f8fe5f6 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -42,8 +42,11 @@ class CatalogDatabase;
 class CatalogRelationSchema;
 class InsertDestination;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -96,6 +99,8 @@ class SampleOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
     input_relation_block_ids_.push_back(input_block_id);
   }
@@ -115,6 +120,13 @@ class SampleOperator : public RelationalOperator {
   }
 
  private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
   const CatalogRelation &input_relation_;
   const CatalogRelationSchema &output_relation_;
   const QueryContext::insert_destination_id output_destination_index_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SaveBlocksOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.cpp b/relational_operators/SaveBlocksOperator.cpp
index 8127d88..5e0f33d 100644
--- a/relational_operators/SaveBlocksOperator.cpp
+++ b/relational_operators/SaveBlocksOperator.cpp
@@ -19,7 +19,9 @@
 
 #include <vector>
 
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
 
@@ -46,6 +48,22 @@ bool SaveBlocksOperator::getAllWorkOrders(
   return done_feeding_input_relation_;
 }
 
+bool SaveBlocksOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  while (num_workorders_generated_ < destination_block_ids_.size()) {
+    serialization::WorkOrder *proto = new serialization::WorkOrder;
+    proto->set_work_order_type(serialization::SAVE_BLOCKS);
+    proto->set_query_id(query_id_);
+    proto->SetExtension(serialization::SaveBlocksWorkOrder::block_id,
+                        destination_block_ids_[num_workorders_generated_]);
+    proto->SetExtension(serialization::SaveBlocksWorkOrder::force, force_);
+
+    container->addWorkOrderProto(proto, op_index_);
+
+    ++num_workorders_generated_;
+  }
+  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);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index 6e2c72b..50032b6 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -37,6 +37,7 @@ namespace quickstep {
 
 class QueryContext;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
 /** \addtogroup RelationalOperators
@@ -69,6 +70,8 @@ class SaveBlocksOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  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 {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index e9a96f3..eb6277e 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -21,7 +21,9 @@
 #include <vector>
 
 #include "query_execution/QueryContext.hpp"
+#include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
+#include "relational_operators/WorkOrder.pb.h"
 #include "storage/InsertDestination.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"
@@ -170,6 +172,47 @@ bool SelectOperator::getAllWorkOrders(
   }
 }
 
+bool SelectOperator::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_);
+      }
+      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_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+serialization::WorkOrder* SelectOperator::createWorkOrderProto(const block_id block) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::SELECT);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::SelectWorkOrder::relation_id, input_relation_.getID());
+  proto->SetExtension(serialization::SelectWorkOrder::insert_destination_index, output_destination_index_);
+  proto->SetExtension(serialization::SelectWorkOrder::predicate_index, predicate_index_);
+  proto->SetExtension(serialization::SelectWorkOrder::block_id, block);
+  proto->SetExtension(serialization::SelectWorkOrder::simple_projection, simple_projection_);
+  if (simple_projection_) {
+    for (const attribute_id attr_id : simple_selection_) {
+      proto->AddExtension(serialization::SelectWorkOrder::simple_selection, attr_id);
+    }
+  }
+  proto->SetExtension(serialization::SelectWorkOrder::selection_index, selection_index_);
+
+  return proto;
+}
+
+
 void SelectWorkOrder::execute() {
   BlockReference block(
       storage_manager_->getBlock(input_block_id_, input_relation_, getPreferredNUMANodes()[0]));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c9214ecb/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index ac7b038..0c10686 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -49,8 +49,11 @@ class InsertDestination;
 class Predicate;
 class Scalar;
 class StorageManager;
+class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -192,6 +195,8 @@ class SelectOperator : public RelationalOperator {
                         const tmb::client_id scheduler_client_id,
                         tmb::MessageBus *bus) override;
 
+  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override;
+
   void feedInputBlock(const block_id input_block_id, const relation_id input_relation_id) override {
     if (input_relation_.hasPartitionScheme()) {
       const partition_id part_id =
@@ -243,6 +248,13 @@ class SelectOperator : public RelationalOperator {
                                    InsertDestination *output_destination);
 
  private:
+  /**
+   * @brief Create Work Order proto.
+   *
+   * @param block The block id used in the Work Order.
+   **/
+  serialization::WorkOrder* createWorkOrderProto(const block_id block);
+
   const CatalogRelation &input_relation_;
   const CatalogRelation &output_relation_;
   const QueryContext::insert_destination_id output_destination_index_;



[06/13] incubator-quickstep git commit: Long lived Foreman thread

Posted by zu...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/PolicyEnforcer.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcer.cpp b/query_execution/PolicyEnforcer.cpp
new file mode 100644
index 0000000..2145429
--- /dev/null
+++ b/query_execution/PolicyEnforcer.cpp
@@ -0,0 +1,183 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#include "query_execution/PolicyEnforcer.hpp"
+
+#include <cstddef>
+#include <memory>
+#include <queue>
+#include <utility>
+#include <unordered_map>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "query_execution/QueryExecutionMessages.pb.h"
+#include "query_execution/QueryManager.hpp"
+#include "query_optimizer/QueryHandle.hpp"
+#include "relational_operators/WorkOrder.hpp"
+
+#include "gflags/gflags.h"
+#include "glog/logging.h"
+
+namespace quickstep {
+
+DEFINE_uint64(max_msgs_per_dispatch_round, 20, "Maximum number of messages that"
+              " can be allocated in a single round of dispatch of messages to"
+              " the workers.");
+
+bool PolicyEnforcer::admitQuery(QueryHandle *query_handle) {
+  if (admitted_queries_.size() < kMaxConcurrentQueries) {
+    // Ok to admit the query.
+    const std::size_t query_id = query_handle->query_id();
+    if (admitted_queries_.find(query_id) == admitted_queries_.end()) {
+      // Query with the same ID not present, ok to admit.
+      admitted_queries_[query_id].reset(
+          new QueryManager(foreman_client_id_, num_numa_nodes_, query_handle,
+                           catalog_database_, storage_manager_, bus_));
+      return true;
+    } else {
+      LOG(ERROR) << "Query with the same ID " << query_id << " exists";
+      return false;
+    }
+  } else {
+    // This query will have to wait.
+    waiting_queries_.push(query_handle);
+    return false;
+  }
+}
+
+void PolicyEnforcer::processMessage(const TaggedMessage &tagged_message) {
+  // TODO(harshad) : Provide processXMessage() public functions in
+  // QueryManager, so that we need to extract message from the
+  // TaggedMessage only once.
+  std::size_t query_id;
+  switch (tagged_message.message_type()) {
+    case kWorkOrderCompleteMessage:  // Fall through.
+    case kRebuildWorkOrderCompleteMessage: {
+      serialization::WorkOrderCompletionMessage proto;
+      CHECK(proto.ParseFromArray(tagged_message.message(),
+                                 tagged_message.message_bytes()));
+      query_id = proto.query_id();
+      break;
+    }
+    case kCatalogRelationNewBlockMessage: {
+      serialization::CatalogRelationNewBlockMessage proto;
+      CHECK(proto.ParseFromArray(tagged_message.message(),
+                                 tagged_message.message_bytes()));
+      query_id = proto.query_id();
+      break;
+    }
+    case kDataPipelineMessage: {
+      serialization::DataPipelineMessage proto;
+      CHECK(proto.ParseFromArray(tagged_message.message(),
+                                 tagged_message.message_bytes()));
+      query_id = proto.query_id();
+      break;
+    }
+    case kWorkOrdersAvailableMessage: {
+      serialization::WorkOrdersAvailableMessage proto;
+      CHECK(proto.ParseFromArray(tagged_message.message(),
+                                 tagged_message.message_bytes()));
+      query_id = proto.query_id();
+      break;
+    }
+    case kWorkOrderFeedbackMessage: {
+      WorkOrder::FeedbackMessage msg(const_cast<void *>(tagged_message.message()), tagged_message.message_bytes());
+      query_id = msg.header().query_id;
+      break;
+    }
+    default:
+      LOG(FATAL) << "Unknown message type found in PolicyEnforcer";
+  }
+  DCHECK(admitted_queries_.find(query_id) != admitted_queries_.end());
+  const QueryManager::QueryStatusCode return_code =
+      admitted_queries_[query_id]->processMessage(tagged_message);
+  if (return_code == QueryManager::QueryStatusCode::kQueryExecuted) {
+    removeQuery(query_id);
+    if (!waiting_queries_.empty()) {
+      // Admit the earliest waiting query.
+      QueryHandle *new_query = waiting_queries_.front();
+      waiting_queries_.pop();
+      admitQuery(new_query);
+    }
+  }
+}
+
+void PolicyEnforcer::getWorkerMessages(
+    std::vector<std::unique_ptr<WorkerMessage>> *worker_messages) {
+  // Iterate over admitted queries until either there are no more
+  // messages available, or the maximum number of messages have
+  // been collected.
+  DCHECK(worker_messages->empty());
+  // TODO(harshad) - Make this function generic enough so that it
+  // works well when multiple queries are getting executed.
+  std::size_t per_query_share = 0;
+  if (!admitted_queries_.empty()) {
+    per_query_share = FLAGS_max_msgs_per_dispatch_round / admitted_queries_.size();
+  } else {
+    LOG(WARNING) << "Requesting WorkerMessages when no query is running";
+    return;
+  }
+  DCHECK_GT(per_query_share, 0u);
+  std::vector<std::size_t> finished_queries_ids;
+
+  for (const auto &admitted_query_info : admitted_queries_) {
+    QueryManager *curr_query_manager = admitted_query_info.second.get();
+    DCHECK(curr_query_manager != nullptr);
+    std::size_t messages_collected_curr_query = 0;
+    while (messages_collected_curr_query < per_query_share) {
+      WorkerMessage *next_worker_message =
+          curr_query_manager->getNextWorkerMessage(0, kAnyNUMANodeID);
+      if (next_worker_message != nullptr) {
+        ++messages_collected_curr_query;
+        worker_messages->push_back(std::unique_ptr<WorkerMessage>(next_worker_message));
+      } else {
+        // No more work ordes from the current query at this time.
+        // Check if the query's execution is over.
+        if (curr_query_manager->getQueryExecutionState().hasQueryExecutionFinished()) {
+          // If the query has been executed, remove it.
+          finished_queries_ids.push_back(admitted_query_info.first);
+        }
+        break;
+      }
+    }
+  }
+  for (const std::size_t finished_qid : finished_queries_ids) {
+    removeQuery(finished_qid);
+  }
+}
+
+void PolicyEnforcer::removeQuery(const std::size_t query_id) {
+  DCHECK(admitted_queries_.find(query_id) != admitted_queries_.end());
+  if (!admitted_queries_[query_id]->getQueryExecutionState().hasQueryExecutionFinished()) {
+    LOG(WARNING) << "Removing query with ID " << query_id
+                 << " that hasn't finished its execution";
+  }
+  admitted_queries_.erase(query_id);
+}
+
+bool PolicyEnforcer::admitQueries(
+    const std::vector<QueryHandle*> &query_handles) {
+  for (QueryHandle *curr_query : query_handles) {
+    if (!admitQuery(curr_query)) {
+      return false;
+    }
+  }
+  return true;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/PolicyEnforcer.hpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcer.hpp b/query_execution/PolicyEnforcer.hpp
new file mode 100644
index 0000000..5915b79
--- /dev/null
+++ b/query_execution/PolicyEnforcer.hpp
@@ -0,0 +1,167 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed under the Apache License, Version 2.0 (the "License");
+ *   you may not use this file except in compliance with the License.
+ *   You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_HPP_
+#define QUICKSTEP_QUERY_EXECUTION_POLICY_ENFORCER_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <queue>
+#include <unordered_map>
+#include <vector>
+
+#include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/QueryManager.hpp"
+#include "query_execution/WorkerMessage.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+#include "tmb/message_bus.h"
+#include "tmb/tagged_message.h"
+
+namespace quickstep {
+
+class CatalogDatabaseLite;
+class QueryHandle;
+class StorageManager;
+
+/**
+ * @brief A class that ensures that a high level policy is maintained
+ *        in sharing resources among concurrent queries.
+ **/
+class PolicyEnforcer {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param foreman_client_id The TMB client ID of the Foreman.
+   * @param num_numa_nodes Number of NUMA nodes used by the system.
+   * @param catalog_database The CatalogDatabase used.
+   * @param storage_manager The StorageManager used.
+   * @param bus The TMB.
+   **/
+  PolicyEnforcer(const tmb::client_id foreman_client_id,
+                 const std::size_t num_numa_nodes,
+                 CatalogDatabaseLite *catalog_database,
+                 StorageManager *storage_manager,
+                 tmb::MessageBus *bus)
+      : foreman_client_id_(foreman_client_id),
+        num_numa_nodes_(num_numa_nodes),
+        catalog_database_(catalog_database),
+        storage_manager_(storage_manager),
+        bus_(bus) {}
+
+  /**
+   * @brief Destructor.
+   **/
+  ~PolicyEnforcer() {
+    if (hasQueries()) {
+      LOG(WARNING) << "Destructing PolicyEnforcer with some unfinished or "
+                      "waiting queries";
+    }
+  }
+
+  /**
+   * @brief Admit a query to the system.
+   *
+   * @param query_handle The QueryHandle for the new query.
+   *
+   * @return Whether the query was admitted to the system.
+   **/
+  bool admitQuery(QueryHandle *query_handle);
+
+  /**
+   * @brief Admit multiple queries in the system.
+   *
+   * @note In the current simple implementation, we only allow one active
+   *       query in the system. Other queries will have to wait.
+   *
+   * @param query_handles A vector of QueryHandles for the queries to be
+   *        admitted.
+   *
+   * @return True if all the queries were admitted, false if at least one query
+   *         was not admitted.
+   **/
+  bool admitQueries(const std::vector<QueryHandle*> &query_handles);
+
+  /**
+   * @brief Remove a given query that is under execution.
+   *
+   * @note This function is made public so that it is possible for a query
+   *       to be killed. Otherwise, it should only be used privately by the
+   *       class.
+   *
+   * TODO(harshad) - Extend this function to support removal of waiting queries.
+   *
+   * @param query_id The ID of the query to be removed.
+   **/
+  void removeQuery(const std::size_t query_id);
+
+  /**
+   * @brief Get worker messages to be dispatched. These worker messages come
+   *        from the active queries.
+   *
+   * @param worker_messages The worker messages to be dispatched.
+   **/
+  void getWorkerMessages(
+      std::vector<std::unique_ptr<WorkerMessage>> *worker_messages);
+
+  /**
+   * @brief Process a message sent to the Foreman, which gets passed on to the
+   *        policy enforcer.
+   *
+   * @param message The message.
+   **/
+  void processMessage(const TaggedMessage &tagged_message);
+
+  /**
+   * @brief Check if there are any queries to be executed.
+   *
+   * @return True if there is at least one active or waiting query, false if
+   *         the policy enforcer doesn't have any query.
+   **/
+  inline bool hasQueries() const {
+    return !(admitted_queries_.empty() && waiting_queries_.empty());
+  }
+
+ private:
+  static constexpr std::size_t kMaxConcurrentQueries = 1;
+
+  const tmb::client_id foreman_client_id_;
+  const std::size_t num_numa_nodes_;
+
+  CatalogDatabaseLite *catalog_database_;
+  StorageManager *storage_manager_;
+
+  tmb::MessageBus *bus_;
+
+  // Key = query ID, value = QueryManager* for the key query.
+  std::unordered_map<std::size_t, std::unique_ptr<QueryManager>> admitted_queries_;
+
+  // The queries which haven't been admitted yet.
+  std::queue<QueryHandle*> waiting_queries_;
+
+  DISALLOW_COPY_AND_ASSIGN(PolicyEnforcer);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_EXECUTION_QUERY_MANAGER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/QueryContext.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.cpp b/query_execution/QueryContext.cpp
index 3bfce17..54dd557 100644
--- a/query_execution/QueryContext.cpp
+++ b/query_execution/QueryContext.cpp
@@ -89,13 +89,13 @@ QueryContext::QueryContext(const serialization::QueryContext &proto,
 
   for (int i = 0; i < proto.insert_destinations_size(); ++i) {
     const serialization::InsertDestination &insert_destination_proto = proto.insert_destinations(i);
-    insert_destinations_.emplace_back(
-        InsertDestination::ReconstructFromProto(insert_destination_proto,
-                                                database.getRelationSchemaById(
-                                                    insert_destination_proto.relation_id()),
-                                                storage_manager,
-                                                scheduler_client_id,
-                                                bus));
+    insert_destinations_.emplace_back(InsertDestination::ReconstructFromProto(
+        proto.query_id(),
+        insert_destination_proto,
+        database.getRelationSchemaById(insert_destination_proto.relation_id()),
+        storage_manager,
+        scheduler_client_id,
+        bus));
   }
 
   for (int i = 0; i < proto.predicates_size(); ++i) {
@@ -231,7 +231,7 @@ bool QueryContext::ProtoIsValid(const serialization::QueryContext &proto,
     }
   }
 
-  return true;
+  return proto.IsInitialized();
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/QueryContext.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryContext.proto b/query_execution/QueryContext.proto
index b37286c..98cd0b6 100644
--- a/query_execution/QueryContext.proto
+++ b/query_execution/QueryContext.proto
@@ -54,4 +54,6 @@ message QueryContext {
 
   // NOTE(zuyu): For UpdateWorkOrder only.
   repeated UpdateGroup update_groups = 10;
+
+  required uint64 query_id = 11;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/QueryExecutionMessages.proto
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionMessages.proto b/query_execution/QueryExecutionMessages.proto
index 15803cf..9d9a9e5 100644
--- a/query_execution/QueryExecutionMessages.proto
+++ b/query_execution/QueryExecutionMessages.proto
@@ -27,6 +27,7 @@ message EmptyMessage {
 message WorkOrderCompletionMessage {
   required uint64 operator_index = 1;
   required uint64 worker_thread_index = 2;
+  required uint64 query_id = 3;
 }
 
 message CatalogRelationNewBlockMessage {
@@ -35,16 +36,19 @@ message CatalogRelationNewBlockMessage {
 
   // Used by PartitionAwareInsertDestination.
   optional uint64 partition_id = 3;
+  required uint64 query_id = 4;
 }
 
 message DataPipelineMessage {
   required uint64 operator_index = 1;
   required fixed64 block_id = 2;
   required int32 relation_id = 3;
+  required uint64 query_id = 4;
 }
 
 message WorkOrdersAvailableMessage {
   required uint64 operator_index = 1;
+  required uint64 query_id = 2;
 }
 
 // BlockLocator related messages.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/QueryExecutionTypedefs.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionTypedefs.hpp b/query_execution/QueryExecutionTypedefs.hpp
index fc253bc..9d1060f 100644
--- a/query_execution/QueryExecutionTypedefs.hpp
+++ b/query_execution/QueryExecutionTypedefs.hpp
@@ -58,6 +58,8 @@ using ClientIDMap = ThreadIDBasedMap<client_id,
 
 // We sort the following message types in the order of a life cycle of a query.
 enum QueryExecutionMessageType : message_type_id {
+  kAdmitRequestMessage,  // Requesting a query (or queries) to be admitted, from
+                         // the main thread to Foreman.
   kWorkOrderMessage,  // From Foreman to Worker.
   kWorkOrderCompleteMessage,  // From Worker to Foreman.
   kCatalogRelationNewBlockMessage,  // From InsertDestination to Foreman.
@@ -67,7 +69,8 @@ enum QueryExecutionMessageType : message_type_id {
                               // their corresponding RelationalOperators.
   kRebuildWorkOrderMessage,  // From Foreman to Worker.
   kRebuildWorkOrderCompleteMessage,  // From Worker to Foreman.
-  kPoisonMessage,  // From the CLI shell to Foreman, then from Foreman to Workers.
+  kWorkloadCompletionMessage,  // From Foreman to main thread.
+  kPoisonMessage,  // From the main thread to Foreman and Workers.
 
 #ifdef QUICKSTEP_DISTRIBUTED
   // BlockLocator related messages, sorted in a life cycle of StorageManager

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/QueryExecutionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryExecutionUtil.hpp b/query_execution/QueryExecutionUtil.hpp
index a8b6a38..78fd159 100644
--- a/query_execution/QueryExecutionUtil.hpp
+++ b/query_execution/QueryExecutionUtil.hpp
@@ -17,9 +17,12 @@
 #ifndef QUICKSTEP_QUERY_EXECUTION_QUERY_EXECUTION_UTIL_HPP_
 #define QUICKSTEP_QUERY_EXECUTION_QUERY_EXECUTION_UTIL_HPP_
 
+#include <memory>
 #include <utility>
 
+#include "query_execution/AdmitRequestMessage.hpp"
 #include "query_execution/QueryExecutionTypedefs.hpp"
+#include "query_execution/WorkerMessage.hpp"
 #include "utility/Macros.hpp"
 
 #include "tmb/address.h"
@@ -60,6 +63,55 @@ class QueryExecutionUtil {
                      std::move(tagged_message));
   }
 
+  /**
+   * @brief Construct and send an AdmitRequestMessage from a given sender to a
+   *        given recipient.
+   *
+   * @param sender_id The TMB client ID of the sender.
+   * @param receiver_id The TMB client ID of the receiver.
+   * @param query_handle The QueryHandle used in the AdmitRequestMessage.
+   * @param bus A pointer to the TMB.
+   * @param tagged_message A moved from reference to the tagged message.
+   *
+   * @return A status code indicating the result of the message delivery.
+   *         The caller should ensure that the status is SendStatus::kOK.
+   **/
+  static tmb::MessageBus::SendStatus ConstructAndSendAdmitRequestMessage(
+      const tmb::client_id sender_id,
+      const tmb::client_id receiver_id,
+      QueryHandle *query_handle,
+      MessageBus *bus) {
+    std::unique_ptr<AdmitRequestMessage> request_message(
+        new AdmitRequestMessage(query_handle));
+    const std::size_t size_of_request_msg = sizeof(*request_message);
+    TaggedMessage admit_tagged_message(
+        request_message.release(), size_of_request_msg, kAdmitRequestMessage);
+
+    return QueryExecutionUtil::SendTMBMessage(
+        bus, sender_id, receiver_id, std::move(admit_tagged_message));
+  }
+
+  static void BroadcastPoisonMessage(const tmb::client_id sender_id, tmb::MessageBus *bus) {
+    // Terminate all threads.
+    // The sender thread broadcasts poison message to the workers and foreman.
+    // Each worker dies after receiving poison message. The order of workers'
+    // death is irrelavant.
+    MessageStyle style;
+    style.Broadcast(true);
+    Address address;
+    address.All(true);
+    std::unique_ptr<WorkerMessage> poison_message(WorkerMessage::PoisonMessage());
+    TaggedMessage poison_tagged_message(poison_message.get(),
+                                        sizeof(*poison_message),
+                                        kPoisonMessage);
+
+    const tmb::MessageBus::SendStatus send_status = bus->Send(
+        sender_id, address, style, std::move(poison_tagged_message));
+    CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
+       "Broadcast poison message from sender with TMB client ID " << sender_id
+       << " failed";
+  }
+
  private:
   /**
    * @brief Constructor. Made private to avoid instantiation.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/QueryManager.hpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManager.hpp b/query_execution/QueryManager.hpp
index 47f54c5..b52460f 100644
--- a/query_execution/QueryManager.hpp
+++ b/query_execution/QueryManager.hpp
@@ -25,18 +25,21 @@
 #include "catalog/CatalogTypedefs.hpp"
 #include "query_execution/QueryContext.hpp"
 #include "query_execution/QueryExecutionState.hpp"
+#include "query_execution/QueryExecutionTypedefs.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
 #include "relational_operators/RelationalOperator.hpp"
+#include "relational_operators/WorkOrder.hpp"
+#include "storage/StorageBlockInfo.hpp"
 #include "utility/DAG.hpp"
 #include "utility/Macros.hpp"
 
+#include "tmb/id_typedefs.h"
 #include "tmb/message_bus.h"
 #include "tmb/tagged_message.h"
 
 namespace quickstep {
 
 class CatalogDatabaseLite;
-class ForemanMessage;
 class QueryHandle;
 class StorageManager;
 class WorkerMessage;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/WorkOrdersContainer.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkOrdersContainer.hpp b/query_execution/WorkOrdersContainer.hpp
index eb9aedd..d023daa 100644
--- a/query_execution/WorkOrdersContainer.hpp
+++ b/query_execution/WorkOrdersContainer.hpp
@@ -28,6 +28,8 @@
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 
 /** \addtogroup QueryExecution
@@ -76,7 +78,7 @@ class WorkOrdersContainer {
    * @return If there are pending WorkOrders.
    **/
   inline bool hasNormalWorkOrder(const std::size_t operator_index) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK_LT(operator_index, num_operators_);
     return normal_workorders_[operator_index].hasWorkOrder();
   }
 
@@ -92,9 +94,9 @@ class WorkOrdersContainer {
    **/
   inline bool hasNormalWorkOrderForNUMANode(
       const std::size_t operator_index, const int numa_node_id) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
-    DEBUG_ASSERT(numa_node_id >= 0);
-    DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+    DCHECK_LT(operator_index, num_operators_);
+    DCHECK_GE(numa_node_id, 0);
+    DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
     return normal_workorders_[operator_index].hasWorkOrderForNUMANode(
         numa_node_id);
   }
@@ -108,7 +110,7 @@ class WorkOrdersContainer {
    * @return If there are pending rebuild WorkOrders.
    **/
   inline bool hasRebuildWorkOrder(const std::size_t operator_index) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK_LT(operator_index, num_operators_);
     return rebuild_workorders_[operator_index].hasWorkOrder();
   }
 
@@ -124,9 +126,9 @@ class WorkOrdersContainer {
    **/
   inline bool hasRebuildWorkOrderForNUMANode(
       const std::size_t operator_index, const int numa_node_id) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
-    DEBUG_ASSERT(numa_node_id >= 0);
-    DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+    DCHECK_LT(operator_index, num_operators_);
+    DCHECK_GE(numa_node_id, 0);
+    DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
     return rebuild_workorders_[operator_index].hasWorkOrderForNUMANode(
         numa_node_id);
   }
@@ -144,9 +146,9 @@ class WorkOrdersContainer {
    **/
   WorkOrder* getNormalWorkOrderForNUMANode(const std::size_t operator_index,
                                            const int numa_node_id) {
-    DEBUG_ASSERT(operator_index < num_operators_);
-    DEBUG_ASSERT(numa_node_id >= 0);
-    DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+    DCHECK_LT(operator_index, num_operators_);
+    DCHECK_GE(numa_node_id, 0);
+    DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
     return normal_workorders_[operator_index].getWorkOrderForNUMANode(
         numa_node_id);
   }
@@ -164,7 +166,7 @@ class WorkOrdersContainer {
    **/
   WorkOrder* getNormalWorkOrder(const std::size_t operator_index,
                                 const bool prefer_single_NUMA_node = true) {
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK_LT(operator_index, num_operators_);
     return normal_workorders_[operator_index].getWorkOrder(
         prefer_single_NUMA_node);
   }
@@ -182,9 +184,9 @@ class WorkOrdersContainer {
    **/
   WorkOrder* getRebuildWorkOrderForNUMANode(const std::size_t operator_index,
                                             const int numa_node_id) {
-    DEBUG_ASSERT(operator_index < num_operators_);
-    DEBUG_ASSERT(numa_node_id >= 0);
-    DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+    DCHECK_LT(operator_index, num_operators_);
+    DCHECK_GE(numa_node_id, 0);
+    DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
     return rebuild_workorders_[operator_index].getWorkOrderForNUMANode(
         numa_node_id);
   }
@@ -202,7 +204,7 @@ class WorkOrdersContainer {
    **/
   WorkOrder* getRebuildWorkOrder(const std::size_t operator_index,
                                  const bool prefer_single_NUMA_node = true) {
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK_LT(operator_index, num_operators_);
     return rebuild_workorders_[operator_index].getWorkOrder(
         prefer_single_NUMA_node);
   }
@@ -220,8 +222,8 @@ class WorkOrdersContainer {
    * @param operator_index The index of the operator in the query DAG.
    **/
   void addNormalWorkOrder(WorkOrder *workorder, const std::size_t operator_index) {
-    DEBUG_ASSERT(workorder != nullptr);
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK(workorder != nullptr);
+    DCHECK_LT(operator_index, num_operators_);
     normal_workorders_[operator_index].addWorkOrder(workorder);
   }
 
@@ -238,8 +240,8 @@ class WorkOrdersContainer {
    **/
   void addRebuildWorkOrder(WorkOrder *workorder,
                            const std::size_t operator_index) {
-    DEBUG_ASSERT(workorder != nullptr);
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK(workorder != nullptr);
+    DCHECK_LT(operator_index, num_operators_);
     rebuild_workorders_[operator_index].addWorkOrder(workorder);
   }
 
@@ -254,9 +256,9 @@ class WorkOrdersContainer {
    **/
   inline std::size_t getNumNormalWorkOrdersForNUMANode(
       const std::size_t operator_index, const int numa_node_id) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
-    DEBUG_ASSERT(numa_node_id >= 0);
-    DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+    DCHECK_LT(operator_index, num_operators_);
+    DCHECK_GE(numa_node_id, 0);
+    DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
     return normal_workorders_[operator_index].getNumWorkOrdersForNUMANode(
         numa_node_id);
   }
@@ -271,7 +273,7 @@ class WorkOrdersContainer {
    **/
   inline std::size_t getNumNormalWorkOrders(
       const std::size_t operator_index) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK_LT(operator_index, num_operators_);
     return normal_workorders_[operator_index].getNumWorkOrders();
   }
 
@@ -286,9 +288,9 @@ class WorkOrdersContainer {
    **/
   inline std::size_t getNumRebuildWorkOrdersForNUMANode(
       const std::size_t operator_index, const int numa_node_id) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
-    DEBUG_ASSERT(numa_node_id >= 0);
-    DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+    DCHECK_LT(operator_index, num_operators_);
+    DCHECK_GE(numa_node_id, 0);
+    DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
     return rebuild_workorders_[operator_index].getNumWorkOrdersForNUMANode(
         numa_node_id);
   }
@@ -303,7 +305,7 @@ class WorkOrdersContainer {
    **/
   inline std::size_t getNumRebuildWorkOrders(
       const std::size_t operator_index) const {
-    DEBUG_ASSERT(operator_index < num_operators_);
+    DCHECK_LT(operator_index, num_operators_);
     return rebuild_workorders_[operator_index].getNumWorkOrders();
   }
 
@@ -418,8 +420,8 @@ class WorkOrdersContainer {
     void addWorkOrder(WorkOrder *workorder);
 
     bool hasWorkOrderForNUMANode(const int numa_node_id) const {
-      DEBUG_ASSERT(numa_node_id >= 0);
-      DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+      DCHECK_GE(numa_node_id, 0);
+      DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
       return single_numa_node_workorders_[numa_node_id].hasWorkOrder() ||
              multiple_numa_nodes_workorders_.hasWorkOrderForNUMANode(
                  numa_node_id);
@@ -440,8 +442,8 @@ class WorkOrdersContainer {
 
     std::size_t getNumWorkOrdersForNUMANode(
         const int numa_node_id) const {
-      DEBUG_ASSERT(numa_node_id >= 0);
-      DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+      DCHECK_GE(numa_node_id, 0);
+      DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
       return single_numa_node_workorders_[numa_node_id].getNumWorkOrders() +
              multiple_numa_nodes_workorders_.getNumWorkOrdersForNUMANode(
                  numa_node_id);
@@ -463,8 +465,8 @@ class WorkOrdersContainer {
     }
 
     WorkOrder* getWorkOrderForNUMANode(const int numa_node_id) {
-      DEBUG_ASSERT(numa_node_id >= 0);
-      DEBUG_ASSERT(static_cast<std::size_t>(numa_node_id) < num_numa_nodes_);
+      DCHECK_GE(numa_node_id, 0);
+      DCHECK_LT(static_cast<std::size_t>(numa_node_id), num_numa_nodes_);
       WorkOrder *work_order = single_numa_node_workorders_[numa_node_id].getWorkOrder();
       if (work_order == nullptr) {
         work_order = multiple_numa_nodes_workorders_.getWorkOrderForNUMANode(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/Worker.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.cpp b/query_execution/Worker.cpp
index 645fd05..ef596e1 100644
--- a/query_execution/Worker.cpp
+++ b/query_execution/Worker.cpp
@@ -58,11 +58,14 @@ void Worker::run() {
         WorkerMessage message(*static_cast<const WorkerMessage*>(tagged_message.message()));
         DCHECK(message.getWorkOrder() != nullptr);
         message.getWorkOrder()->execute();
+        const std::size_t query_id_for_workorder =
+            message.getWorkOrder()->getQueryID();
         delete message.getWorkOrder();
 
-        sendWorkOrderCompleteMessage(annotated_msg.sender,
-                                     message.getRelationalOpIndex(),
-                                     tagged_message.message_type() == kRebuildWorkOrderMessage);
+        sendWorkOrderCompleteMessage(
+            annotated_msg.sender, message.getRelationalOpIndex(),
+            query_id_for_workorder,
+            tagged_message.message_type() == kRebuildWorkOrderMessage);
         break;
       }
       case kPoisonMessage: {
@@ -76,10 +79,12 @@ void Worker::run() {
 
 void Worker::sendWorkOrderCompleteMessage(const tmb::client_id receiver,
                                           const size_t op_index,
+                                          const size_t query_id,
                                           const bool is_rebuild_work_order) {
   serialization::WorkOrderCompletionMessage proto;
   proto.set_operator_index(op_index);
   proto.set_worker_thread_index(worker_thread_index_);
+  proto.set_query_id(query_id);
 
   // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
   const size_t proto_length = proto.ByteSize();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/Worker.hpp
----------------------------------------------------------------------
diff --git a/query_execution/Worker.hpp b/query_execution/Worker.hpp
index b94e937..c0bafdc 100644
--- a/query_execution/Worker.hpp
+++ b/query_execution/Worker.hpp
@@ -97,11 +97,13 @@ class Worker : public Thread {
    *
    * @param receiver The id of the TMB client which should receive the response.
    * @param op_index The index of the operator to which the WorkOrder belongs.
+   * @param query_id The ID of the query which the WorkOrder belongs to.
    * @param is_rebuild_work_order True if it is a RebuildWorkOrder. Otherwise
    *        false.
    **/
   void sendWorkOrderCompleteMessage(const tmb::client_id receiver,
                                     const std::size_t op_index,
+                                    const std::size_t query_id,
                                     const bool is_rebuild_work_order);
 
   const std::size_t worker_thread_index_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/WorkerMessage.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkerMessage.hpp b/query_execution/WorkerMessage.hpp
index ec63af9..560c1ba 100644
--- a/query_execution/WorkerMessage.hpp
+++ b/query_execution/WorkerMessage.hpp
@@ -30,6 +30,8 @@ class WorkOrder;
  **/
 class WorkerMessage {
  public:
+  static constexpr int kInvalidRecipientIndexHint = -1;
+
   enum class WorkerMessageType {
     kRebuildWorkOrder = 0,
     kWorkOrder,
@@ -105,6 +107,23 @@ class WorkerMessage {
     return type_;
   }
 
+  /**
+   * @brief Set a hint for the recipient worker thread.
+   *
+   * @param recipient_index_hint The hint i.e. the worker thread index.
+   **/
+  inline void setRecipientHint(const int recipient_index_hint) {
+    recipient_index_hint_ = recipient_index_hint;
+  }
+
+  /**
+   * @brief Get the hint for the recipient worker thread. The hint is invalid if
+   *        it is kInvalidRecipientIndexHint.
+   **/
+  inline int getRecipientHint() const {
+    return recipient_index_hint_;
+  }
+
  private:
   /**
    * @brief Constructor.
@@ -120,12 +139,13 @@ class WorkerMessage {
                 const WorkerMessageType type)
       : work_unit_(work_unit),
         relational_op_index_(relational_op_index),
-        type_(type) {
-  }
+        type_(type),
+        recipient_index_hint_(kInvalidRecipientIndexHint) {}
 
   WorkOrder *work_unit_;
   const std::size_t relational_op_index_;
   const WorkerMessageType type_;
+  int recipient_index_hint_;
 };
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/tests/Foreman_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/Foreman_unittest.cpp b/query_execution/tests/Foreman_unittest.cpp
deleted file mode 100644
index cbe5088..0000000
--- a/query_execution/tests/Foreman_unittest.cpp
+++ /dev/null
@@ -1,952 +0,0 @@
-/**
- *   Copyright 2011-2015 Quickstep Technologies LLC.
- *   Copyright 2015-2016 Pivotal Software, Inc.
- *
- *   Licensed under the Apache License, Version 2.0 (the "License");
- *   you may not use this file except in compliance with the License.
- *   You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- **/
-
-#include <climits>
-#include <memory>
-#include <utility>
-#include <vector>
-
-#include "catalog/CatalogDatabase.hpp"
-#include "catalog/CatalogRelation.hpp"
-#include "catalog/CatalogTypedefs.hpp"
-#include "query_execution/Foreman.hpp"
-#include "query_execution/QueryContext.hpp"
-#include "query_execution/QueryContext.pb.h"
-#include "query_execution/QueryExecutionState.hpp"
-#include "query_execution/QueryExecutionTypedefs.hpp"
-#include "query_execution/WorkOrdersContainer.hpp"
-#include "query_execution/WorkerDirectory.hpp"
-#include "query_execution/WorkerMessage.hpp"
-#include "query_optimizer/QueryPlan.hpp"
-#include "relational_operators/RelationalOperator.hpp"
-#include "relational_operators/WorkOrder.hpp"
-#include "storage/InsertDestination.hpp"
-#include "storage/InsertDestination.pb.h"
-#include "storage/StorageBlock.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/StorageManager.hpp"
-#include "utility/DAG.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-#include "gtest/gtest.h"
-
-#include "tmb/id_typedefs.h"
-#include "tmb/message_bus.h"
-#include "tmb/tagged_message.h"
-
-using std::move;
-using std::unique_ptr;
-using std::vector;
-
-using tmb::client_id;
-
-namespace quickstep {
-
-class WorkOrderProtosContainer;
-
-class MockWorkOrder : public WorkOrder {
- public:
-  explicit MockWorkOrder(const int op_index)
-      : WorkOrder(0), op_index_(op_index) {}
-
-  void execute() override {
-    VLOG(3) << "WorkOrder[" << op_index_ << "] executing.";
-  }
-
-  inline QueryPlan::DAGNodeIndex getOpIndex() const {
-    return op_index_;
-  }
-
- private:
-  const QueryPlan::DAGNodeIndex op_index_;
-
-  DISALLOW_COPY_AND_ASSIGN(MockWorkOrder);
-};
-
-class MockOperator: public RelationalOperator {
- public:
-  enum function_name {
-    kFeedInputBlock = 0,
-    kFeedInputBlocks,
-    kDoneFeedingInputBlocks,
-    kGetAllWorkOrders
-  };
-
-  MockOperator(const bool produce_workorders,
-               const bool has_streaming_input,
-               const int max_getworkorder_iters = 1,
-               const int max_workorders = INT_MAX)
-      : RelationalOperator(0 /* Query Id */),
-        produce_workorders_(produce_workorders),
-        has_streaming_input_(has_streaming_input),
-        max_workorders_(max_workorders),
-        max_getworkorder_iters_(max_getworkorder_iters),
-        num_calls_get_workorders_(0),
-        num_workorders_generated_(0),
-        num_calls_feedblock_(0),
-        num_calls_feedblocks_(0),
-        num_calls_donefeedingblocks_(0) {
-  }
-
-#define MOCK_OP_LOG(x) VLOG(x) << "Op[" << op_index_ << "]: " << __func__ << ": "
-
-  // The methods below are used to check whether Foreman calls the Relational
-  // operator, how many times it calls a particular method etc.
-  inline int getNumWorkOrders() const {
-    return num_workorders_generated_;
-  }
-
-  inline int getNumCalls(const function_name fname) const {
-    switch (fname) {
-      case kFeedInputBlock:
-        return num_calls_feedblock_;
-      case kFeedInputBlocks:
-        return num_calls_feedblocks_;
-      case kDoneFeedingInputBlocks:
-        return num_calls_donefeedingblocks_;
-      case kGetAllWorkOrders:
-        return num_calls_get_workorders_;
-      default:
-        return -1;
-    }
-  }
-
-  inline bool getBlockingDependenciesMet() const {
-    MOCK_OP_LOG(3) << "met.";
-    return blocking_dependencies_met_;
-  }
-
-  void setInsertDestinationID(const QueryContext::insert_destination_id insert_destination_index) {
-    insert_destination_index_ = insert_destination_index;
-  }
-
-  // Mock to trigger doneFeedingInputBlocks for the dependent operators
-  // in Foreman::markOperatorFinished.
-  void setOutputRelationID(const relation_id rel_id) {
-    output_relation_id_ = rel_id;
-  }
-
-  // Override methods from the base class.
-  bool getAllWorkOrders(
-      WorkOrdersContainer *container,
-      QueryContext *query_context,
-      StorageManager *storage_manager,
-      const tmb::client_id foreman_client_id,
-      tmb::MessageBus *bus) override {
-    ++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_)) {
-          MOCK_OP_LOG(3) << "[stream] generate WorkOrder";
-          container->addNormalWorkOrder(new MockWorkOrder(op_index_), op_index_);
-          ++num_workorders_generated_;
-        }
-      } else {
-        if (blocking_dependencies_met_ && (num_workorders_generated_ < max_workorders_)) {
-          MOCK_OP_LOG(3) << "[static] generate WorkOrder";
-          container->addNormalWorkOrder(new MockWorkOrder(op_index_), op_index_);
-          ++num_workorders_generated_;
-        }
-      }
-    }
-    MOCK_OP_LOG(3) << "count(" << num_calls_get_workorders_ << ") "
-                   << "return(" << (num_calls_get_workorders_ == max_getworkorder_iters_) << ")";
-    return num_calls_get_workorders_ == max_getworkorder_iters_;
-  }
-
-  bool getAllWorkOrderProtos(WorkOrderProtosContainer *container) override {
-    return true;
-  }
-
-  void feedInputBlock(const block_id input_block_id,
-                      const relation_id input_relation_id) override {
-    ++num_calls_feedblock_;
-    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_ << ")";
-  }
-
-  QueryContext::insert_destination_id getInsertDestinationID() const override {
-    return insert_destination_index_;
-  }
-
-  const relation_id getOutputRelationID() const override {
-    return output_relation_id_;
-  }
-
- private:
-  const bool produce_workorders_;
-  const bool has_streaming_input_;
-  const int max_workorders_;
-  const int max_getworkorder_iters_;
-
-  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;
-
-  relation_id output_relation_id_ = -1;
-
-#undef MOCK_OP_LOG
-
-  DISALLOW_COPY_AND_ASSIGN(MockOperator);
-};
-
-
-class ForemanTest : public ::testing::Test {
- protected:
-  // Class ForemanTest is the friend of class Foreman. Each TEST_F behaves
-  // as a separate class, so we can't access Foreman's private members in
-  // TEST_F.
-  virtual void SetUp() {
-    db_.reset(new CatalogDatabase(nullptr /* catalog */, "database"));
-    storage_manager_.reset(new StorageManager("./"));
-
-    query_plan_.reset(new QueryPlan());
-
-    bus_.Initialize();
-
-    foreman_.reset(new Foreman(&bus_, db_.get(), storage_manager_.get()));
-
-    // This thread acts both as Foreman as well as Worker. Foreman connects to
-    // the bus in its constructor.
-    worker_client_id_ = bus_.Connect();
-
-    // Register as sender and receiver for relevant types of messages.
-    bus_.RegisterClientAsSender(worker_client_id_, kWorkOrderCompleteMessage);
-    bus_.RegisterClientAsSender(worker_client_id_, kRebuildWorkOrderCompleteMessage);
-    bus_.RegisterClientAsSender(worker_client_id_, kDataPipelineMessage);
-    bus_.RegisterClientAsReceiver(worker_client_id_, kWorkOrderMessage);
-    bus_.RegisterClientAsReceiver(worker_client_id_, kRebuildWorkOrderMessage);
-    bus_.RegisterClientAsReceiver(worker_client_id_, kPoisonMessage);
-
-    std::vector<client_id> worker_client_ids;
-    worker_client_ids.push_back(worker_client_id_);
-
-    std::vector<numa_node_id> numa_nodes;
-    numa_nodes.push_back(static_cast<numa_node_id>(-1));
-
-    workers_.reset(new WorkerDirectory(1, worker_client_ids, numa_nodes));
-    foreman_->setWorkerDirectory(workers_.get());
-  }
-
-  inline const int getNumWorkOrdersInExecution(const QueryPlan::DAGNodeIndex index) const {
-    return foreman_->query_exec_state_->getNumQueuedWorkOrders(index);
-  }
-
-  inline const int getNumOperatorsFinished() const {
-    return foreman_->query_exec_state_->getNumOperatorsFinished();
-  }
-
-  inline bool getOperatorFinishedStatus(const QueryPlan::DAGNodeIndex index) const {
-    return foreman_->query_exec_state_->hasExecutionFinished(index);
-  }
-
-  inline bool popWorkOrderIfAvailable(MockWorkOrder **workorder) {
-    AnnotatedMessage msg;
-    if (bus_.ReceiveIfAvailable(worker_client_id_, &msg)) {
-      WorkerMessage message(*static_cast<const WorkerMessage*>(msg.tagged_message.message()));
-      *workorder = static_cast<MockWorkOrder*>(message.getWorkOrder());
-      return true;
-    }
-    return false;
-  }
-
-  inline bool popRebuildWorkOrderIfAvailable(MockWorkOrder **workorder) {
-    return popWorkOrderIfAvailable(workorder);
-  }
-
-  inline bool placeDataPipelineMessage(const QueryPlan::DAGNodeIndex source_operator_index) {
-    VLOG(3) << "Place DataPipeline message for Op[" << source_operator_index << "]";
-    foreman_->processDataPipelineMessage(source_operator_index, 0 /* block_id */, 0 /* relation_id */);
-    return foreman_->query_exec_state_->hasQueryExecutionFinished();
-  }
-
-  inline bool placeWorkOrderCompleteMessage(const QueryPlan::DAGNodeIndex index) {
-    VLOG(3) << "Place WorkOrderComplete message for Op[" << index << "]";
-    foreman_->processWorkOrderCompleteMessage(index, 0 /* worker id */);
-    return foreman_->query_exec_state_->hasQueryExecutionFinished();
-  }
-
-  inline bool placeRebuildWorkOrderCompleteMessage(const QueryPlan::DAGNodeIndex index) {
-    VLOG(3) << "Place RebuildWorkOrderComplete message for Op[" << index << "]";
-    foreman_->processRebuildWorkOrderCompleteMessage(index, 0 /* worker id */);
-    return foreman_->query_exec_state_->hasQueryExecutionFinished();
-  }
-
-  inline bool placeOutputBlockMessage(const QueryPlan::DAGNodeIndex index) {
-    VLOG(3) << "Place OutputBlock message for Op[" << index << "]";
-    foreman_->processDataPipelineMessage(index,
-                                         BlockIdUtil::GetBlockId(1 /* domain */, 1),
-                                         0 /* relation_id */);
-    return foreman_->query_exec_state_->hasQueryExecutionFinished();
-  }
-
-  inline bool startForeman() {
-    foreman_->initialize();
-    return foreman_->query_exec_state_->hasQueryExecutionFinished();
-  }
-
-  inline int getWorkerInputQueueSize() {
-    return bus_.CountQueuedMessagesForClient(worker_client_id_);
-  }
-
-  unique_ptr<CatalogDatabase> db_;
-  unique_ptr<StorageManager> storage_manager_;
-
-  unique_ptr<QueryPlan> query_plan_;
-
-  unique_ptr<Foreman> foreman_;
-  MessageBusImpl bus_;
-
-  client_id worker_client_id_;
-
-  unique_ptr<WorkerDirectory> workers_;
-};
-
-TEST_F(ForemanTest, SingleNodeDAGNoWorkOrdersTest) {
-  // This test creates a DAG of a single node. No workorders are generated.
-  query_plan_->addRelationalOperator(new MockOperator(false, false));
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-
-  const MockOperator &op = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(0));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  // Foreman exits after initialize, since no workorders are generated.
-  EXPECT_TRUE(startForeman());
-
-  // op doesn't have any dependencies.
-  EXPECT_TRUE(op.getBlockingDependenciesMet());
-
-  // No workorder is generated. No response is received.
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  // 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(ForemanTest, SingleNodeDAGStaticWorkOrdersTest) {
-  // This test creates a DAG of a single node. Static workorders are generated.
-  const QueryPlan::DAGNodeIndex id = query_plan_->addRelationalOperator(new MockOperator(true, false, 1));
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-
-  const MockOperator &op = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-  EXPECT_FALSE(startForeman());
-
-  // op doesn't have any dependencies.
-  EXPECT_TRUE(op.getBlockingDependenciesMet());
-
-  // 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, getWorkerInputQueueSize());
-  EXPECT_EQ(1, op.getNumWorkOrders());
-
-  // Worker receives a WorkOrder.
-  MockWorkOrder *work_order;
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  EXPECT_EQ(id, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(id));
-  EXPECT_EQ(0, getNumOperatorsFinished());
-
-  // Send a message to Foreman upon workorder completion.
-  // Last event processed by Foreman.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(id));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id));
-  EXPECT_EQ(1, getNumOperatorsFinished());
-  EXPECT_TRUE(getOperatorFinishedStatus(id));
-}
-
-TEST_F(ForemanTest, SingleNodeDAGDynamicWorkOrdersTest) {
-  // This test creates a DAG of a single node. WorkOrders are generated
-  // dynamically as pending work orders complete execution, i.e.,
-  // getAllWorkOrders() is called multiple times.  getAllWorkOrders() will be
-  // called 5 times and 3 work orders will be returned, i.e., 1st 3 calls to
-  // getAllWorkOrders() insert 1 WorkOrder and return false, and the next will insert no
-  // WorkOrder and return true.
-
-  // TODO(shoban): This test can not be more robust than this because of fixed
-  // scaffolding of mocking. If we use gMock, we can do much better.
-  const QueryPlan::DAGNodeIndex id = query_plan_->addRelationalOperator(new MockOperator(true, false, 4, 3));
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-
-  const MockOperator &op = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-  EXPECT_FALSE(startForeman());
-
-  // op doesn't have any dependencies.
-  EXPECT_TRUE(op.getBlockingDependenciesMet());
-
-  for (int i = 0; i < 3; i++) {
-    // We expect one call for op's getAllWorkOrders().
-    EXPECT_EQ(i + 1, op.getNumCalls(MockOperator::kGetAllWorkOrders));
-
-    // One workorder is generated.
-    EXPECT_EQ(1, getWorkerInputQueueSize());
-    EXPECT_EQ(i + 1, op.getNumWorkOrders());
-
-    // Worker receives a WorkOrder.
-    MockWorkOrder *work_order;
-    ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-    EXPECT_EQ(id, work_order->getOpIndex());
-
-    work_order->execute();
-    delete work_order;
-
-    EXPECT_EQ(1, getNumWorkOrdersInExecution(id));
-    EXPECT_EQ(0, getNumOperatorsFinished());
-
-    if (i < 2) {
-      // Send a message to Foreman upon workorder completion.
-      EXPECT_FALSE(placeWorkOrderCompleteMessage(id));
-    } else {
-      // Send a message to Foreman upon workorder completion.
-      // Last event.
-      EXPECT_TRUE(placeWorkOrderCompleteMessage(id));
-    }
-  }
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id));
-
-  EXPECT_EQ(1, getNumOperatorsFinished());
-  EXPECT_TRUE(getOperatorFinishedStatus(id));
-
-  // 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(ForemanTest, TwoNodesDAGBlockingLinkTest) {
-  // We use two nodes in the DAG with a blocking link between them.
-  // There is no streaming of data involved in this test.
-  const QueryPlan::DAGNodeIndex id1 = query_plan_->addRelationalOperator(new MockOperator(true, false));
-  const QueryPlan::DAGNodeIndex id2 = query_plan_->addRelationalOperator(new MockOperator(true, false));
-
-  // Create a blocking link.
-  query_plan_->addDirectDependency(id2, id1, true);
-
-  static_cast<MockOperator*>(query_plan_->getQueryPlanDAGMutable()->getNodePayloadMutable(id1))
-      ->setOutputRelationID(0xdead);
-
-  const MockOperator &op1 = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id1));
-  const MockOperator &op2 = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id2));
-
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-
-  // Make sure queues are empty initially.
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  EXPECT_FALSE(startForeman());
-
-  // op1 doesn't have any dependencies
-  EXPECT_TRUE(op1.getBlockingDependenciesMet());
-
-  // 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, getWorkerInputQueueSize());
-  EXPECT_EQ(1, op1.getNumWorkOrders());
-  EXPECT_EQ(0, op2.getNumWorkOrders());
-
-  // Worker receives a WorkOrder.
-  MockWorkOrder *work_order;
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // This workorder's source should be op1.
-  EXPECT_EQ(id1, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-  // Foreman hasn't yet got workorder completion response for the workorder.
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(id1));
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id2));
-  EXPECT_EQ(0, getNumOperatorsFinished());
-
-  // Send a message to Foreman upon workorder (generated by op1) completion.
-  EXPECT_FALSE(placeWorkOrderCompleteMessage(id1));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id1));
-  // op1 is over now, op2 still to go.
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kDoneFeedingInputBlocks));
-  EXPECT_EQ(1, getNumOperatorsFinished());
-
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kDoneFeedingInputBlocks));
-  EXPECT_FALSE(getOperatorFinishedStatus(id2));
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(id2));
-
-  // op1 is op2's blocking dependency.
-  EXPECT_TRUE(op2.getBlockingDependenciesMet());
-
-  EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
-  // op2 should get first call of getAllWorkOrders() when op1 is over.
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-
-  EXPECT_EQ(1, op2.getNumWorkOrders());
-
-  // Send a message to Foreman upon workorder (generated by op2) completion.
-
-  // Note that the worker hasn't yet popped the workorder. Usually this won't
-  // happen as workers pop workorders first, execute and then send the response.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(id2));
-
-  // WorkOrder yet to be popped by the worker.
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id1));
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id2));
-
-  EXPECT_EQ(2, getNumOperatorsFinished());
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-  EXPECT_TRUE(getOperatorFinishedStatus(id2));
-
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // The workorder should have come from op2.
-  EXPECT_EQ(id2, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  // Expect no additional calls to getAllWorkOrders.
-  EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-}
-
-TEST_F(ForemanTest, TwoNodesDAGPipeLinkTest) {
-  // We use two nodes in the DAG with a non-blocking link between them.
-  // We stream output of op1 to op2. Sequeuce of events is as follows:
-  // 1. op1 creates a workorder.
-  // 2. We send a "block full" (from op1) to Foreman.
-  // 3. op2 creates a workorder because of step 2.
-  const QueryPlan::DAGNodeIndex id1 = query_plan_->addRelationalOperator(new MockOperator(true, false, 1));
-  const QueryPlan::DAGNodeIndex id2 = query_plan_->addRelationalOperator(new MockOperator(true, true, 3));
-
-  // Create a non-blocking link.
-  query_plan_->addDirectDependency(id2, id1, false);
-
-  static_cast<MockOperator*>(query_plan_->getQueryPlanDAGMutable()->getNodePayloadMutable(id1))
-      ->setOutputRelationID(0xdead);
-
-  const MockOperator &op1 = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id1));
-  const MockOperator &op2 = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id2));
-
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-
-  // Make sure queues are empty initially.
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  startForeman();
-
-  // As none of the operators have a blocking link, blocking dependencies should
-  // be met.
-  EXPECT_TRUE(op1.getBlockingDependenciesMet());
-  EXPECT_TRUE(op2.getBlockingDependenciesMet());
-
-  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));
-
-  // There should be one workorder sent to the worker so far.
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  // Worker receives a WorkOrder.
-  MockWorkOrder *work_order;
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // This workorder's source be op1.
-  EXPECT_EQ(id1, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  // Send a message to Foreman upon block getting full (output of op1).
-  EXPECT_FALSE(placeOutputBlockMessage(id1));
-
-  // op1 is not finished yet because the response of workorder completion hasn't
-  // been received yet by the Foreman.
-  EXPECT_FALSE(getOperatorFinishedStatus(id1));
-  EXPECT_FALSE(getOperatorFinishedStatus(id2));
-
-  // 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));
-  EXPECT_EQ(1, op2.getNumWorkOrders());
-
-  // Place a message of a workorder completion of op1 on Foreman's input queue.
-  EXPECT_FALSE(placeWorkOrderCompleteMessage(id1));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id1));
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kDoneFeedingInputBlocks));
-
-  // An additional call to op2's getAllWorkOrders because of completion of op1.
-  EXPECT_EQ(3, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(2, op2.getNumWorkOrders());
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  // Pop a workorder from Foreman's output queue.
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // The workorder should have been generated by op2.
-  EXPECT_EQ(id2, work_order->getOpIndex());
-  work_order->execute();
-  delete work_order;
-
-  // Place a message of a workorder completion of op2 on Foreman's input queue.
-  EXPECT_FALSE(placeWorkOrderCompleteMessage(id2));
-
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(id2));
-  EXPECT_FALSE(getOperatorFinishedStatus(id2));
-
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  // Pop a workorder from Foreman's output queue.
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // The workorder should have been generated by op2.
-  EXPECT_EQ(id2, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  // Send a message to Foreman upon workorder (generated by op2) completion.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(id2));
-
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kDoneFeedingInputBlocks));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id2));
-  EXPECT_TRUE(getOperatorFinishedStatus(id2));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-}
-
-TEST_F(ForemanTest, TwoNodesDAGPartiallyFilledBlocksTest) {
-  // In this test, we create a 2-node DAG with a non-blocking link between them.
-  // There is no streaming of data from op1 to op2 during the execution of op1.
-  // op1 produces a partially filled block at the end of its execution which is
-  // rebuilt and then fed to op2.
-  const QueryPlan::DAGNodeIndex id1 = query_plan_->addRelationalOperator(new MockOperator(true, false, 1));
-  const QueryPlan::DAGNodeIndex id2 = query_plan_->addRelationalOperator(new MockOperator(true, true, 3, 1));
-
-  // Create a non-blocking link.
-  query_plan_->addDirectDependency(id2, id1, false);
-
-  // Create a relation, owned by db_.
-  CatalogRelation *relation = new CatalogRelation(nullptr /* catalog_database */, "test_relation");
-  const relation_id output_relation_id = db_->addRelation(relation);
-
-  // Setup the InsertDestination proto in the query context proto.
-  serialization::QueryContext query_context_proto;
-
-  const QueryContext::insert_destination_id insert_destination_index =
-      query_context_proto.insert_destinations_size();
-  serialization::InsertDestination *insert_destination_proto = query_context_proto.add_insert_destinations();
-
-  insert_destination_proto->set_insert_destination_type(serialization::InsertDestinationType::BLOCK_POOL);
-  insert_destination_proto->set_relation_id(output_relation_id);
-  insert_destination_proto->set_relational_op_index(id1);
-
-  MockOperator *op1_mutable =
-      static_cast<MockOperator*>(query_plan_->getQueryPlanDAGMutable()->getNodePayloadMutable(id1));
-  op1_mutable->setInsertDestinationID(insert_destination_index);
-  op1_mutable->setOutputRelationID(output_relation_id);
-
-  const MockOperator &op1 = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id1));
-  const MockOperator &op2 = static_cast<const MockOperator&>(query_plan_->getQueryPlanDAG().getNodePayload(id2));
-
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-  foreman_->reconstructQueryContextFromProto(query_context_proto);
-
-  // NOTE(zuyu): An operator generally has no ideas about partially filled
-  // blocks, but InsertDestination in QueryContext does.
-  // Mock to add partially filled blocks in the InsertDestination.
-  InsertDestination *insert_destination =
-      foreman_->query_context_->getInsertDestination(insert_destination_index);
-  DCHECK(insert_destination != nullptr);
-  MutableBlockReference block_ref;
-  static_cast<BlockPoolInsertDestination*>(insert_destination)->available_block_refs_.push_back(move(block_ref));
-
-  // Make sure queues are empty initially.
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  startForeman();
-
-  // There's no blocking dependency in the DAG.
-  EXPECT_TRUE(op1.getBlockingDependenciesMet());
-  EXPECT_TRUE(op2.getBlockingDependenciesMet());
-
-  EXPECT_EQ(1, op1.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(1, op1.getNumWorkOrders());
-
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(0, op2.getNumWorkOrders());
-
-  // Worker receives a WorkOrder.
-  MockWorkOrder *work_order;
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // The workorder should have been generated by op1.
-  EXPECT_EQ(id1, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  // Send a message to Foreman upon workorder (generated by op1) completion.
-  EXPECT_FALSE(placeWorkOrderCompleteMessage(id1));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id1));
-
-  // op1 generates a rebuild workorder. The block is rebuilt and streamed
-  // to Foreman.
-  EXPECT_FALSE(placeDataPipelineMessage(id1));
-
-  // Based on the streamed input, op2's getAllWorkOrders should produce a
-  // workorder.
-  EXPECT_EQ(3, op2.getNumCalls(MockOperator::kGetAllWorkOrders));
-  EXPECT_EQ(1, op2.getNumWorkOrders());
-
-  // Worker receives a rebuild WorkOrder.
-  MockWorkOrder *rebuild_op1;
-  ASSERT_TRUE(popRebuildWorkOrderIfAvailable(&rebuild_op1));
-  // We skip the check for relation ID of the rebuild WorkOrder, as the partially
-  // filled block reference is a mock reference with garbage contents.
-  delete rebuild_op1;
-
-  EXPECT_FALSE(placeRebuildWorkOrderCompleteMessage(id1));
-
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-  EXPECT_EQ(1, op2.getNumCalls(MockOperator::kDoneFeedingInputBlocks));
-  EXPECT_FALSE(getOperatorFinishedStatus(id2));
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(id2));
-
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  // Worker receives a WorkOrder.
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // The workorder should have been generated by op2.
-  EXPECT_EQ(id2, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  // Send a message to Foreman upon workorder (generated by op2) completion.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(id2));
-
-  EXPECT_EQ(0, getNumWorkOrdersInExecution(id2));
-
-  EXPECT_TRUE(getOperatorFinishedStatus(id2));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-}
-
-TEST_F(ForemanTest, MultipleNodesNoOutputTest) {
-  // When an operator produces workorders but no output, the Foreman should
-  // check the dependents of this operator to make progress.
-  const QueryPlan::DAGNodeIndex kNumNodes = 5;
-  std::vector<QueryPlan::DAGNodeIndex> ids;
-  ids.reserve(kNumNodes);
-
-  for (QueryPlan::DAGNodeIndex i = 0; i < kNumNodes; ++i) {
-    if (i == 0) {
-      ids[i] = query_plan_->addRelationalOperator(new MockOperator(true, false));
-    } else {
-      ids[i] = query_plan_->addRelationalOperator(new MockOperator(true, true));
-    }
-    VLOG(3) << ids[i];
-  }
-
-  /**
-   * The DAG looks like this:
-   *
-   * op1 -> op2 -> op3 -> op4 -> op5
-   *
-   **/
-  for (QueryPlan::DAGNodeIndex i = 0; i < kNumNodes - 1; ++i) {
-    query_plan_->addDirectDependency(ids[i + 1], ids[i], false);
-    static_cast<MockOperator*>(query_plan_->getQueryPlanDAGMutable()->getNodePayloadMutable(ids[i]))
-        ->setOutputRelationID(0xdead);
-  }
-
-  std::vector<const MockOperator*> operators;
-  for (QueryPlan::DAGNodeIndex i = 0; i < kNumNodes; ++i) {
-    operators.push_back(static_cast<const MockOperator*>(&query_plan_->getQueryPlanDAG().getNodePayload(ids[i])));
-  }
-
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-
-  // Make sure queues are empty initially.
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  startForeman();
-
-  // operators[0] should have produced a workorder by now.
-  EXPECT_EQ(1, operators[0]->getNumWorkOrders());
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  EXPECT_EQ(1, getNumWorkOrdersInExecution(ids[0]));
-  EXPECT_FALSE(getOperatorFinishedStatus(ids[0]));
-
-  for (QueryPlan::DAGNodeIndex i = 0; i < kNumNodes; ++i) {
-    EXPECT_EQ(1, operators[ids[i]]->getNumCalls(MockOperator::kGetAllWorkOrders));
-  }
-
-  // Worker receives a WorkOrder.
-  MockWorkOrder *work_order;
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // The workorder should have been generated by operators[0].
-  EXPECT_EQ(ids[0], work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  // Send a message to Foreman upon workorder (generated by operators[0])
-  // completion.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(ids[0]));
-
-  for (QueryPlan::DAGNodeIndex i = 0; i < kNumNodes; ++i) {
-    EXPECT_EQ(0, getNumWorkOrdersInExecution(ids[i]));
-    EXPECT_TRUE(getOperatorFinishedStatus(ids[i]));
-    if (i < kNumNodes - 1) {
-      EXPECT_EQ(1, operators[i + 1]->getNumCalls(MockOperator::kDoneFeedingInputBlocks));
-    }
-  }
-}
-
-TEST_F(ForemanTest, OutOfOrderWorkOrderCompletionTest) {
-  // Consider two operators, both generate one workorder each. The dependent's
-  // workorder finishes before dependency's workorder.
-  const QueryPlan::DAGNodeIndex id1 = query_plan_->addRelationalOperator(new MockOperator(true, false, 1));
-  const QueryPlan::DAGNodeIndex id2 = query_plan_->addRelationalOperator(new MockOperator(true, true, 2, 1));
-
-  // Create a non-blocking link.
-  query_plan_->addDirectDependency(id2, id1, false);
-
-  foreman_->setQueryPlan(query_plan_->getQueryPlanDAGMutable());
-  // There should be two workorders on Worker's private queue, for this test.
-  foreman_->setMaxMessagesPerWorker(2);
-
-  // Make sure queues are empty initially.
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  startForeman();
-
-  // Expect one workorder produced by op1.
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  // Pop a workorder from Foreman's output queue.
-  MockWorkOrder *work_order;
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // This workorder's source be op1.
-  EXPECT_EQ(id1, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  // Send a message to Foreman upon a block (output of op1) getting full.
-  EXPECT_FALSE(placeOutputBlockMessage(id1));
-
-  // op1 is not finished yet because the response of workorder completion hasn't
-  // been received yet.
-  EXPECT_FALSE(getOperatorFinishedStatus(id1));
-  EXPECT_FALSE(getOperatorFinishedStatus(id2));
-
-  // Expect one workorder produced by op2.
-  EXPECT_EQ(1, getWorkerInputQueueSize());
-
-  // Worker receives a WorkOrder.
-  ASSERT_TRUE(popWorkOrderIfAvailable(&work_order));
-  // This workorder's source should be op2.
-  EXPECT_EQ(id2, work_order->getOpIndex());
-
-  work_order->execute();
-  delete work_order;
-
-  // As mentioned earlier, op2 finishes before op1.
-  EXPECT_FALSE(placeWorkOrderCompleteMessage(id2));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-
-  // op1's workorder execution is over.
-  EXPECT_TRUE(placeWorkOrderCompleteMessage(id1));
-
-  EXPECT_TRUE(getOperatorFinishedStatus(id1));
-  EXPECT_TRUE(getOperatorFinishedStatus(id2));
-
-  EXPECT_EQ(0, getWorkerInputQueueSize());
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/tests/QueryManager_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManager_unittest.cpp b/query_execution/tests/QueryManager_unittest.cpp
index 9ba5978..4f98748 100644
--- a/query_execution/tests/QueryManager_unittest.cpp
+++ b/query_execution/tests/QueryManager_unittest.cpp
@@ -228,8 +228,9 @@ class QueryManagerTest : public ::testing::Test {
     db_.reset(new CatalogDatabase(nullptr /* catalog */, "database"));
     storage_manager_.reset(new StorageManager("./"));
     bus_.Initialize();
-    query_handle_.reset(new QueryHandle(0));
+    query_handle_.reset(new QueryHandle(0));  // dummy query ID.
     query_plan_ = query_handle_->getQueryPlanMutable();
+    query_handle_->getQueryContextProtoMutable()->set_query_id(query_handle_->query_id());
   }
 
   inline void constructQueryManager() {
@@ -256,6 +257,7 @@ class QueryManagerTest : public ::testing::Test {
 
     proto.set_block_id(0);  // dummy block ID
     proto.set_relation_id(0);  // dummy relation ID.
+    proto.set_query_id(0);  // dummy query ID.
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const std::size_t proto_length = proto.ByteSize();
@@ -276,6 +278,7 @@ class QueryManagerTest : public ::testing::Test {
     serialization::WorkOrderCompletionMessage proto;
     proto.set_operator_index(index);
     proto.set_worker_thread_index(1);  // dummy worker ID.
+    proto.set_query_id(0);  // dummy query ID.
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const size_t proto_length = proto.ByteSize();
@@ -297,6 +300,7 @@ class QueryManagerTest : public ::testing::Test {
     serialization::WorkOrderCompletionMessage proto;
     proto.set_operator_index(index);
     proto.set_worker_thread_index(1);  // dummy worker thread ID.
+    proto.set_query_id(0);  // dummy query ID.
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const size_t proto_length = proto.ByteSize();
@@ -320,6 +324,7 @@ class QueryManagerTest : public ::testing::Test {
 
     proto.set_block_id(0);  // dummy block ID
     proto.set_relation_id(0);  // dummy relation ID.
+    proto.set_query_id(0);  // dummy query ID.
 
     // NOTE(zuyu): Using the heap memory to serialize proto as a c-like string.
     const std::size_t proto_length = proto.ByteSize();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_execution/tests/WorkOrdersContainer_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/WorkOrdersContainer_unittest.cpp b/query_execution/tests/WorkOrdersContainer_unittest.cpp
index cf133c4..cb583ab 100644
--- a/query_execution/tests/WorkOrdersContainer_unittest.cpp
+++ b/query_execution/tests/WorkOrdersContainer_unittest.cpp
@@ -72,6 +72,7 @@ TEST(WorkOrdersContainerTest, ZeroNUMANodesAddWorkOrderTest) {
   // they get inserted and retrieved correctly.
   std::vector<int> numa_node_ids;
   // A container for one operator and no NUMA nodes.
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(1, 0);
 
   EXPECT_EQ(0u, w.getNumNormalWorkOrders(0));
@@ -104,11 +105,15 @@ TEST(WorkOrdersContainerTest, ZeroNUMANodesAddWorkOrderTest) {
   ASSERT_TRUE(returned_work_order != nullptr);
   EXPECT_EQ(work_order.getID(), static_cast<MockNUMAWorkOrder*>(returned_work_order)->getID());
 
+  EXPECT_EQ(query_id, returned_work_order->getQueryID());
+
   WorkOrder *returned_rebuild_work_order = w.getRebuildWorkOrder(0);
   ASSERT_TRUE(returned_rebuild_work_order != nullptr);
   EXPECT_EQ(work_order1.getID(),
             static_cast<MockNUMAWorkOrder *>(returned_rebuild_work_order)->getID());
 
+  EXPECT_EQ(query_id, returned_rebuild_work_order->getQueryID());
+
   // Container should be empty now.
   EXPECT_EQ(0u, w.getNumNormalWorkOrders(0));
   EXPECT_EQ(0u, w.getNumRebuildWorkOrders(0));
@@ -123,6 +128,7 @@ TEST(WorkOrdersContainerTest, ZeroNUMANodesMultipleWorkOrdersTest) {
   // if they get inserted and retrieved correctly and the order of retrieval.
   // A container for one operator and no NUMA nodes.
   std::vector<int> numa_node_ids;
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(1, 0);
 
   EXPECT_EQ(0u, w.getNumNormalWorkOrders(0));
@@ -164,6 +170,8 @@ TEST(WorkOrdersContainerTest, ZeroNUMANodesMultipleWorkOrdersTest) {
     ASSERT_TRUE(returned_work_order != nullptr);
     EXPECT_EQ(static_cast<int>(kNumWorkOrders + i),
               static_cast<MockNUMAWorkOrder *>(returned_rebuild_work_order)->getID());
+    EXPECT_EQ(query_id, returned_work_order->getQueryID());
+    EXPECT_EQ(query_id, returned_rebuild_work_order->getQueryID());
   }
 
   // Container should be empty now.
@@ -190,6 +198,7 @@ TEST(WorkOrdersContainerTest, MultipleNUMANodesTest) {
   const std::size_t kNUMANodesUsed = numa_node_ids.size();
 
   // A container for one operator and kNUMANodes.
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(1, kNUMANodes);
 
   for (std::size_t i = 0; i < kNUMANodesUsed; ++i) {
@@ -246,6 +255,9 @@ TEST(WorkOrdersContainerTest, MultipleNUMANodesTest) {
     ASSERT_TRUE(returned_rebuild_work_order != nullptr);
     EXPECT_EQ(rebuild_workorders[i].getID(),
               static_cast<MockNUMAWorkOrder *>(returned_rebuild_work_order)->getID());
+
+    EXPECT_EQ(query_id, returned_work_order->getQueryID());
+    EXPECT_EQ(query_id, returned_rebuild_work_order->getQueryID());
   }
 
   // No workorder should be left for this operator on any NUMA node.
@@ -291,6 +303,7 @@ TEST(WorkOrdersContainerTest, AllTypesWorkOrdersTest) {
   const std::size_t kNUMANodesUsed = numa_nodes.size();
 
   // Create the container.
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(1, kNUMANodes);
 
   w.addNormalWorkOrder(&multiple_numa_work_order, 0);
@@ -331,6 +344,7 @@ TEST(WorkOrdersContainerTest, AllTypesWorkOrdersTest) {
       w.getNormalWorkOrderForNUMANode(0, numa_nodes[0]));
   ASSERT_TRUE(observed_work_order != nullptr);
 
+  EXPECT_EQ(query_id, observed_work_order->getQueryID());
   EXPECT_EQ(one_numa_work_order.getPreferredNUMANodes().front(),
             observed_work_order->getPreferredNUMANodes().front());
   EXPECT_EQ(one_numa_work_order.getID(), observed_work_order->getID());
@@ -348,6 +362,7 @@ TEST(WorkOrdersContainerTest, AllTypesWorkOrdersTest) {
   EXPECT_EQ(no_numa_work_order.getID(),
             static_cast<MockNUMAWorkOrder *>(observed_non_numa_work_order)->getID());
 
+  EXPECT_EQ(query_id, observed_non_numa_work_order->getQueryID());
   EXPECT_EQ(1u, w.getNumNormalWorkOrdersForNUMANode(0, numa_nodes[0]));
   EXPECT_EQ(1u, w.getNumNormalWorkOrdersForNUMANode(0, numa_nodes[1]));
 
@@ -361,6 +376,7 @@ TEST(WorkOrdersContainerTest, AllTypesWorkOrdersTest) {
   ASSERT_TRUE(observed_work_order_multiple_numa_nodes != nullptr);
   EXPECT_EQ(multiple_numa_work_order.getID(), observed_work_order_multiple_numa_nodes->getID());
 
+  EXPECT_EQ(query_id, observed_work_order_multiple_numa_nodes->getQueryID());
   std::vector<int> observed_numa_nodes(
       observed_work_order_multiple_numa_nodes->getPreferredNUMANodes());
   // Look up the expected numa nodes in the observed_numa_nodes vector.
@@ -427,6 +443,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsNormalWorkOrderTest) {
   const std::size_t kNUMANodes = numa_node_ids.size();
 
   // Create the container.
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(kNumOperators, kNUMANodes);
 
   std::vector<std::size_t> operator_ids;
@@ -538,6 +555,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsNormalWorkOrderTest) {
             curr_operator_id, single_numa_node_id));
     ASSERT_TRUE(observed_work_order_single_numa != nullptr);
 
+    EXPECT_EQ(query_id, observed_work_order_single_numa->getQueryID());
     // Verify if the workorder ID is correct.
     const int expected_workorder_id_single_numa =
         normal_workorders_one_numa_ids[curr_operator_id];
@@ -550,6 +568,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsNormalWorkOrderTest) {
             curr_operator_id, multiple_numa_node_id));
     ASSERT_TRUE(observed_work_order_multiple_numa != nullptr);
 
+    EXPECT_EQ(query_id, observed_work_order_multiple_numa->getQueryID());
     // Verify if the workorder ID is correct.
     const int expected_workorder_id_multiple_numa =
         normal_workorders_multiple_numa_ids[curr_operator_id];
@@ -562,6 +581,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsNormalWorkOrderTest) {
         static_cast<MockNUMAWorkOrder *>(w.getNormalWorkOrder(curr_operator_id));
     ASSERT_TRUE(observed_work_order_no_numa != nullptr);
 
+    EXPECT_EQ(query_id, observed_work_order_no_numa->getQueryID());
     // Verify if the workorder ID is correct.
     const int expected_workorder_id_no_numa =
         normal_workorders_no_numa_ids[curr_operator_id];
@@ -620,6 +640,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsRebuildWorkOrderTest) {
   const std::size_t kNUMANodes = numa_node_ids.size();
 
   // Create the container.
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(kNumOperators, kNUMANodes);
 
   std::vector<std::size_t> operator_ids;
@@ -732,6 +753,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsRebuildWorkOrderTest) {
             curr_operator_id, single_numa_node_id));
     ASSERT_TRUE(observed_work_order_single_numa != nullptr);
 
+    EXPECT_EQ(query_id, observed_work_order_single_numa->getQueryID());
     // Verify if the workorder ID is correct.
     const int expected_workorder_id_single_numa =
         rebuild_workorders_one_numa_ids[curr_operator_id];
@@ -744,6 +766,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsRebuildWorkOrderTest) {
             curr_operator_id, multiple_numa_node_id));
     ASSERT_TRUE(observed_work_order_multiple_numa != nullptr);
 
+    EXPECT_EQ(query_id, observed_work_order_multiple_numa->getQueryID());
     // Verify if the workorder ID is correct.
     const int expected_workorder_id_multiple_numa =
         rebuild_workorders_multiple_numa_ids[curr_operator_id];
@@ -755,6 +778,7 @@ TEST(WorkOrdersContainerTest, MultipleOperatorsRebuildWorkOrderTest) {
     MockNUMAWorkOrder *observed_work_order_no_numa =
         static_cast<MockNUMAWorkOrder *>(w.getRebuildWorkOrder(curr_operator_id));
 
+    EXPECT_EQ(query_id, observed_work_order_no_numa->getQueryID());
     // Verify if the workorder ID is correct.
     const int expected_workorder_id_no_numa =
         rebuild_workorders_no_numa_ids[curr_operator_id];
@@ -772,6 +796,7 @@ TEST(WorkOrdersContainerTest, RetrievalOrderTest) {
   numa_node_ids.push_back(0);
   const std::size_t kNumWorkOrdersPerType = 100;
 
+  const std::size_t query_id = 0;
   WorkOrdersContainer w(1, 2);
 
   std::vector<int> single_numa_node_workorder_ids;
@@ -820,6 +845,7 @@ TEST(WorkOrdersContainerTest, RetrievalOrderTest) {
     MockNUMAWorkOrder *observed_work_order = static_cast<MockNUMAWorkOrder *>(
         w.getNormalWorkOrder(0, prefer_single_NUMA_node));
     ASSERT_TRUE(observed_work_order != nullptr);
+    EXPECT_EQ(query_id, observed_work_order->getQueryID());
     if (prefer_single_NUMA_node) {
       EXPECT_EQ(*single_numa_it, observed_work_order->getID());
       EXPECT_EQ(1u, observed_work_order->getPreferredNUMANodes().size());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index 0630bca..c7fd018 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -105,6 +105,7 @@ class ExecutionGenerator {
         execution_plan_(DCHECK_NOTNULL(query_handle->getQueryPlanMutable())),
         query_context_proto_(DCHECK_NOTNULL(query_handle->getQueryContextProtoMutable())),
         execution_heuristics_(new ExecutionHeuristics()) {
+    query_context_proto_->set_query_id(query_handle_->query_id());
 #ifdef QUICKSTEP_DISTRIBUTED
     catalog_database_cache_proto_ = DCHECK_NOTNULL(query_handle->getCatalogDatabaseCacheProtoMutable());
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_optimizer/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/CMakeLists.txt b/query_optimizer/tests/CMakeLists.txt
index 6ef2a03..5b58f75 100644
--- a/query_optimizer/tests/CMakeLists.txt
+++ b/query_optimizer/tests/CMakeLists.txt
@@ -116,9 +116,11 @@ target_link_libraries(quickstep_queryoptimizer_tests_ExecutionGeneratorTest
                       quickstep_cli_PrintToScreen
                       quickstep_parser_ParseStatement
                       quickstep_parser_SqlParserWrapper
+                      quickstep_queryexecution_AdmitRequestMessage
                       quickstep_queryexecution_Foreman
                       quickstep_queryexecution_QueryContext
                       quickstep_queryexecution_QueryExecutionTypedefs
+                      quickstep_queryexecution_QueryExecutionUtil
                       quickstep_queryexecution_Worker
                       quickstep_queryexecution_WorkerDirectory
                       quickstep_queryexecution_WorkerMessage

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8230b124/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
index 56b53ba..ea871d0 100644
--- a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
+++ b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
@@ -24,7 +24,9 @@
 #include "cli/DropRelation.hpp"
 #include "cli/PrintToScreen.hpp"
 #include "parser/ParseStatement.hpp"
+#include "query_execution/AdmitRequestMessage.hpp"
 #include "query_execution/Foreman.hpp"
+#include "query_execution/QueryExecutionUtil.hpp"
 #include "query_execution/Worker.hpp"
 #include "query_optimizer/ExecutionGenerator.hpp"
 #include "query_optimizer/LogicalGenerator.hpp"
@@ -40,6 +42,8 @@
 
 #include "glog/logging.h"
 
+#include "tmb/tagged_message.h"
+
 namespace quickstep {
 
 class CatalogRelation;
@@ -90,13 +94,18 @@ void ExecutionGeneratorTestRunner::runTestCase(
             physical_generator.generatePlan(
                 logical_generator.generatePlan(*result.parsed_statement));
         execution_generator.generatePlan(physical_plan);
-        foreman_->setQueryPlan(
-            query_handle.getQueryPlanMutable()->getQueryPlanDAGMutable());
-
-        foreman_->reconstructQueryContextFromProto(query_handle.getQueryContextProto());
 
-        foreman_->start();
-        foreman_->join();
+        QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
+            main_thread_client_id_,
+            foreman_->getBusClientID(),
+            &query_handle,
+            &bus_);
+
+        // Receive workload completion message from Foreman.
+        const AnnotatedMessage annotated_msg =
+            bus_.Receive(main_thread_client_id_, 0, true);
+        const TaggedMessage &tagged_message = annotated_msg.tagged_message;
+        DCHECK_EQ(kWorkloadCompletionMessage, tagged_message.message_type());
 
         const CatalogRelation *query_result_relation = query_handle.getQueryResultRelation();
         if (query_result_relation) {


[09/13] incubator-quickstep git commit: Add a shell script that logs the build process for later profiling.

Posted by zu...@apache.org.
Add a shell script that logs the build process for later profiling.


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

Branch: refs/heads/travis-grpc
Commit: 07c011d27da2e8b14551b439a11e6002fd33c5fa
Parents: 540c09e
Author: Navneet Potti <na...@gmail.com>
Authored: Thu Jun 9 11:18:26 2016 -0500
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Jun 13 15:26:34 2016 -0700

----------------------------------------------------------------------
 build/profile_build.sh | 104 ++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 104 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/07c011d2/build/profile_build.sh
----------------------------------------------------------------------
diff --git a/build/profile_build.sh b/build/profile_build.sh
new file mode 100755
index 0000000..5f2fc33
--- /dev/null
+++ b/build/profile_build.sh
@@ -0,0 +1,104 @@
+#!/bin/bash
+# This script may be useful for developers to profile the build process itself.
+#
+# This shell script runs CMake and make, dumping all output into a log file.
+# It also logs the CPU usage and memory information during the build.
+# All log messages are timestamped to enable profiling.
+#
+# Dependencies:
+# - ts
+# sudo apt-get install -y libtime-duration-perl moreutils
+# - vmstat and mpstat
+# sudo apt-get install -y sysstat
+#
+# Usage: ./profile_build.sh
+# Set the CMake and make command you want to use below.
+# If CMakeLists.txt is detected, the script exits because in-source-tree build
+# is not supported.
+# If CMakeCache.txt is detected, the script skips cmake and runs make only.
+
+#   Licensed under the Apache License, Version 2.0 (the "License");
+#   you may not use this file except in compliance with the License.
+#   You may obtain a copy of the License at
+#
+#       http://www.apache.org/licenses/LICENSE-2.0
+#
+#   Unless required by applicable law or agreed to in writing, software
+#   distributed under the License is distributed on an "AS IS" BASIS,
+#   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#   See the License for the specific language governing permissions and
+#   limitations under the License.
+
+set -e
+
+CMAKE_COMMAND="cmake \
+-D BUILD_SHARED_LIBS=On \
+-D USE_TCMALLOC=0 \
+-D CMAKE_BUILD_TYPE=Debug \
+.. "
+MAKE_COMMAND="make VERBOSE=1"
+LOG_FILENAME=$LOG_FILENAME
+
+# Continuously dump memory usage and cpu load info to files for later analysis
+function start_stat_collectors {
+  rm -f stats_*.txt
+  vmstat -SM 3 | ts "%.s (%H:%M:%S)" > stats_mem.txt 3>&1 &
+  PID_vmstat=$!
+  mpstat 3 | ts "%.s (%H:%M:%S)" > stats_cpu.txt 2>&1  &
+  PID_mpstat=$!
+}
+
+function kill_stat_collectors {
+  kill $PID_vmstat
+  kill $PID_mpstat
+  exit
+}
+
+function check_directory {
+  if [[ -f CMakeLists.txt ]]; then
+    echo "Running the build in the source tree is not supported."
+    exit 1
+  fi
+}
+
+function log_repo_version_info {
+  git log master... >> $LOG_FILENAME
+  git diff master >> $LOG_FILENAME
+}
+
+function run_cmake {
+  if [[ ! -f CMakeCache.txt ]]; then
+    echo "$CMAKE_COMMAND" | tee -a $LOG_FILENAME
+    $CMAKE_COMMAND 2>&1 | ts "%.s (%H:%M:%S)" | tee -a $LOG_FILENAME
+  else
+    echo "CMakeCache.txt detected. Not running CMake again."
+  fi
+}
+
+function run_make {
+  echo "$MAKE_COMMAND" | tee -a $LOG_FILENAME
+  $MAKE_COMMAND 2>&1 | ts "%.s (%H:%M:%S)" | tee -a $LOG_FILENAME
+}
+
+function print_stats {
+  avg_mem=`grep -v r stats_mem.txt | tr -s ' ' | awk -F " " '{s+= $6; c++} END {print s/c/1024}'`
+  echo -e "\n\n"
+  echo "Average memory used was $avg_mem GB"  | tee -a $LOG_FILENAME
+
+  time_taken=`expr $END_TIME - $START_TIME`
+  mins=`expr $time_taken / 60`
+  secs=`expr $time_taken % 60`
+  echo "Time taken was ${mins}m ${secs}s" | tee -a $LOG_FILENAME
+}
+
+check_directory
+echo "Starting build in " `pwd`  >> $LOG_FILENAME
+start_stat_collectors
+trap kill_stat_collectors SIGHUP SIGINT SIGTERM
+
+START_TIME=`date +"%s"`
+run_cmake
+run_make
+END_TIME=`date +"%s"`
+kill_stat_collectors
+print_stats


[04/13] incubator-quickstep git commit: Test the default vector elision level `selection` in CI.

Posted by zu...@apache.org.
Test the default vector elision level `selection` in CI.


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

Branch: refs/heads/travis-grpc
Commit: 659967a96c597ec6c54d277d79e563e7d736d8a0
Parents: e40e3ba
Author: Jignesh Patel <jm...@hotmail.com>
Authored: Fri Jun 10 12:40:15 2016 -0500
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Fri Jun 10 11:00:08 2016 -0700

----------------------------------------------------------------------
 .travis.yml | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/659967a9/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 142dfce..fce3ba0 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -17,8 +17,8 @@ compiler:
 env:
   - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=joinwithbinaryexpressions
   - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=joinwithbinaryexpressions
-  - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=none
-  - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=none
+  - BUILD_TYPE=Debug VECTOR_COPY_ELISION_LEVEL=selection
+  - BUILD_TYPE=Release VECTOR_COPY_ELISION_LEVEL=selection
 
 before_install:
   - LLVM_VERSION=3.7.1
@@ -34,7 +34,7 @@ before_install:
     fi
 
 install:
-  - if [ "$VECTOR_COPY_ELISION_LEVEL" = "joinwithbinaryexpressions" ] && [ "$CC" = "gcc" ]; then
+  - if [ "$CC" = "gcc" ]; then
       export MAKE_JOBS=1;
     else
       export MAKE_JOBS=2;