You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by ji...@apache.org on 2017/01/26 02:34:22 UTC

[01/22] incubator-quickstep git commit: Add min/max statistics and the exactness flag into CatalogRelationStatistics. [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/exact-filter 70624122f -> d52b91265 (forced update)


Add min/max statistics and the exactness flag into CatalogRelationStatistics.


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

Branch: refs/heads/exact-filter
Commit: 9fcb0aca86cf0e63b04e307534f65644877ee327
Parents: 1b0328b
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Oct 27 14:16:32 2016 -0500
Committer: jianqiao <ji...@node-2.jianqiao.quickstep-pg0.wisc.cloudlab.us>
Committed: Tue Dec 20 09:19:50 2016 -0600

----------------------------------------------------------------------
 catalog/CMakeLists.txt                          |   3 +
 catalog/Catalog.proto                           |  11 +-
 catalog/CatalogRelationStatistics.cpp           |  60 +++++--
 catalog/CatalogRelationStatistics.hpp           | 174 ++++++++++++++++---
 cli/CMakeLists.txt                              |   3 +
 cli/CommandExecutor.cpp                         | 105 ++++++++---
 query_optimizer/ExecutionGenerator.cpp          |  36 ++--
 query_optimizer/cost_model/CMakeLists.txt       |   1 +
 query_optimizer/cost_model/SimpleCostModel.cpp  |  13 +-
 .../cost_model/StarSchemaSimpleCostModel.cpp    |  16 +-
 relational_operators/CMakeLists.txt             |   1 +
 relational_operators/SaveBlocksOperator.cpp     |  11 ++
 relational_operators/SaveBlocksOperator.hpp     |   7 +
 13 files changed, 353 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/catalog/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/catalog/CMakeLists.txt b/catalog/CMakeLists.txt
index dd4ef99..143d12f 100644
--- a/catalog/CMakeLists.txt
+++ b/catalog/CMakeLists.txt
@@ -120,6 +120,9 @@ target_link_libraries(quickstep_catalog_CatalogRelation
 target_link_libraries(quickstep_catalog_CatalogRelationStatistics
                       quickstep_catalog_CatalogTypedefs
                       quickstep_catalog_Catalog_proto
+                      quickstep_types_LongType
+                      quickstep_types_NullType
+                      quickstep_types_TypedValue
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_catalog_IndexScheme
                       glog

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index 90ce37e..a54862f 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -81,13 +81,16 @@ message IndexScheme {
 }
 
 message CatalogRelationStatistics {
-  optional fixed64 num_tuples = 1;
+  optional bool is_exact = 1;
+  optional TypedValue num_tuples = 2;
 
-  message NumDistinctValuesEntry {
+  message ColumnStats {
     required int32 attr_id = 1;
-    required fixed64 num_distinct_values = 2;
+    optional TypedValue num_distinct_values = 2;
+    optional TypedValue min_value = 3;
+    optional TypedValue max_value = 4;
   }
-  repeated NumDistinctValuesEntry num_distinct_values_map = 2;
+  repeated ColumnStats column_stats = 3;
 }
 
 message CatalogRelationSchema {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/catalog/CatalogRelationStatistics.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.cpp b/catalog/CatalogRelationStatistics.cpp
index 6a51570..f3f224d 100644
--- a/catalog/CatalogRelationStatistics.cpp
+++ b/catalog/CatalogRelationStatistics.cpp
@@ -23,28 +23,68 @@
 
 namespace quickstep {
 
+const TypedValue CatalogRelationStatistics::kNullValue =
+    NullType::InstanceNullable().makeNullValue();
+
 CatalogRelationStatistics::CatalogRelationStatistics(
     const serialization::CatalogRelationStatistics &proto) {
+  if (proto.has_is_exact()) {
+    is_exact_ = proto.is_exact();
+  } else {
+    is_exact_ = false;
+  }
+
   if (proto.has_num_tuples()) {
-    num_tuples_ = proto.num_tuples();
+    num_tuples_ = TypedValue::ReconstructFromProto(proto.num_tuples());
+  } else {
+    num_tuples_ = kNullValue;
   }
-  for (int i = 0; i < proto.num_distinct_values_map_size(); ++i) {
-    const auto &entry = proto.num_distinct_values_map(i);
-    num_distinct_values_map_.emplace(entry.attr_id(),
-                                     entry.num_distinct_values());
+
+  for (int i = 0; i < proto.column_stats_size(); ++i) {
+    const auto &stat_proto = proto.column_stats(i);
+    auto &stat = column_stats_[stat_proto.attr_id()];
+    if (stat_proto.has_num_distinct_values()) {
+      stat.num_distinct_values =
+          TypedValue::ReconstructFromProto(stat_proto.num_distinct_values());
+    }
+    if (stat_proto.has_min_value()) {
+      stat.min_value =
+          TypedValue::ReconstructFromProto(stat_proto.min_value());
+    }
+    if (stat_proto.has_max_value()) {
+      stat.max_value =
+          TypedValue::ReconstructFromProto(stat_proto.max_value());
+    }
   }
 }
 
 serialization::CatalogRelationStatistics CatalogRelationStatistics::getProto() const {
   serialization::CatalogRelationStatistics proto;
-  if (num_tuples_ != 0) {
-    proto.set_num_tuples(num_tuples_);
+
+  proto.set_is_exact(is_exact_);
+
+  if (!num_tuples_.isNull()) {
+    proto.mutable_num_tuples()->CopyFrom(num_tuples_.getProto());
   }
-  for (const auto &pair : num_distinct_values_map_) {
-    auto entry = proto.add_num_distinct_values_map();
+
+  for (const auto &pair : column_stats_) {
+    auto entry = proto.add_column_stats();
     entry->set_attr_id(pair.first);
-    entry->set_num_distinct_values(pair.second);
+    const auto &stat = pair.second;
+    if (!stat.num_distinct_values.isNull()) {
+      entry->mutable_num_distinct_values()->CopyFrom(
+          stat.num_distinct_values.getProto());
+    }
+    if (!stat.min_value.isNull()) {
+      entry->mutable_min_value()->CopyFrom(
+          stat.min_value.getProto());
+    }
+    if (!stat.max_value.isNull()) {
+      entry->mutable_max_value()->CopyFrom(
+          stat.max_value.getProto());
+    }
   }
+
   return proto;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/catalog/CatalogRelationStatistics.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.hpp b/catalog/CatalogRelationStatistics.hpp
index f2056f3..df95231 100644
--- a/catalog/CatalogRelationStatistics.hpp
+++ b/catalog/CatalogRelationStatistics.hpp
@@ -26,6 +26,9 @@
 
 #include "catalog/Catalog.pb.h"
 #include "catalog/CatalogTypedefs.hpp"
+#include "types/LongType.hpp"
+#include "types/NullType.hpp"
+#include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -44,7 +47,8 @@ class CatalogRelationStatistics {
    * @brief Constructor.
    **/
   CatalogRelationStatistics()
-      : num_tuples_(0) {}
+      : is_exact_(false),
+        num_tuples_(kNullValue) {}
 
   /**
    * @brief Reconstruct a CatalogRelationStatistics object from its serialized
@@ -64,12 +68,31 @@ class CatalogRelationStatistics {
   serialization::CatalogRelationStatistics getProto() const;
 
   /**
-   * @brief Set the number of tuples statistic.
+   * @brief Check whether the statistics are exact for the relation.
    *
-   * @param num_tuples The number of tuples statistic.
+   * return True if the statistics are exact for the relation, false otherwise.
    */
-  void setNumTuples(std::size_t num_tuples) {
-    num_tuples_ = num_tuples;
+  bool isExact() const {
+    return is_exact_;
+  }
+
+  /**
+   * @brief Set the boolean flag that indicates whether the statistics are exact
+   *        for the relation.
+   *
+   * @param Exactness of the statistics.
+   */
+  void setExactness(const bool is_exact) {
+    is_exact_ = is_exact;
+  }
+
+  /**
+   * @brief Check whether the number of tuples statistic is available.
+   *
+   * @return True if the number of tuples statistic is available, false otherwise.
+   */
+  bool hasNumTuples() const {
+    return !num_tuples_.isNull();
   }
 
   /**
@@ -78,41 +101,148 @@ class CatalogRelationStatistics {
    * @return The number of tuples. Returns 0 if the statistic is not set.
    */
   std::size_t getNumTuples() const {
-    return num_tuples_;
+    DCHECK(hasNumTuples());
+    return num_tuples_.getLiteral<std::int64_t>();
+  }
+
+  /**
+   * @brief Set the number of tuples statistic.
+   *
+   * @param num_tuples The number of tuples statistic.
+   */
+  void setNumTuples(const std::size_t num_tuples) {
+    num_tuples_ = LongType::InstanceNonNullable().makeValue(&num_tuples);
   }
 
   /**
-   * @brief Set the number of distinct values statistic for a column (catalog attribute).
+   * @brief Check whether the number of distinct values statistic is available
+   *        for a column.
+   *
+   * @param The id of the column.
+   * @return True if the number of distinct values statistic is available,
+   *         false otherwise.
+   */
+  bool hasNumDistinctValues(const attribute_id attr_id) const {
+    const ColumnStats *stats = getColumnStats(attr_id);
+    return (stats != nullptr && !stats->num_distinct_values.isNull());
+  }
+
+  /**
+   * @brief Get the number of distinct values statistic for a column.
+   *
+   * @param The id of the column.
+   * @return The number of distinct values statistic for the column.
+   */
+  std::size_t getNumDistinctValues(const attribute_id attr_id) const {
+    DCHECK(hasNumDistinctValues(attr_id));
+    return column_stats_.at(attr_id).num_distinct_values.getLiteral<std::int64_t>();
+  }
+
+  /**
+   * @brief Set the number of distinct values statistic for a column.
    *
    * @param attr_id The id of the column.
    * @param num_distinct_values The number of distinct values statistic.
    */
-  void setNumDistinctValues(attribute_id attr_id, std::size_t num_distinct_values) {
-    num_distinct_values_map_[attr_id] = num_distinct_values;
+  void setNumDistinctValues(const attribute_id attr_id,
+                            const std::size_t num_distinct_values) {
+    column_stats_[attr_id].num_distinct_values =
+        LongType::InstanceNonNullable().makeValue(&num_distinct_values);
   }
 
   /**
-   * @brief Get the number of distinct values statistic for a column (catalog attribute).
+   * @brief Check whether the minimum value statistic is available for a column.
    *
    * @param The id of the column.
-   * @return The number of distinct values statistic for the column. Returns 0
-   *         if the statistic is not set.
+   * @return True if the minimum value statistic is available, false otherwise.
    */
-  std::size_t getNumDistinctValues(attribute_id attr_id) const {
-    const auto it = num_distinct_values_map_.find(attr_id);
-    if (it == num_distinct_values_map_.end()) {
-      return static_cast<std::size_t>(0);
-    } else {
-      return it->second;
-    }
+  bool hasMinValue(const attribute_id attr_id) const {
+    const ColumnStats *stats = getColumnStats(attr_id);
+    return (stats != nullptr && !stats->min_value.isNull());
+  }
+
+  /**
+   * @brief Get the minimum value statistic for a column.
+   *
+   * @param The id of the column.
+   * @return The minimum value statistic for the column.
+   */
+  const TypedValue& getMinValue(const attribute_id attr_id) const {
+    DCHECK(hasMinValue(attr_id));
+    return column_stats_.at(attr_id).min_value;
+  }
+
+  /**
+   * @brief Set the minimum value statistic for a column.
+   *
+   * @param The id of the column.
+   * @return The minimum value statistic for the column.
+   */
+  void setMinValue(const attribute_id attr_id,
+                   const TypedValue &min_value) {
+    column_stats_[attr_id].min_value = min_value;
+  }
+
+  /**
+   * @brief Check whether the maximum value statistic is available for a column.
+   *
+   * @param The id of the column.
+   * @return True if the maximum value statistic is available, false otherwise.
+   */
+  bool hasMaxValue(const attribute_id attr_id) const {
+    const ColumnStats *stats = getColumnStats(attr_id);
+    return (stats != nullptr && !stats->max_value.isNull());
+  }
+
+  /**
+   * @brief Get the maximum value statistic for a column.
+   *
+   * @param The id of the column.
+   * @return The maximum value statistic for the column.
+   */
+  const TypedValue& getMaxValue(const attribute_id attr_id) const {
+    DCHECK(hasMaxValue(attr_id));
+    return column_stats_.at(attr_id).max_value;
+  }
+
+  /**
+   * @brief Set the maximum value statistic for a column.
+   *
+   * @param The id of the column.
+   * @return The maximum value statistic for the column.
+   */
+  void setMaxValue(const attribute_id attr_id,
+                   const TypedValue &max_value) {
+    column_stats_[attr_id].max_value = max_value;
   }
 
  private:
+  struct ColumnStats {
+    ColumnStats()
+        : num_distinct_values(kNullValue),
+          min_value(kNullValue),
+          max_value(kNullValue) {
+    }
+    TypedValue num_distinct_values;
+    TypedValue min_value;
+    TypedValue max_value;
+  };
+
+  inline const ColumnStats* getColumnStats(const attribute_id attr_id) const {
+    const auto it = column_stats_.find(attr_id);
+    return (it == column_stats_.end() ? nullptr : &it->second);
+  }
+
+  static const TypedValue kNullValue;
+
+  // Whether the statistics are exact (i.e. up-to-date) for the relation.
+  bool is_exact_;
+
   // Total number of tuples in the relation.
-  std::size_t num_tuples_;
+  TypedValue num_tuples_;
 
-  // Number of distinct values for each column.
-  std::unordered_map<attribute_id, std::size_t> num_distinct_values_map_;
+  // Statistics for each column.
+  std::unordered_map<attribute_id, ColumnStats> column_stats_;
 
   DISALLOW_COPY_AND_ASSIGN(CatalogRelationStatistics);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/cli/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/CMakeLists.txt b/cli/CMakeLists.txt
index be13c82..8c7fe2d 100644
--- a/cli/CMakeLists.txt
+++ b/cli/CMakeLists.txt
@@ -89,8 +89,11 @@ target_link_libraries(quickstep_cli_CommandExecutor
                       quickstep_catalog_CatalogDatabase
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationSchema
+                      quickstep_catalog_CatalogRelationStatistics
                       quickstep_cli_DropRelation
                       quickstep_cli_PrintToScreen
+                      quickstep_expressions_aggregation_AggregateFunctionMax
+                      quickstep_expressions_aggregation_AggregateFunctionMin
                       quickstep_parser_ParseStatement
                       quickstep_parser_SqlParserWrapper
                       quickstep_queryoptimizer_QueryHandle

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index 3c510e7..78bff98 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -31,8 +31,11 @@
 #include "catalog/CatalogDatabase.hpp"
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogRelationSchema.hpp"
+#include "catalog/CatalogRelationStatistics.hpp"
 #include "cli/DropRelation.hpp"
 #include "cli/PrintToScreen.hpp"
+#include "expressions/aggregation/AggregateFunctionMax.hpp"
+#include "expressions/aggregation/AggregateFunctionMin.hpp"
 #include "parser/ParseStatement.hpp"
 #include "parser/ParseString.hpp"
 #include "parser/SqlParserWrapper.hpp"
@@ -69,7 +72,7 @@ namespace {
 
 namespace C = ::quickstep::cli;
 
-void executeDescribeDatabase(
+void ExecuteDescribeDatabase(
     const PtrVector<ParseString> *arguments,
     const CatalogDatabase &catalog_database,
     StorageManager *storage_manager,
@@ -131,7 +134,7 @@ void executeDescribeDatabase(
   }
 }
 
-void executeDescribeTable(
+void ExecuteDescribeTable(
     const PtrVector<ParseString> *arguments,
     const CatalogDatabase &catalog_database, FILE *out) {
   const ParseString &table_name = arguments->front();
@@ -199,7 +202,7 @@ void executeDescribeTable(
 /**
  * @brief A helper function that executes a SQL query to obtain a row of results.
  */
-inline std::vector<TypedValue> executeQueryForSingleRow(
+inline std::vector<TypedValue> ExecuteQueryForSingleRow(
     const tmb::client_id main_thread_client_id,
     const tmb::client_id foreman_client_id,
     const std::string &query_string,
@@ -270,7 +273,7 @@ inline std::vector<TypedValue> executeQueryForSingleRow(
 /**
  * @brief A helper function that executes a SQL query to obtain a scalar result.
  */
-inline TypedValue executeQueryForSingleResult(
+inline TypedValue ExecuteQueryForSingleResult(
     const tmb::client_id main_thread_client_id,
     const tmb::client_id foreman_client_id,
     const std::string &query_string,
@@ -279,7 +282,7 @@ inline TypedValue executeQueryForSingleResult(
     QueryProcessor *query_processor,
     SqlParserWrapper *parser_wrapper) {
   std::vector<TypedValue> results =
-      executeQueryForSingleRow(main_thread_client_id,
+      ExecuteQueryForSingleRow(main_thread_client_id,
                                foreman_client_id,
                                query_string,
                                bus,
@@ -290,7 +293,21 @@ inline TypedValue executeQueryForSingleResult(
   return results[0];
 }
 
-void executeAnalyze(const PtrVector<ParseString> *arguments,
+/**
+ * @brief A helper function for escaping quotes (i.e. ' or ").
+ */
+std::string EscapeQuotes(const std::string &str, const char quote) {
+  std::string ret;
+  for (const char c : str) {
+    ret.push_back(c);
+    if (c == quote) {
+      ret.push_back(c);
+    }
+  }
+  return ret;
+}
+
+void ExecuteAnalyze(const PtrVector<ParseString> *arguments,
                     const tmb::client_id main_thread_client_id,
                     const tmb::client_id foreman_client_id,
                     MessageBus *bus,
@@ -321,17 +338,42 @@ void executeAnalyze(const PtrVector<ParseString> *arguments,
 
     CatalogRelation *mutable_relation =
         query_processor->getDefaultDatabase()->getRelationByIdMutable(relation.getID());
+    CatalogRelationStatistics *mutable_stat =
+        mutable_relation->getStatisticsMutable();
+
+    const std::string rel_name = EscapeQuotes(relation.getName(), '"');
 
     // Get the number of distinct values for each column.
     for (const CatalogAttribute &attribute : relation) {
-      std::string query_string = "SELECT COUNT(DISTINCT ";
-      query_string.append(attribute.getName());
-      query_string.append(") FROM ");
-      query_string.append(relation.getName());
-      query_string.append(";");
+      const std::string attr_name = EscapeQuotes(attribute.getName(), '"');
+      const Type &attr_type = attribute.getType();
+      bool is_min_applicable =
+          AggregateFunctionMin::Instance().canApplyToTypes({&attr_type});
+      bool is_max_applicable =
+          AggregateFunctionMax::Instance().canApplyToTypes({&attr_type});
+
+      // NOTE(jianqiao): Note that the relation name and the attribute names may
+      // contain non-letter characters, e.g. CREATE TABLE "with space"("1" int).
+      // So here we need to format the names with double quotes (").
+      std::string query_string = "SELECT COUNT(DISTINCT \"";
+      query_string.append(attr_name);
+      query_string.append("\")");
+      if (is_min_applicable) {
+        query_string.append(", MIN(\"");
+        query_string.append(attr_name);
+        query_string.append("\")");
+      }
+      if (is_max_applicable) {
+        query_string.append(", MAX(\"");
+        query_string.append(attr_name);
+        query_string.append("\")");
+      }
+      query_string.append(" FROM \"");
+      query_string.append(rel_name);
+      query_string.append("\";");
 
       std::vector<TypedValue> results =
-          executeQueryForSingleRow(main_thread_client_id,
+          ExecuteQueryForSingleRow(main_thread_client_id,
                                    foreman_client_id,
                                    query_string,
                                    bus,
@@ -339,21 +381,29 @@ void executeAnalyze(const PtrVector<ParseString> *arguments,
                                    query_processor,
                                    parser_wrapper.get());
 
-      auto *stat = mutable_relation->getStatisticsMutable();
-      const attribute_id attr_id = attribute.getID();
+      auto results_it = results.begin();
+      DCHECK(results_it->getTypeID() == TypeID::kLong);
 
-      DCHECK(results[0].getTypeID() == TypeID::kLong);
-      stat->setNumDistinctValues(attr_id,
-                                 results[0].getLiteral<std::int64_t>());
+      const attribute_id attr_id = attribute.getID();
+      mutable_stat->setNumDistinctValues(attr_id,
+                                         results_it->getLiteral<std::int64_t>());
+      if (is_min_applicable) {
+        ++results_it;
+        mutable_stat->setMinValue(attr_id, *results_it);
+      }
+      if (is_max_applicable) {
+        ++results_it;
+        mutable_stat->setMaxValue(attr_id, *results_it);
+      }
     }
 
     // Get the number of tuples for the relation.
-    std::string query_string = "SELECT COUNT(*) FROM ";
-    query_string.append(relation.getName());
-    query_string.append(";");
+    std::string query_string = "SELECT COUNT(*) FROM \"";
+    query_string.append(rel_name);
+    query_string.append("\";");
 
     TypedValue num_tuples =
-        executeQueryForSingleResult(main_thread_client_id,
+        ExecuteQueryForSingleResult(main_thread_client_id,
                                     foreman_client_id,
                                     query_string,
                                     bus,
@@ -362,8 +412,9 @@ void executeAnalyze(const PtrVector<ParseString> *arguments,
                                     parser_wrapper.get());
 
     DCHECK(num_tuples.getTypeID() == TypeID::kLong);
-    mutable_relation->getStatisticsMutable()->setNumTuples(
-        num_tuples.getLiteral<std::int64_t>());
+    mutable_stat->setNumTuples(num_tuples.getLiteral<std::int64_t>());
+
+    mutable_stat->setExactness(true);
 
     fprintf(out, "done\n");
     fflush(out);
@@ -386,15 +437,15 @@ void executeCommand(const ParseStatement &statement,
   const PtrVector<ParseString> *arguments = command.arguments();
   const std::string &command_str = command.command()->value();
   if (command_str == C::kDescribeDatabaseCommand) {
-    executeDescribeDatabase(arguments, catalog_database, storage_manager, out);
+    ExecuteDescribeDatabase(arguments, catalog_database, storage_manager, out);
   } else if (command_str == C::kDescribeTableCommand) {
     if (arguments->size() == 0) {
-      executeDescribeDatabase(arguments, catalog_database, storage_manager, out);
+      ExecuteDescribeDatabase(arguments, catalog_database, storage_manager, out);
     } else {
-      executeDescribeTable(arguments, catalog_database, out);
+      ExecuteDescribeTable(arguments, catalog_database, out);
     }
   } else if (command_str == C::kAnalyzeCommand) {
-    executeAnalyze(arguments,
+    ExecuteAnalyze(arguments,
                    main_thread_client_id,
                    foreman_client_id,
                    bus,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 5a2c450..d24f498 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -924,9 +924,11 @@ void ExecutionGenerator::convertCopyFrom(
               insert_destination_index));
   insert_destination_proto->set_relational_op_index(scan_operator_index);
 
+  CatalogRelation *mutable_output_relation =
+      catalog_database_->getRelationByIdMutable(output_relation->getID());
   const QueryPlan::DAGNodeIndex save_blocks_operator_index =
       execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator(query_handle_->query_id()));
+          new SaveBlocksOperator(query_handle_->query_id(), mutable_output_relation));
   execution_plan_->addDirectDependency(save_blocks_operator_index,
                                        scan_operator_index,
                                        false /* is_pipeline_breaker */);
@@ -1038,13 +1040,16 @@ void ExecutionGenerator::convertDeleteTuples(
   const CatalogRelationInfo *input_relation_info =
       findRelationInfoOutputByPhysical(physical_plan->input());
   DCHECK(input_relation_info != nullptr);
+
+  const CatalogRelation *input_relation = input_relation_info->relation;
+
   if (execution_predicate == nullptr ||
       (execution_predicate->hasStaticResult() &&
        execution_predicate->getStaticResult())) {
     const QueryPlan::DAGNodeIndex drop_table_index =
         execution_plan_->addRelationalOperator(
             new DropTableOperator(query_handle_->query_id(),
-                                  *input_relation_info->relation,
+                                  *input_relation,
                                   catalog_database_,
                                   true /* only_drop_blocks */));
     if (!input_relation_info->isStoredRelation()) {
@@ -1059,7 +1064,7 @@ void ExecutionGenerator::convertDeleteTuples(
     const QueryPlan::DAGNodeIndex delete_tuples_index =
         execution_plan_->addRelationalOperator(
             new DeleteOperator(query_handle_->query_id(),
-                               *input_relation_info->relation,
+                               *input_relation,
                                execution_predicate_index,
                                input_relation_info->isStoredRelation()));
 
@@ -1069,9 +1074,11 @@ void ExecutionGenerator::convertDeleteTuples(
                                            false /* is_pipeline_breaker */);
     }
 
+    CatalogRelation *mutable_relation =
+        catalog_database_->getRelationByIdMutable(input_relation->getID());
     const QueryPlan::DAGNodeIndex save_blocks_index =
         execution_plan_->addRelationalOperator(
-            new SaveBlocksOperator(query_handle_->query_id()));
+            new SaveBlocksOperator(query_handle_->query_id(), mutable_relation));
     execution_plan_->addDirectDependency(save_blocks_index,
                                          delete_tuples_index,
                                          false /* is_pipeline_breaker */);
@@ -1150,9 +1157,11 @@ void ExecutionGenerator::convertInsertTuple(
                              tuple_index));
   insert_destination_proto->set_relational_op_index(insert_operator_index);
 
+  CatalogRelation *mutable_relation =
+      catalog_database_->getRelationByIdMutable(input_relation.getID());
   const QueryPlan::DAGNodeIndex save_blocks_index =
       execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator(query_handle_->query_id()));
+          new SaveBlocksOperator(query_handle_->query_id(), mutable_relation));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(insert_operator_index,
                                          input_relation_info->producer_operator_index,
@@ -1201,6 +1210,7 @@ void ExecutionGenerator::convertInsertSelection(
 
   const CatalogRelationInfo *selection_relation_info =
       findRelationInfoOutputByPhysical(physical_plan->selection());
+  const CatalogRelation *selection_relation = selection_relation_info->relation;
 
   // Prepare the attributes, which are output columns of the selection relation.
   std::vector<attribute_id> attributes;
@@ -1221,7 +1231,7 @@ void ExecutionGenerator::convertInsertSelection(
   // physical plan by modifying class Physical.
   SelectOperator *insert_selection_op =
       new SelectOperator(query_handle_->query_id(),
-                         *selection_relation_info->relation,
+                         *selection_relation,
                          destination_relation,
                          insert_destination_index,
                          QueryContext::kInvalidPredicateId,
@@ -1232,8 +1242,11 @@ void ExecutionGenerator::convertInsertSelection(
       execution_plan_->addRelationalOperator(insert_selection_op);
   insert_destination_proto->set_relational_op_index(insert_selection_index);
 
+  CatalogRelation *mutable_relation =
+      catalog_database_->getRelationByIdMutable(selection_relation->getID());
   const QueryPlan::DAGNodeIndex save_blocks_index =
-      execution_plan_->addRelationalOperator(new SaveBlocksOperator(query_handle_->query_id()));
+      execution_plan_->addRelationalOperator(
+          new SaveBlocksOperator(query_handle_->query_id(), mutable_relation));
 
   if (!selection_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(insert_selection_index,
@@ -1253,7 +1266,8 @@ void ExecutionGenerator::convertUpdateTable(
       findRelationInfoOutputByPhysical(physical_plan->input());
   DCHECK(input_relation_info != nullptr);
 
-  const relation_id input_rel_id = input_relation_info->relation->getID();
+  const CatalogRelation *input_relation = input_relation_info->relation;
+  const relation_id input_rel_id = input_relation->getID();
 
   // Create InsertDestination proto.
   const QueryContext::insert_destination_id relocation_destination_index =
@@ -1303,8 +1317,7 @@ void ExecutionGenerator::convertUpdateTable(
   const QueryPlan::DAGNodeIndex update_operator_index =
       execution_plan_->addRelationalOperator(new UpdateOperator(
           query_handle_->query_id(),
-          *catalog_database_->getRelationById(
-              input_rel_id),
+          *input_relation,
           relocation_destination_index,
           execution_predicate_index,
           update_group_index));
@@ -1312,7 +1325,8 @@ void ExecutionGenerator::convertUpdateTable(
 
   const QueryPlan::DAGNodeIndex save_blocks_index =
       execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator(query_handle_->query_id()));
+          new SaveBlocksOperator(query_handle_->query_id(),
+                                 catalog_database_->getRelationByIdMutable(input_rel_id)));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(update_operator_index,
                                          input_relation_info->producer_operator_index,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/query_optimizer/cost_model/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/CMakeLists.txt b/query_optimizer/cost_model/CMakeLists.txt
index 032e34c..90133e7 100644
--- a/query_optimizer/cost_model/CMakeLists.txt
+++ b/query_optimizer/cost_model/CMakeLists.txt
@@ -48,6 +48,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_SimpleCostModel
 target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
                       glog
                       quickstep_catalog_CatalogRelation
+                      quickstep_catalog_CatalogRelationStatistics
                       quickstep_queryoptimizer_costmodel_CostModel
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ComparisonExpression

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index bf6da6a..7808898 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -94,13 +94,12 @@ std::size_t SimpleCostModel::estimateCardinalityForTopLevelPlan(
 
 std::size_t SimpleCostModel::estimateCardinalityForTableReference(
     const P::TableReferencePtr &physical_plan) {
-  const std::size_t num_tuples_in_relation =
-      physical_plan->relation()->getStatistics().getNumTuples();
-  if (num_tuples_in_relation == 0) {
-    return physical_plan->relation()->estimateTupleCardinality();
-  } else {
-    return num_tuples_in_relation;
-  }
+  const CatalogRelation *relation = physical_plan->relation();
+  const CatalogRelationStatistics &stat = relation->getStatistics();
+  const std::size_t num_tuples =
+      stat.hasNumTuples() ? stat.getNumTuples()
+                          : relation->estimateTupleCardinality();
+  return num_tuples;
 }
 
 std::size_t SimpleCostModel::estimateCardinalityForSelection(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index dfafa7d..75b1b2b 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -25,6 +25,7 @@
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
+#include "catalog/CatalogRelationStatistics.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ComparisonExpression.hpp"
@@ -105,10 +106,11 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinalityForTopLevelPlan(
 
 std::size_t StarSchemaSimpleCostModel::estimateCardinalityForTableReference(
     const P::TableReferencePtr &physical_plan) {
-  std::size_t num_tuples = physical_plan->relation()->getStatistics().getNumTuples();
-  if (num_tuples == 0) {
-    num_tuples = physical_plan->relation()->estimateTupleCardinality();
-  }
+  const CatalogRelation *relation = physical_plan->relation();
+  const CatalogRelationStatistics &stat = relation->getStatistics();
+  const std::size_t num_tuples =
+      stat.hasNumTuples() ? stat.getNumTuples()
+                          : relation->estimateTupleCardinality();
   return num_tuples;
 }
 
@@ -385,9 +387,9 @@ std::size_t StarSchemaSimpleCostModel::getNumDistinctValues(
   const std::vector<E::AttributeReferencePtr> &attributes = table_reference->attribute_list();
   for (std::size_t i = 0; i < attributes.size(); ++i) {
     if (attributes[i]->id() == attribute_id) {
-      std::size_t num_distinct_values = relation.getStatistics().getNumDistinctValues(i);
-      if (num_distinct_values > 0) {
-        return num_distinct_values;
+      const CatalogRelationStatistics &stat = relation.getStatistics();
+      if (stat.hasNumDistinctValues(i)) {
+        return stat.getNumDistinctValues(i);
       }
       break;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 0735bce..9e4b1b6 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -303,6 +303,7 @@ target_link_libraries(quickstep_relationaloperators_SampleOperator
                       tmb)
 target_link_libraries(quickstep_relationaloperators_SaveBlocksOperator
                       glog
+                      quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/relational_operators/SaveBlocksOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.cpp b/relational_operators/SaveBlocksOperator.cpp
index 04d4211..a5a96c4 100644
--- a/relational_operators/SaveBlocksOperator.cpp
+++ b/relational_operators/SaveBlocksOperator.cpp
@@ -21,6 +21,7 @@
 
 #include <vector>
 
+#include "catalog/CatalogRelation.hpp"
 #include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
 #include "relational_operators/WorkOrder.pb.h"
@@ -70,6 +71,16 @@ void SaveBlocksOperator::feedInputBlock(const block_id input_block_id, const rel
   destination_block_ids_.push_back(input_block_id);
 }
 
+void SaveBlocksOperator::updateCatalogOnCompletion() {
+  // Note(jianqiao): We need to reset the exactness flag whenever a stored
+  // relation gets changed. Given the pre-condition that all the data manipulation
+  // operations (insert, delete, update, copy) use this SaveBlocksOperator as a
+  // common routine. It is valid to put the updating call here to minimize the
+  // changes to other relational operators (e.g. InsertOperator, DeleteOperator,
+  // TextScanOperator, SelectOperator, etc.).
+  relation_->getStatisticsMutable()->setExactness(false);
+}
+
 void SaveBlocksWorkOrder::execute() {
   // It may happen that the block gets saved to disk as a result of an eviction,
   // before this invocation. In either case, we don't care about the return

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index e84b641..a8d5327 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -38,6 +38,7 @@ namespace tmb { class MessageBus; }
 
 namespace quickstep {
 
+class CatalogRelation;
 class QueryContext;
 class StorageManager;
 class WorkOrderProtosContainer;
@@ -56,13 +57,16 @@ class SaveBlocksOperator : public RelationalOperator {
    * @brief Constructor for saving only modified blocks in a relation.
    *
    * @param query_id The ID of the query to which this operator belongs.
+   * @param relation The relation whose blocks will be saved.
    * @param force If true, force writing of all blocks to disk, otherwise only
    *        write dirty blocks.
    **/
   explicit SaveBlocksOperator(const std::size_t query_id,
+                              CatalogRelation *relation,
                               const bool force = false)
       : RelationalOperator(query_id),
         force_(force),
+        relation_(relation),
         num_workorders_generated_(0) {}
 
   ~SaveBlocksOperator() override {}
@@ -89,9 +93,12 @@ class SaveBlocksOperator : public RelationalOperator {
     }
   }
 
+  void updateCatalogOnCompletion() override;
+
  private:
   const bool force_;
 
+  CatalogRelation *relation_;
   std::vector<block_id> destination_block_ids_;
 
   std::vector<block_id>::size_type num_workorders_generated_;



[10/22] incubator-quickstep git commit: Refactored SelectOperator::getAllWorkOrders.

Posted by ji...@apache.org.
Refactored SelectOperator::getAllWorkOrders.


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

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

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


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

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


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

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


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

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

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


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

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

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

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

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


[21/22] incubator-quickstep git commit: Add BitVectorExactFilter as a LIP filter and supports Join-to-Semijoin transformation.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/rules/InjectJoinFilters.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/InjectJoinFilters.hpp b/query_optimizer/rules/InjectJoinFilters.hpp
new file mode 100644
index 0000000..0eaebdc
--- /dev/null
+++ b/query_optimizer/rules/InjectJoinFilters.hpp
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_RULES_INJECT_JOIN_FILTERS_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_INJECT_JOIN_FILTERS_HPP_
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/physical/LIPFilterConfiguration.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+class InjectJoinFilters : public Rule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  InjectJoinFilters() {}
+
+  ~InjectJoinFilters() override {}
+
+  std::string getName() const override {
+    return "TransformFilterJoins";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  bool isTransformable(const physical::HashJoinPtr &hash_join) const;
+
+  physical::PhysicalPtr transformHashJoinToFilters(
+      const physical::PhysicalPtr &input) const;
+
+  physical::PhysicalPtr pushDownFilters(const physical::PhysicalPtr &input) const;
+
+  physical::PhysicalPtr addFilterAnchors(const physical::PhysicalPtr &input,
+                                         const bool ancestor_can_anchor_filter) const;
+
+  void concretizeAsLIPFilters(const physical::PhysicalPtr &input,
+                              const physical::PhysicalPtr &anchor_node) const;
+
+  physical::PhysicalPtr pushDownFiltersInternal(
+      const physical::PhysicalPtr &probe_child,
+      const physical::PhysicalPtr &build_child,
+      const physical::FilterJoinPtr &filter_join) const;
+
+  bool findMinMaxValuesForAttributeHelper(
+      const physical::PhysicalPtr &physical_plan,
+      const expressions::AttributeReferencePtr &attribute,
+      std::int64_t *min_cpp_value,
+      std::int64_t *max_cpp_value) const;
+
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
+  std::unique_ptr<physical::LIPFilterConfiguration> lip_filter_configuration_;
+
+  // 1G bits = 128MB
+  static constexpr std::int64_t kMaxFilterSize = 1000000000;
+
+  DISALLOW_COPY_AND_ASSIGN(InjectJoinFilters);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_INJECT_JOIN_FILTERS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/relational_operators/BuildLIPFilterOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildLIPFilterOperator.cpp b/relational_operators/BuildLIPFilterOperator.cpp
new file mode 100644
index 0000000..34df385
--- /dev/null
+++ b/relational_operators/BuildLIPFilterOperator.cpp
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "relational_operators/BuildLIPFilterOperator.hpp"
+
+#include <memory>
+#include <vector>
+
+#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/StorageBlock.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/TupleIdSequence.hpp"
+#include "storage/TupleStorageSubBlock.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "utility/lip_filter/LIPFilterAdaptiveProber.hpp"
+#include "utility/lip_filter/LIPFilterBuilder.hpp"
+#include "utility/lip_filter/LIPFilterUtil.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace quickstep {
+
+bool BuildLIPFilterOperator::getAllWorkOrders(
+    WorkOrdersContainer *container,
+    QueryContext *query_context,
+    StorageManager *storage_manager,
+    const tmb::client_id scheduler_client_id,
+    tmb::MessageBus *bus) {
+  DCHECK(query_context != nullptr);
+
+  const Predicate *build_side_predicate =
+      query_context->getPredicate(build_side_predicate_index_);
+
+  if (input_relation_is_stored_) {
+    if (!started_) {
+      for (const block_id input_block_id : input_relation_block_ids_) {
+        container->addNormalWorkOrder(
+            new BuildLIPFilterWorkOrder(query_id_,
+                                       input_relation_,
+                                       input_block_id,
+                                       build_side_predicate,
+                                       storage_manager,
+                                       CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context),
+                                       CreateLIPFilterBuilderHelper(lip_deployment_index_, query_context)),
+            op_index_);
+      }
+      started_ = true;
+    }
+    return started_;
+  } else {
+    while (num_workorders_generated_ < input_relation_block_ids_.size()) {
+      container->addNormalWorkOrder(
+          new BuildLIPFilterWorkOrder(
+              query_id_,
+              input_relation_,
+              input_relation_block_ids_[num_workorders_generated_],
+              build_side_predicate,
+              storage_manager,
+              CreateLIPFilterAdaptiveProberHelper(lip_deployment_index_, query_context),
+              CreateLIPFilterBuilderHelper(lip_deployment_index_, query_context)),
+          op_index_);
+      ++num_workorders_generated_;
+    }
+    return done_feeding_input_relation_;
+  }
+}
+
+bool BuildLIPFilterOperator::getAllWorkOrderProtos(WorkOrderProtosContainer *container) {
+  // TODO
+  return true;
+}
+
+serialization::WorkOrder* BuildLIPFilterOperator::createWorkOrderProto(const block_id block) {
+  // TODO
+  return nullptr;
+}
+
+void BuildLIPFilterWorkOrder::execute() {
+  BlockReference block(
+      storage_manager_->getBlock(build_block_id_, input_relation_));
+
+  std::unique_ptr<TupleIdSequence> predicate_matches;
+  if (build_side_predicate_ != nullptr) {
+    predicate_matches.reset(block->getMatchesForPredicate(build_side_predicate_));
+  }
+
+  std::unique_ptr<ValueAccessor> accessor(
+      block->getTupleStorageSubBlock().createValueAccessor(predicate_matches.get()));
+
+  if (lip_filter_adaptive_prober_ != nullptr) {
+    std::unique_ptr<TupleIdSequence> matches(
+        lip_filter_adaptive_prober_->filterValueAccessor(accessor.get()));
+    std::unique_ptr<ValueAccessor> filtered_accessor(
+        accessor->createSharedTupleIdSequenceAdapterVirtual(*matches));
+
+    lip_filter_builder_->insertValueAccessor(filtered_accessor.get());
+  } else {
+    lip_filter_builder_->insertValueAccessor(accessor.get());
+  }
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/relational_operators/BuildLIPFilterOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildLIPFilterOperator.hpp b/relational_operators/BuildLIPFilterOperator.hpp
new file mode 100644
index 0000000..fe8a0fb
--- /dev/null
+++ b/relational_operators/BuildLIPFilterOperator.hpp
@@ -0,0 +1,165 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_RELATIONAL_OPERATORS_BUILD_LIP_FILTER_OPERATOR_HPP_
+#define QUICKSTEP_RELATIONAL_OPERATORS_BUILD_LIP_FILTER_OPERATOR_HPP_
+
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "catalog/CatalogRelation.hpp"
+#include "catalog/CatalogTypedefs.hpp"
+#include "query_execution/QueryContext.hpp"
+#include "relational_operators/RelationalOperator.hpp"
+#include "relational_operators/WorkOrder.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "utility/Macros.hpp"
+#include "utility/lip_filter/LIPFilterAdaptiveProber.hpp"
+#include "utility/lip_filter/LIPFilterBuilder.hpp"
+
+#include "glog/logging.h"
+
+#include "tmb/id_typedefs.h"
+
+namespace tmb { class MessageBus; }
+
+namespace quickstep {
+
+class CatalogRelationSchema;
+class Predicate;
+class StorageManager;
+class WorkOrderProtosContainer;
+class WorkOrdersContainer;
+
+namespace serialization { class WorkOrder; }
+
+/** \addtogroup RelationalOperators
+ *  @{
+ */
+
+/**
+ * @brief An operator which builds a LIPFilter on one relation.
+ **/
+class BuildLIPFilterOperator : public RelationalOperator {
+ public:
+  BuildLIPFilterOperator(const std::size_t query_id,
+                         const CatalogRelation &input_relation,
+                         const QueryContext::predicate_id build_side_predicate_index,
+                         const bool input_relation_is_stored)
+    : RelationalOperator(query_id),
+      input_relation_(input_relation),
+      build_side_predicate_index_(build_side_predicate_index),
+      input_relation_is_stored_(input_relation_is_stored),
+      input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
+                                                         : std::vector<block_id>()),
+      num_workorders_generated_(0),
+      started_(false) {}
+
+  ~BuildLIPFilterOperator() override {}
+
+  const CatalogRelation& input_relation() const {
+    return input_relation_;
+  }
+
+  std::string getName() const override {
+    return "BuildLIPFilterOperator";
+  }
+
+  bool getAllWorkOrders(WorkOrdersContainer *container,
+                        QueryContext *query_context,
+                        StorageManager *storage_manager,
+                        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,
+                      const partition_id part_id) override {
+    input_relation_block_ids_.push_back(input_block_id);
+  }
+
+ 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 QueryContext::predicate_id build_side_predicate_index_;
+  const bool input_relation_is_stored_;
+
+  std::vector<block_id> input_relation_block_ids_;
+  std::vector<block_id>::size_type num_workorders_generated_;
+
+  bool started_;
+
+  DISALLOW_COPY_AND_ASSIGN(BuildLIPFilterOperator);
+};
+
+/**
+ * @brief A WorkOrder produced by BuildLIPFilterOperator
+ **/
+class BuildLIPFilterWorkOrder : public WorkOrder {
+ public:
+  BuildLIPFilterWorkOrder(const std::size_t query_id,
+                          const CatalogRelationSchema &input_relation,
+                          const block_id build_block_id,
+                          const Predicate *build_side_predicate,
+                          StorageManager *storage_manager,
+                          LIPFilterAdaptiveProber *lip_filter_adaptive_prober,
+                          LIPFilterBuilder *lip_filter_builder)
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
+        build_block_id_(build_block_id),
+        build_side_predicate_(build_side_predicate),
+        storage_manager_(DCHECK_NOTNULL(storage_manager)),
+        lip_filter_adaptive_prober_(lip_filter_adaptive_prober),
+        lip_filter_builder_(DCHECK_NOTNULL(lip_filter_builder)) {}
+
+  ~BuildLIPFilterWorkOrder() override {}
+
+  const CatalogRelationSchema& input_relation() const {
+    return input_relation_;
+  }
+
+  void execute() override;
+
+ private:
+  const CatalogRelationSchema &input_relation_;
+  const block_id build_block_id_;
+  const Predicate *build_side_predicate_;
+
+  StorageManager *storage_manager_;
+
+  std::unique_ptr<LIPFilterAdaptiveProber> lip_filter_adaptive_prober_;
+  std::unique_ptr<LIPFilterBuilder> lip_filter_builder_;
+
+  DISALLOW_COPY_AND_ASSIGN(BuildLIPFilterWorkOrder);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_RELATIONAL_OPERATORS_BUILD_LIP_FILTER_OPERATOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 78da7b8..85887b1 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -38,6 +38,7 @@ endif()
 # Declare micro-libs:
 add_library(quickstep_relationaloperators_AggregationOperator AggregationOperator.cpp AggregationOperator.hpp)
 add_library(quickstep_relationaloperators_BuildHashOperator BuildHashOperator.cpp BuildHashOperator.hpp)
+add_library(quickstep_relationaloperators_BuildLIPFilterOperator BuildLIPFilterOperator.cpp BuildLIPFilterOperator.hpp)
 add_library(quickstep_relationaloperators_CreateIndexOperator CreateIndexOperator.cpp CreateIndexOperator.hpp)
 add_library(quickstep_relationaloperators_CreateTableOperator CreateTableOperator.cpp CreateTableOperator.hpp)
 add_library(quickstep_relationaloperators_DestroyAggregationStateOperator
@@ -117,6 +118,27 @@ target_link_libraries(quickstep_relationaloperators_BuildHashOperator
                       quickstep_utility_lipfilter_LIPFilterBuilder
                       quickstep_utility_lipfilter_LIPFilterUtil
                       tmb)
+target_link_libraries(quickstep_relationaloperators_BuildLIPFilterOperator
+                      glog
+                      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_StorageBlock
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageManager
+                      quickstep_storage_TupleIdSequence
+                      quickstep_storage_TupleStorageSubBlock
+                      quickstep_storage_ValueAccessor
+                      quickstep_utility_Macros
+                      quickstep_utility_lipfilter_LIPFilterAdaptiveProber
+                      quickstep_utility_lipfilter_LIPFilterBuilder
+                      quickstep_utility_lipfilter_LIPFilterUtil
+                      tmb)
 target_link_libraries(quickstep_relationaloperators_CreateIndexOperator
                       glog
                       quickstep_catalog_CatalogRelation
@@ -517,6 +539,7 @@ target_link_libraries(quickstep_relationaloperators_WorkOrder_proto
 add_library(quickstep_relationaloperators ../empty_src.cpp RelationalOperatorsModule.hpp)
 target_link_libraries(quickstep_relationaloperators
                       quickstep_relationaloperators_AggregationOperator
+                      quickstep_relationaloperators_BuildLIPFilterOperator
                       quickstep_relationaloperators_BuildHashOperator
                       quickstep_relationaloperators_CreateIndexOperator
                       quickstep_relationaloperators_CreateTableOperator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index 6a656fb..4990083 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -274,6 +274,7 @@ target_link_libraries(quickstep_utility_PlanVisualizer
                       quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ExprId
+                      quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_LIPFilterConfiguration
                       quickstep_queryoptimizer_physical_Physical

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/utility/PlanVisualizer.cpp
----------------------------------------------------------------------
diff --git a/utility/PlanVisualizer.cpp b/utility/PlanVisualizer.cpp
index df7a20c..e533e15 100644
--- a/utility/PlanVisualizer.cpp
+++ b/utility/PlanVisualizer.cpp
@@ -32,6 +32,7 @@
 #include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/PhysicalType.hpp"
@@ -58,6 +59,8 @@ std::string PlanVisualizer::visualize(const P::PhysicalPtr &input) {
 
   color_map_["TableReference"] = "skyblue";
   color_map_["Selection"] = "#90EE90";
+  color_map_["FilterJoin"] = "pink";
+  color_map_["FilterJoin(Anti)"] = "pink";
   color_map_["HashJoin"] = "red";
   color_map_["HashLeftOuterJoin"] = "orange";
   color_map_["HashLeftSemiJoin"] = "orange";
@@ -126,7 +129,8 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
     edge_info.dst_node_id = node_id;
     edge_info.dashed = false;
 
-    if (input->getPhysicalType() == P::PhysicalType::kHashJoin &&
+    if ((input->getPhysicalType() == P::PhysicalType::kHashJoin ||
+         input->getPhysicalType() == P::PhysicalType::kFilterJoin) &&
         child == input->children()[1]) {
       edge_info.dashed = true;
     }
@@ -165,6 +169,20 @@ void PlanVisualizer::visit(const P::PhysicalPtr &input) {
       }
       break;
     }
+    case P::PhysicalType::kFilterJoin: {
+      const P::FilterJoinPtr filter_join =
+        std::static_pointer_cast<const P::FilterJoin>(input);
+      node_info.labels.emplace_back(input->getName());
+
+      const auto &probe_attributes = filter_join->probe_attributes();
+      const auto &build_attributes = filter_join->build_attributes();
+      for (std::size_t i = 0; i < probe_attributes.size(); ++i) {
+        node_info.labels.emplace_back(
+            probe_attributes[i]->attribute_alias() + " = " +
+                build_attributes[i]->attribute_alias());
+      }
+      break;
+    }
     default: {
       node_info.labels.emplace_back(input->getName());
       break;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/utility/lip_filter/BitVectorExactFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/BitVectorExactFilter.hpp b/utility/lip_filter/BitVectorExactFilter.hpp
new file mode 100644
index 0000000..15c8f0b
--- /dev/null
+++ b/utility/lip_filter/BitVectorExactFilter.hpp
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_LIP_FILTER_BIT_VECTOR_EXACT_FILTER_HPP_
+#define QUICKSTEP_UTILITY_LIP_FILTER_BIT_VECTOR_EXACT_FILTER_HPP_
+
+#include <atomic>
+#include <cstddef>
+#include <cstdint>
+#include <cstring>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/Type.hpp"
+#include "utility/Macros.hpp"
+#include "utility/lip_filter/LIPFilter.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+template <typename CppType, bool is_anti_filter>
+class BitVectorExactFilter : public LIPFilter {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param filter_cardinality The cardinality of this hash filter.
+   */
+  explicit BitVectorExactFilter(const std::size_t filter_cardinality)
+      : LIPFilter(LIPFilterType::kBitVectorExactFilter),
+        filter_cardinality_(filter_cardinality),
+        bit_array_(GetByteSize(filter_cardinality)) {
+    DCHECK_GE(filter_cardinality, 0u);
+    std::memset(bit_array_.data(),
+                0x0,
+                sizeof(std::atomic<std::uint8_t>) * GetByteSize(filter_cardinality));
+  }
+
+  void insertValueAccessor(ValueAccessor *accessor,
+                           const attribute_id attr_id,
+                           const Type *attr_type) override {
+    InvokeOnAnyValueAccessor(
+        accessor,
+        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+      if (attr_type->isNullable()) {
+        this->insertValueAccessorInternal<true>(accessor, attr_id);
+      } else {
+        this->insertValueAccessorInternal<false>(accessor, attr_id);
+      }
+    });
+  }
+
+  std::size_t filterBatch(ValueAccessor *accessor,
+                          const attribute_id attr_id,
+                          const bool is_attr_nullable,
+                          std::vector<tuple_id> *batch,
+                          const std::size_t batch_size) const override {
+    DCHECK(batch != nullptr);
+    DCHECK_LE(batch_size, batch->size());
+
+    return InvokeOnAnyValueAccessor(
+        accessor,
+        [&](auto *accessor) -> std::size_t {  // NOLINT(build/c++11)
+      if (is_attr_nullable) {
+        return this->filterBatchInternal<true>(accessor, attr_id, batch, batch_size);
+      } else {
+        return this->filterBatchInternal<false>(accessor, attr_id, batch, batch_size);
+      }
+    });
+  }
+
+ private:
+  /**
+   * @brief Round up bit_size to multiples of 8.
+   */
+  inline static std::size_t GetByteSize(const std::size_t bit_size) {
+    return (bit_size + 7) / 8;
+  }
+
+  /**
+   * @brief Iterate through the accessor and hash values into the internal bit
+   *        array.
+   */
+  template <bool is_attr_nullable, typename ValueAccessorT>
+  inline void insertValueAccessorInternal(ValueAccessorT *accessor,
+                                          const attribute_id attr_id) {
+    accessor->beginIteration();
+    while (accessor->next()) {
+      const void *value = accessor->template getUntypedValue<is_attr_nullable>(attr_id);
+      if (!is_attr_nullable || value != nullptr) {
+        insert(value);
+      }
+    }
+  }
+
+  /**
+   * @brief Filter the given batch of tuples from a ValueAccessor. Write the
+   *        tuple ids which survive in the filtering back to \p batch.
+   */
+  template <bool is_attr_nullable, typename ValueAccessorT>
+  inline std::size_t filterBatchInternal(const ValueAccessorT *accessor,
+                                         const attribute_id attr_id,
+                                         std::vector<tuple_id> *batch,
+                                         const std::size_t batch_size) const {
+    std::size_t out_size = 0;
+    for (std::size_t i = 0; i < batch_size; ++i) {
+      const tuple_id tid = batch->at(i);
+      const void *value =
+          accessor->template getUntypedValueAtAbsolutePosition(attr_id, tid);
+      if (is_attr_nullable && value == nullptr) {
+        continue;
+      }
+      if (contains(value)) {
+        batch->at(out_size) = tid;
+        ++out_size;
+      }
+    }
+    return out_size;
+  }
+
+  /**
+   * @brief Inserts a given value into the exact filter.
+   */
+  inline void insert(const void *key_begin) {
+    const CppType loc = *reinterpret_cast<const CppType *>(key_begin);
+    DCHECK_LE(loc, filter_cardinality_);
+    bit_array_[loc >> 3u].fetch_or(1u << (loc & 7u), std::memory_order_relaxed);
+  }
+
+  /**
+   * @brief Test membership of a given value in the exact filter.
+   */
+  inline bool contains(const void *key_begin) const {
+    const CppType loc = *reinterpret_cast<const CppType *>(key_begin);
+    DCHECK_LE(loc, filter_cardinality_);
+    const bool is_bit_set =
+        (bit_array_[loc >> 3u].load(std::memory_order_relaxed) & (1u << (loc & 7u))) != 0;
+    if (is_anti_filter) {
+      return !is_bit_set;
+    } else {
+      return is_bit_set;
+    }
+  }
+
+  std::size_t filter_cardinality_;
+  alignas(kCacheLineBytes) std::vector<std::atomic<std::uint8_t>> bit_array_;
+
+  DISALLOW_COPY_AND_ASSIGN(BitVectorExactFilter);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_LIP_FILTER_BIT_VECTOR_EXACT_FILTER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/utility/lip_filter/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/lip_filter/CMakeLists.txt b/utility/lip_filter/CMakeLists.txt
index 23b3763..edd0d24 100644
--- a/utility/lip_filter/CMakeLists.txt
+++ b/utility/lip_filter/CMakeLists.txt
@@ -20,6 +20,7 @@ QS_PROTOBUF_GENERATE_CPP(utility_lipfilter_LIPFilter_proto_srcs
                          LIPFilter.proto)
 
 # Declare micro-libs:
+add_library(quickstep_utility_lipfilter_BitVectorExactFilter ../../empty_src.cpp BitVectorExactFilter.hpp)
 add_library(quickstep_utility_lipfilter_LIPFilter ../../empty_src.cpp LIPFilter.hpp)
 add_library(quickstep_utility_lipfilter_LIPFilterAdaptiveProber ../../empty_src.cpp LIPFilterAdaptiveProber.hpp)
 add_library(quickstep_utility_lipfilter_LIPFilterBuilder ../../empty_src.cpp LIPFilterBuilder.hpp)
@@ -31,6 +32,15 @@ add_library(quickstep_utility_lipfilter_LIPFilter_proto
 add_library(quickstep_utility_lipfilter_SingleIdentityHashFilter ../../empty_src.cpp SingleIdentityHashFilter.hpp)
 
 # Link dependencies:
+target_link_libraries(quickstep_utility_lipfilter_BitVectorExactFilter
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageConstants
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_types_Type
+                      quickstep_utility_lipfilter_LIPFilter
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_lipfilter_LIPFilter
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_StorageBlockInfo
@@ -56,6 +66,7 @@ target_link_libraries(quickstep_utility_lipfilter_LIPFilterDeployment
                       quickstep_utility_lipfilter_LIPFilterBuilder
                       quickstep_utility_lipfilter_LIPFilter_proto)
 target_link_libraries(quickstep_utility_lipfilter_LIPFilterFactory
+                      quickstep_utility_lipfilter_BitVectorExactFilter
                       quickstep_utility_lipfilter_LIPFilter_proto
                       quickstep_utility_lipfilter_SingleIdentityHashFilter
                       quickstep_utility_Macros)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/utility/lip_filter/LIPFilter.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilter.hpp b/utility/lip_filter/LIPFilter.hpp
index 682d69f..ba38264 100644
--- a/utility/lip_filter/LIPFilter.hpp
+++ b/utility/lip_filter/LIPFilter.hpp
@@ -37,8 +37,8 @@ class ValueAccessor;
  */
 
 enum class LIPFilterType {
+  kBitVectorExactFilter,
   kBloomFilter,
-  kExactFilter,
   kSingleIdentityHashFilter
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/utility/lip_filter/LIPFilter.proto
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilter.proto b/utility/lip_filter/LIPFilter.proto
index def13dd..fcbaa7b 100644
--- a/utility/lip_filter/LIPFilter.proto
+++ b/utility/lip_filter/LIPFilter.proto
@@ -22,8 +22,8 @@ package quickstep.serialization;
 import "types/Type.proto";
 
 enum LIPFilterType {
-  BLOOM_FILTER = 1;
-  EXACT_FILTER = 2;
+  BIT_VECTOR_EXACT_FILTER = 1;
+  BLOOM_FILTER = 2;
   SINGLE_IDENTITY_HASH_FILTER = 3;
 }
 
@@ -33,17 +33,21 @@ message LIPFilter {
   extensions 16 to max;
 }
 
-message SingleIdentityHashFilter {
+message BitVectorExactFilter {
   extend LIPFilter {
     // All required
     optional uint64 filter_cardinality = 16;
     optional uint64 attribute_size = 17;
+    optional bool is_anti_filter = 18;
   }
 }
 
-enum LIPFilterActionType {
-  BUILD = 1;
-  PROBE = 2;
+message SingleIdentityHashFilter {
+  extend LIPFilter {
+    // All required
+    optional uint64 filter_cardinality = 24;
+    optional uint64 attribute_size = 25;
+  }
 }
 
 message LIPFilterDeployment {
@@ -53,6 +57,6 @@ message LIPFilterDeployment {
     required Type attribute_type = 3;
   }
 
-  required LIPFilterActionType action_type = 1;
-  repeated Entry entries = 2;
+  repeated Entry build_entries = 1;
+  repeated Entry probe_entries = 2;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/utility/lip_filter/LIPFilterDeployment.cpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilterDeployment.cpp b/utility/lip_filter/LIPFilterDeployment.cpp
index cd4d90f..bbb6dd6 100644
--- a/utility/lip_filter/LIPFilterDeployment.cpp
+++ b/utility/lip_filter/LIPFilterDeployment.cpp
@@ -35,38 +35,44 @@ namespace quickstep {
 LIPFilterDeployment::LIPFilterDeployment(
     const serialization::LIPFilterDeployment &proto,
     const std::vector<std::unique_ptr<LIPFilter>> &lip_filters) {
-  switch (proto.action_type()) {
-    case serialization::LIPFilterActionType::BUILD:
-      action_type_ = LIPFilterActionType::kBuild;
-      break;
-    case serialization::LIPFilterActionType::PROBE:
-      action_type_ = LIPFilterActionType::kProbe;
-      break;
-    default:
-      LOG(FATAL) << "Unsupported LIPFilterActionType: "
-                 << serialization::LIPFilterActionType_Name(proto.action_type());
+  if (proto.build_entries_size() > 0) {
+    build_.reset(new LIPFilterDeploymentInfo());
+    for (int i = 0; i < proto.build_entries_size(); ++i) {
+      const auto &entry_proto = proto.build_entries(i);
+      build_->lip_filters_.emplace_back(
+          lip_filters.at(entry_proto.lip_filter_id()).get());
+      build_->attr_ids_.emplace_back(entry_proto.attribute_id());
+      build_->attr_types_.emplace_back(
+          &TypeFactory::ReconstructFromProto(entry_proto.attribute_type()));
+    }
   }
 
-  for (int i = 0; i < proto.entries_size(); ++i) {
-    const auto &entry_proto = proto.entries(i);
-    lip_filters_.emplace_back(lip_filters.at(entry_proto.lip_filter_id()).get());
-    attr_ids_.emplace_back(entry_proto.attribute_id());
-    attr_types_.emplace_back(&TypeFactory::ReconstructFromProto(entry_proto.attribute_type()));
+  if (proto.probe_entries_size() > 0) {
+    probe_.reset(new LIPFilterDeploymentInfo());
+    for (int i = 0; i < proto.probe_entries_size(); ++i) {
+      const auto &entry_proto = proto.probe_entries(i);
+      probe_->lip_filters_.emplace_back(
+          lip_filters.at(entry_proto.lip_filter_id()).get());
+      probe_->attr_ids_.emplace_back(entry_proto.attribute_id());
+      probe_->attr_types_.emplace_back(
+          &TypeFactory::ReconstructFromProto(entry_proto.attribute_type()));
+    }
   }
 }
 
 bool LIPFilterDeployment::ProtoIsValid(
     const serialization::LIPFilterDeployment &proto) {
-  if (proto.action_type() != serialization::LIPFilterActionType::BUILD &&
-      proto.action_type() != serialization::LIPFilterActionType::PROBE) {
-    LOG(FATAL) << "Unsupported LIPFilterActionType: "
-               << serialization::LIPFilterActionType_Name(proto.action_type());
-  }
-  if (proto.entries_size() == 0) {
+  if (proto.build_entries_size() == 0 && proto.probe_entries_size() == 0) {
     return false;
   }
-  for (int i = 0; i < proto.entries_size(); ++i) {
-    const auto &entry_proto = proto.entries(i);
+  for (int i = 0; i < proto.build_entries_size(); ++i) {
+    const auto &entry_proto = proto.build_entries(i);
+    if (!TypeFactory::ProtoIsValid(entry_proto.attribute_type())) {
+      return false;
+    }
+  }
+  for (int i = 0; i < proto.probe_entries_size(); ++i) {
+    const auto &entry_proto = proto.probe_entries(i);
     if (!TypeFactory::ProtoIsValid(entry_proto.attribute_type())) {
       return false;
     }
@@ -75,13 +81,23 @@ bool LIPFilterDeployment::ProtoIsValid(
 }
 
 LIPFilterBuilder* LIPFilterDeployment::createLIPFilterBuilder() const {
-  DCHECK(action_type_ == LIPFilterActionType::kBuild);
-  return new LIPFilterBuilder(lip_filters_, attr_ids_, attr_types_);
+  if (build_ == nullptr) {
+    return nullptr;
+  }
+
+  return new LIPFilterBuilder(build_->lip_filters_,
+                              build_->attr_ids_,
+                              build_->attr_types_);
 }
 
 LIPFilterAdaptiveProber* LIPFilterDeployment::createLIPFilterAdaptiveProber() const {
-  DCHECK(action_type_ == LIPFilterActionType::kProbe);
-  return new LIPFilterAdaptiveProber(lip_filters_, attr_ids_, attr_types_);
+  if (probe_ == nullptr) {
+    return nullptr;
+  }
+
+  return new LIPFilterAdaptiveProber(probe_->lip_filters_,
+                                     probe_->attr_ids_,
+                                     probe_->attr_types_);
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/utility/lip_filter/LIPFilterDeployment.hpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilterDeployment.hpp b/utility/lip_filter/LIPFilterDeployment.hpp
index 9b37f88..ab1259b 100644
--- a/utility/lip_filter/LIPFilterDeployment.hpp
+++ b/utility/lip_filter/LIPFilterDeployment.hpp
@@ -39,11 +39,6 @@ class Type;
  *  @{
  */
 
-enum class LIPFilterActionType {
-  kBuild = 0,
-  kProbe
-};
-
 /**
  * @brief Helper class for organizing a group of LIPFilters in the backend.
  *        Each LIPFilterDeployment object is attached to a RelationalOperator.
@@ -69,16 +64,6 @@ class LIPFilterDeployment {
   static bool ProtoIsValid(const serialization::LIPFilterDeployment &proto);
 
   /**
-   * @brief Get the action type for this group of LIPFilters (i.e. whether
-   *        to build or probe the filters).
-   *
-   * @return The action type.
-   */
-  LIPFilterActionType getActionType() const {
-    return action_type_;
-  }
-
-  /**
    * @brief Create a LIPFilterBuilder for this group of LIPFilters.
    *
    * @return A new LIPFilterBuilder object for this group of LIPFilters.
@@ -95,11 +80,14 @@ class LIPFilterDeployment {
   LIPFilterAdaptiveProber* createLIPFilterAdaptiveProber() const;
 
  private:
-  LIPFilterActionType action_type_;
-
-  std::vector<LIPFilter *> lip_filters_;
-  std::vector<attribute_id> attr_ids_;
-  std::vector<const Type *> attr_types_;
+  struct LIPFilterDeploymentInfo {
+    std::vector<LIPFilter *> lip_filters_;
+    std::vector<attribute_id> attr_ids_;
+    std::vector<const Type *> attr_types_;
+  };
+
+  std::unique_ptr<LIPFilterDeploymentInfo> build_;
+  std::unique_ptr<LIPFilterDeploymentInfo> probe_;
 
   DISALLOW_COPY_AND_ASSIGN(LIPFilterDeployment);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/utility/lip_filter/LIPFilterFactory.cpp
----------------------------------------------------------------------
diff --git a/utility/lip_filter/LIPFilterFactory.cpp b/utility/lip_filter/LIPFilterFactory.cpp
index ebc4a0e..a96151f 100644
--- a/utility/lip_filter/LIPFilterFactory.cpp
+++ b/utility/lip_filter/LIPFilterFactory.cpp
@@ -23,6 +23,7 @@
 #include <cstdint>
 
 #include "utility/lip_filter/LIPFilter.pb.h"
+#include "utility/lip_filter/BitVectorExactFilter.hpp"
 #include "utility/lip_filter/SingleIdentityHashFilter.hpp"
 
 #include "glog/logging.h"
@@ -31,6 +32,44 @@ namespace quickstep {
 
 LIPFilter* LIPFilterFactory::ReconstructFromProto(const serialization::LIPFilter &proto) {
   switch (proto.lip_filter_type()) {
+    case serialization::LIPFilterType::BIT_VECTOR_EXACT_FILTER: {
+      const std::size_t attr_size =
+          proto.GetExtension(serialization::BitVectorExactFilter::attribute_size);
+      const std::size_t filter_cardinality =
+          proto.GetExtension(serialization::BitVectorExactFilter::filter_cardinality);
+      const bool is_anti_filter =
+          proto.GetExtension(serialization::BitVectorExactFilter::is_anti_filter);
+
+      switch (attr_size) {
+        case 1:
+          if (is_anti_filter) {
+            return new BitVectorExactFilter<std::uint8_t, true>(filter_cardinality);
+          } else {
+            return new BitVectorExactFilter<std::uint8_t, false>(filter_cardinality);
+          }
+        case 2:
+          if (is_anti_filter) {
+            return new BitVectorExactFilter<std::uint16_t, true>(filter_cardinality);
+          } else {
+            return new BitVectorExactFilter<std::uint16_t, false>(filter_cardinality);
+          }
+        case 4:
+          if (is_anti_filter) {
+            return new BitVectorExactFilter<std::uint32_t, true>(filter_cardinality);
+          } else {
+            return new BitVectorExactFilter<std::uint32_t, false>(filter_cardinality);
+          }
+        case 8:
+          if (is_anti_filter) {
+            return new BitVectorExactFilter<std::uint64_t, true>(filter_cardinality);
+          } else {
+            return new BitVectorExactFilter<std::uint64_t, false>(filter_cardinality);
+          }
+        default:
+          LOG(FATAL) << "Invalid attribute size for BitVectorExactFilter: "
+                     << attr_size;
+      }
+    }
     case serialization::LIPFilterType::SINGLE_IDENTITY_HASH_FILTER: {
       const std::size_t attr_size =
           proto.GetExtension(serialization::SingleIdentityHashFilter::attribute_size);
@@ -57,6 +96,13 @@ LIPFilter* LIPFilterFactory::ReconstructFromProto(const serialization::LIPFilter
 
 bool LIPFilterFactory::ProtoIsValid(const serialization::LIPFilter &proto) {
   switch (proto.lip_filter_type()) {
+    case serialization::LIPFilterType::BIT_VECTOR_EXACT_FILTER: {
+      const std::size_t attr_size =
+          proto.GetExtension(serialization::BitVectorExactFilter::attribute_size);
+      const std::size_t filter_cardinality =
+          proto.GetExtension(serialization::BitVectorExactFilter::filter_cardinality);
+      return (attr_size != 0 && filter_cardinality != 0);
+    }
     case serialization::LIPFilterType::SINGLE_IDENTITY_HASH_FILTER: {
       const std::size_t attr_size =
           proto.GetExtension(serialization::SingleIdentityHashFilter::attribute_size);



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

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


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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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


[03/22] incubator-quickstep git commit: Removes packed row store.

Posted by ji...@apache.org.
Removes packed row store.


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

Branch: refs/heads/exact-filter
Commit: 5fee82186da965eb204ec2435dab5b0f7690920e
Parents: c394405
Author: cramja <ma...@gmail.com>
Authored: Thu Nov 17 14:16:50 2016 -0600
Committer: cramja <ma...@gmail.com>
Committed: Sun Jan 8 14:55:21 2017 -0600

----------------------------------------------------------------------
 catalog/tests/Catalog_unittest.cpp              |  26 +-
 query_optimizer/OptimizerTree.hpp               |   4 -
 query_optimizer/resolver/Resolver.cpp           |   5 +-
 .../tests/logical_generator/Create.test         |   4 +-
 query_optimizer/tests/resolver/Create.test      |  56 +-
 relational_operators/CMakeLists.txt             |   2 -
 .../tests/SortMergeRunOperator_unittest.cpp     |   1 -
 .../SortRunGenerationOperator_unittest.cpp      |   1 -
 storage/PackedRowStoreTupleStorageSubBlock.cpp  | 491 ----------------
 storage/PackedRowStoreTupleStorageSubBlock.hpp  | 216 -------
 storage/PackedRowStoreValueAccessor.hpp         | 150 -----
 storage/SplitRowStoreTupleStorageSubBlock.cpp   |   4 +-
 storage/SplitRowStoreTupleStorageSubBlock.hpp   |   2 +-
 storage/StorageBlock.cpp                        |   7 -
 storage/StorageBlockInfo.cpp                    |   3 +-
 storage/StorageBlockInfo.hpp                    |   1 -
 storage/StorageBlockLayout.hpp                  |   4 +-
 storage/StorageBlockLayout.proto                |   9 +-
 storage/SubBlockTypeRegistry.hpp                |   4 +-
 storage/ValueAccessor.hpp                       |   1 -
 storage/ValueAccessorUtil.hpp                   |  11 -
 ...ColumnStoreTupleStorageSubBlock_unittest.cpp |   2 +-
 ...kedRowStoreTupleStorageSubBlock_unittest.cpp |   2 +-
 ...kedRowStoreTupleStorageSubBlock_unittest.cpp | 584 -------------------
 24 files changed, 44 insertions(+), 1546 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/catalog/tests/Catalog_unittest.cpp
----------------------------------------------------------------------
diff --git a/catalog/tests/Catalog_unittest.cpp b/catalog/tests/Catalog_unittest.cpp
index e430b89..276bfa9 100644
--- a/catalog/tests/Catalog_unittest.cpp
+++ b/catalog/tests/Catalog_unittest.cpp
@@ -52,7 +52,6 @@ QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(BasicColumnStoreTupleStorageSubBlock);
 QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(CSBTreeIndexSubBlock);
 QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(CompressedColumnStoreTupleStorageSubBlock);
 QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(CompressedPackedRowStoreTupleStorageSubBlock);
-QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(PackedRowStoreTupleStorageSubBlock);
 QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(SMAIndexSubBlock);
 QUICKSTEP_FORCE_SUB_BLOCK_REGISTRATION(SplitRowStoreTupleStorageSubBlock);
 
@@ -79,8 +78,6 @@ class CatalogTest : public ::testing::Test {
     EXPECT_EQ(expected.sub_block_type(), checked.sub_block_type());
 
     switch (expected.sub_block_type()) {
-      case TupleStorageSubBlockDescription::PACKED_ROW_STORE:
-        break;
       case TupleStorageSubBlockDescription::BASIC_COLUMN_STORE:
         EXPECT_TRUE(expected.HasExtension(BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id));
         EXPECT_TRUE(checked.HasExtension(BasicColumnStoreTupleStorageSubBlockDescription::sort_attribute_id));
@@ -436,27 +433,6 @@ TEST_F(CatalogTest, DroppedCatalogRelationSerializationTest) {
   checkCatalogSerialization();
 }
 
-TEST_F(CatalogTest, CatalogPackedRowStoreSerializationTest) {
-  CatalogRelation* const rel = createCatalogRelation("rel");
-  StorageBlockLayoutDescription layout_description(rel->getDefaultStorageBlockLayout().getDescription());
-
-  rel->addAttribute(new CatalogAttribute(nullptr, "attr_int", TypeFactory::GetType(kInt)));
-  rel->addAttribute(new CatalogAttribute(nullptr, "attr_long", TypeFactory::GetType(kLong)));
-  rel->addAttribute(new CatalogAttribute(nullptr, "attr_float", TypeFactory::GetType(kFloat)));
-  rel->addAttribute(new CatalogAttribute(nullptr, "attr_double", TypeFactory::GetType(kDouble)));
-
-  const std::size_t str_type_length = 20;
-  rel->addAttribute(new CatalogAttribute(nullptr, "attr_char", TypeFactory::GetType(kChar, str_type_length)));
-  // NOTE(zuyu): PackedRowStoreTupleStorageSubBlock does NOT support variable-length attributes.
-
-  layout_description.mutable_tuple_store_description()->set_sub_block_type(
-      TupleStorageSubBlockDescription::PACKED_ROW_STORE);
-
-  rel->setDefaultStorageBlockLayout(new StorageBlockLayout(*rel, layout_description));
-
-  checkCatalogSerialization();
-}
-
 TEST_F(CatalogTest, CatalogBasicColumnStoreSerializationTest) {
   CatalogRelation* const rel = createCatalogRelation("rel");
   StorageBlockLayoutDescription layout_description(rel->getDefaultStorageBlockLayout().getDescription());
@@ -569,7 +545,7 @@ TEST_F(CatalogTest, CatalogIndexTest) {
   rel->addAttribute(new CatalogAttribute(nullptr, "attr_idx2", TypeFactory::GetType(kInt)));
 
   layout_description.mutable_tuple_store_description()->set_sub_block_type(
-      TupleStorageSubBlockDescription::PACKED_ROW_STORE);
+      TupleStorageSubBlockDescription::SPLIT_ROW_STORE);
 
   rel->setDefaultStorageBlockLayout(new StorageBlockLayout(*rel, layout_description));
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/query_optimizer/OptimizerTree.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/OptimizerTree.hpp b/query_optimizer/OptimizerTree.hpp
index 8ea2c6f..62df66d 100644
--- a/query_optimizer/OptimizerTree.hpp
+++ b/query_optimizer/OptimizerTree.hpp
@@ -233,10 +233,6 @@ OptimizerProtoRepresentation<TreeNodeType>* getOptimizerRepresentationForProto(
   const ::quickstep::TupleStorageSubBlockDescription &storage_block_description
       = description->tuple_store_description();
   switch (storage_block_description.sub_block_type()) {
-    case TupleStorageSubBlockDescription::PACKED_ROW_STORE: {
-      node->addProperty("blocktype", "rowstore");
-      break;
-    }
     case TupleStorageSubBlockDescription::SPLIT_ROW_STORE: {
       node->addProperty("blocktype", "split_rowstore");
       break;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 38ec24a..2580342 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -520,10 +520,7 @@ StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
         << "TYPE property must be specified and be a string.";
   }
   const std::string type_string = ToLower(type_parse_string->value());
-  if (type_string.compare("rowstore") == 0) {
-    description->set_sub_block_type(
-        quickstep::TupleStorageSubBlockDescription::PACKED_ROW_STORE);
-  } else if (type_string.compare("split_rowstore") == 0) {
+  if (type_string.compare("split_rowstore") == 0) {
     description->set_sub_block_type(
         quickstep::TupleStorageSubBlockDescription::SPLIT_ROW_STORE);
   } else if (type_string.compare("columnstore") == 0) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/query_optimizer/tests/logical_generator/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/logical_generator/Create.test b/query_optimizer/tests/logical_generator/Create.test
index a35a92a..aeff9ec 100644
--- a/query_optimizer/tests/logical_generator/Create.test
+++ b/query_optimizer/tests/logical_generator/Create.test
@@ -16,12 +16,12 @@
 # under the License.
 
 [default optimized_logical_plan]
-CREATE TABLE foo (attr int) WITH BLOCKPROPERTIES (TYPE rowstore, BLOCKSIZEMB 10)
+CREATE TABLE foo (attr int) WITH BLOCKPROPERTIES (TYPE split_rowstore, BLOCKSIZEMB 10)
 --
 TopLevelPlan
 +-plan=CreateTable[relation=foo]
 | +-block_properties=ProtoDescription
-| | +-Property=ProtoProperty[Property=blocktype,Value=rowstore]
+| | +-Property=ProtoProperty[Property=blocktype,Value=split_rowstore]
 | | +-Property=ProtoProperty[Property=slots,Value=5]
 | +-attributes=
 |   +-AttributeReference[id=0,name=attr,relation=foo,type=Int]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/query_optimizer/tests/resolver/Create.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/resolver/Create.test b/query_optimizer/tests/resolver/Create.test
index 7cd980f..b04d785 100644
--- a/query_optimizer/tests/resolver/Create.test
+++ b/query_optimizer/tests/resolver/Create.test
@@ -90,20 +90,20 @@ create table "_qstemp_result_s" (col int)
 
 # Cannot repeat Block properties.
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, TYPE columnstore);
+(TYPE columnstore, TYPE columnstore);
 --
-ERROR: Properties must be specified at most once. (2 : 17)
-(TYPE rowstore, TYPE columnstore);
-                ^
+ERROR: Properties must be specified at most once. (2 : 20)
+(TYPE columnstore, TYPE columnstore);
+                   ^
 ==
 
 # Unrecognized BLOCKPROPERTIES should throw.
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, INVALID_PROP val);
+(TYPE split_rowstore, INVALID_PROP val);
 --
-ERROR: Unrecognized property name. (2 : 17)
-(TYPE rowstore, INVALID_PROP val);
-                ^
+ERROR: Unrecognized property name. (2 : 23)
+(TYPE split_rowstore, INVALID_PROP val);
+                      ^
 ==
 
 # TYPE property only accepts names of storage blocks.
@@ -126,11 +126,11 @@ BLOCKPROPERTIES (BLOCKSIZEMB 1...
 
 # Rowstores cannot have a sorted attribute.
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, SORT attr);
+(TYPE split_rowstore, SORT attr);
 --
-ERROR: The SORT property does not apply to this block type. (2 : 22)
-(TYPE rowstore, SORT attr);
-                     ^
+ERROR: The SORT property does not apply to this block type. (2 : 28)
+(TYPE split_rowstore, SORT attr);
+                           ^
 ==
 
 # Columnstores require a sort attribute.
@@ -197,35 +197,35 @@ ERROR: The given attribute was not found. (2 : 44)
 
 # BLOCKSIZEMB property must be an integer.
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, BLOCKSIZEMB ten);
+(TYPE split_rowstore, BLOCKSIZEMB ten);
 --
-ERROR: The BLOCKSIZEMB property must be an integer. (2 : 17)
-(TYPE rowstore, BLOCKSIZEMB ten);
-                ^
+ERROR: The BLOCKSIZEMB property must be an integer. (2 : 23)
+(TYPE split_rowstore, BLOCKSIZEMB ten);
+                      ^
 ==
 
 # BLOCKSIZEMB property must be multiple times of the slot size.
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, BLOCKSIZEMB 25);
+(TYPE split_rowstore, BLOCKSIZEMB 25);
 --
-ERROR: The BLOCKSIZEMB property must be multiple times of 2MB. (2 : 17)
-(TYPE rowstore, BLOCKSIZEMB 25);
-                ^
+ERROR: The BLOCKSIZEMB property must be multiple times of 2MB. (2 : 23)
+(TYPE split_rowstore, BLOCKSIZEMB 25);
+                      ^
 ==
 
 # BLOCKSIZEMB must be greater than the minimum (defined in StorageConstants.hpp).
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, BLOCKSIZEMB 0);
+(TYPE split_rowstore, BLOCKSIZEMB 0);
 --
-ERROR: The BLOCKSIZEMB property must be between 2MB and 1024MB. (2 : 17)
-(TYPE rowstore, BLOCKSIZEMB 0);
-                ^
+ERROR: The BLOCKSIZEMB property must be between 2MB and 1024MB. (2 : 23)
+(TYPE split_rowstore, BLOCKSIZEMB 0);
+                      ^
 ==
 
 # BLOCKSIZEMB must be less than the maximum (defined in StorageConstants.hpp).
 CREATE TABLE foo (attr INT) WITH BLOCKPROPERTIES
-(TYPE rowstore, BLOCKSIZEMB 2000);
+(TYPE split_rowstore, BLOCKSIZEMB 2000);
 --
-ERROR: The BLOCKSIZEMB property must be between 2MB and 1024MB. (2 : 17)
-(TYPE rowstore, BLOCKSIZEMB 2000);
-                ^
+ERROR: The BLOCKSIZEMB property must be between 2MB and 1024MB. (2 : 23)
+(TYPE split_rowstore, BLOCKSIZEMB 2000);
+                      ^

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 9e4b1b6..526820e 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -674,7 +674,6 @@ target_link_libraries(SortMergeRunOperator_unittest
                       quickstep_storage_CountedReference
                       quickstep_storage_InsertDestination
                       quickstep_storage_InsertDestination_proto
-                      quickstep_storage_PackedRowStoreValueAccessor
                       quickstep_storage_SplitRowStoreValueAccessor
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo
@@ -723,7 +722,6 @@ target_link_libraries(SortRunGenerationOperator_unittest
                       quickstep_storage_CompressedPackedRowStoreValueAccessor
                       quickstep_storage_CountedReference
                       quickstep_storage_InsertDestination
-                      quickstep_storage_PackedRowStoreValueAccessor
                       quickstep_storage_SplitRowStoreValueAccessor
                       quickstep_storage_StorageBlock
                       quickstep_storage_StorageBlockInfo

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index b37d81f..a9128b1 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -51,7 +51,6 @@
 #include "storage/CountedReference.hpp"
 #include "storage/InsertDestination.hpp"
 #include "storage/InsertDestination.pb.h"
-#include "storage/PackedRowStoreValueAccessor.hpp"
 #include "storage/SplitRowStoreValueAccessor.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index d09ff07..99fafa8 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -47,7 +47,6 @@
 #include "storage/CountedReference.hpp"
 #include "storage/InsertDestination.hpp"
 #include "storage/InsertDestination.pb.h"
-#include "storage/PackedRowStoreValueAccessor.hpp"
 #include "storage/SplitRowStoreValueAccessor.hpp"
 #include "storage/StorageBlock.hpp"
 #include "storage/StorageBlockInfo.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/PackedRowStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreTupleStorageSubBlock.cpp b/storage/PackedRowStoreTupleStorageSubBlock.cpp
deleted file mode 100644
index 0ad4a4c..0000000
--- a/storage/PackedRowStoreTupleStorageSubBlock.cpp
+++ /dev/null
@@ -1,491 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#include "storage/PackedRowStoreTupleStorageSubBlock.hpp"
-
-#include <cstddef>
-#include <cstring>
-#include <vector>
-
-#include "catalog/CatalogAttribute.hpp"
-#include "catalog/CatalogRelationSchema.hpp"
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/PackedRowStoreValueAccessor.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/StorageBlockLayout.pb.h"
-#include "storage/StorageErrors.hpp"
-#include "storage/SubBlockTypeRegistry.hpp"
-#include "storage/TupleIdSequence.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
-#include "types/Type.hpp"
-#include "types/TypedValue.hpp"
-#include "types/containers/Tuple.hpp"
-#include "utility/BitVector.hpp"
-#include "utility/Macros.hpp"
-
-using std::vector;
-using std::memcpy;
-using std::size_t;
-
-namespace quickstep {
-
-QUICKSTEP_REGISTER_TUPLE_STORE(PackedRowStoreTupleStorageSubBlock, PACKED_ROW_STORE);
-
-PackedRowStoreTupleStorageSubBlock::PackedRowStoreTupleStorageSubBlock(
-    const CatalogRelationSchema &relation,
-    const TupleStorageSubBlockDescription &description,
-    const bool new_block,
-    void *sub_block_memory,
-    const std::size_t sub_block_memory_size)
-    : TupleStorageSubBlock(relation,
-                           description,
-                           new_block,
-                           sub_block_memory,
-                           sub_block_memory_size),
-      header_(static_cast<PackedRowStoreHeader*>(sub_block_memory)),
-      null_bitmap_bytes_(0) {
-  if (!DescriptionIsValid(relation_, description_)) {
-    FATAL_ERROR("Attempted to construct a PackedRowStoreTupleStorageSubBlock from an invalid description.");
-  }
-
-  if (sub_block_memory_size < sizeof(PackedRowStoreHeader)) {
-    throw BlockMemoryTooSmall("PackedRowStoreTupleStorageSubBlock", sub_block_memory_size);
-  }
-
-  if (relation_.hasNullableAttributes()) {
-    // Compute on the order of bits to account for bits in null_bitmap_.
-    tuple_id row_capacity = ((sub_block_memory_size_ - sizeof(PackedRowStoreHeader)) << 3)
-                            / ((relation.getFixedByteLength() << 3) + relation.numNullableAttributes());
-    null_bitmap_bytes_ = BitVector<false>::BytesNeeded(row_capacity * relation.numNullableAttributes());
-
-    if (sub_block_memory_size < sizeof(PackedRowStoreHeader) + null_bitmap_bytes_) {
-      if (relation_.getFixedByteLength() == 0) {
-        // Special case: relation consists entirely of NullType attributes.
-        row_capacity = BitVector<false>::MaxCapacityForBytes(
-                           sub_block_memory_size - sizeof(PackedRowStoreHeader))
-                       / relation.numNullableAttributes();
-        null_bitmap_bytes_ = sub_block_memory_size - sizeof(PackedRowStoreHeader);
-      } else {
-        throw BlockMemoryTooSmall("PackedRowStoreTupleStorageSubBlock", sub_block_memory_size);
-      }
-    }
-
-    null_bitmap_.reset(new BitVector<false>(static_cast<char*>(sub_block_memory_)
-                                                + sizeof(PackedRowStoreHeader),
-                                            row_capacity * relation.numNullableAttributes()));
-    tuple_storage_ = static_cast<char*>(sub_block_memory_)
-                         + sizeof(PackedRowStoreHeader)
-                         + null_bitmap_bytes_;
-  } else {
-    tuple_storage_ = static_cast<char*>(sub_block_memory_)
-                         + sizeof(PackedRowStoreHeader);
-  }
-
-  if (new_block) {
-    header_->num_tuples = 0;
-    if (relation_.hasNullableAttributes()) {
-      null_bitmap_->clear();
-    }
-  }
-}
-
-bool PackedRowStoreTupleStorageSubBlock::DescriptionIsValid(
-    const CatalogRelationSchema &relation,
-    const TupleStorageSubBlockDescription &description) {
-  // Make sure description is initialized and specifies PackedRowStore.
-  if (!description.IsInitialized()) {
-    return false;
-  }
-  if (description.sub_block_type() != TupleStorageSubBlockDescription::PACKED_ROW_STORE) {
-    return false;
-  }
-
-  // Make sure relation is not variable-length.
-  if (relation.isVariableLength()) {
-    return false;
-  }
-
-  return true;
-}
-
-std::size_t PackedRowStoreTupleStorageSubBlock::EstimateBytesPerTuple(
-    const CatalogRelationSchema &relation,
-    const TupleStorageSubBlockDescription &description) {
-  DEBUG_ASSERT(DescriptionIsValid(relation, description));
-
-  // NOTE(chasseur): We round-up the number of bytes needed in the NULL bitmap
-  // to avoid estimating 0 bytes needed for a relation with less than 8
-  // attributes which are all NullType.
-  return relation.getFixedByteLength()
-         + ((relation.numNullableAttributes() + 7) >> 3);
-}
-
-tuple_id PackedRowStoreTupleStorageSubBlock::bulkInsertTuples(ValueAccessor *accessor) {
-  const tuple_id original_num_tuples = header_->num_tuples;
-  char *dest_addr = static_cast<char*>(tuple_storage_)
-                      + header_->num_tuples * relation_.getFixedByteLength();
-  const unsigned num_nullable_attrs = relation_.numNullableAttributes();
-
-  InvokeOnAnyValueAccessor(
-      accessor,
-      [this, &dest_addr, &num_nullable_attrs](auto *accessor) -> void {  // NOLINT(build/c++11)
-    const std::size_t num_attrs = relation_.size();
-    const std::vector<std::size_t> &attrs_max_size =
-        relation_.getMaximumAttributeByteLengths();
-
-    if (num_nullable_attrs != 0) {
-      while (this->hasSpaceToInsert<true>(1) && accessor->next()) {
-        for (std::size_t curr_attr = 0; curr_attr < num_attrs; ++curr_attr) {
-          const std::size_t attr_size = attrs_max_size[curr_attr];
-          const attribute_id nullable_idx = relation_.getNullableAttributeIndex(curr_attr);
-          // If this attribute is nullable, check for a returned null value.
-          if (nullable_idx != kInvalidCatalogId) {
-            const void *attr_value
-                = accessor->template getUntypedValue<true>(curr_attr);
-            if (attr_value == nullptr) {
-              null_bitmap_->setBit(
-                  header_->num_tuples * num_nullable_attrs + nullable_idx,
-                  true);
-            } else {
-              memcpy(dest_addr, attr_value, attr_size);
-            }
-          } else {
-            memcpy(dest_addr,
-                   accessor->template getUntypedValue<false>(curr_attr),
-                   attr_size);
-          }
-          dest_addr += attr_size;
-        }
-        ++(header_->num_tuples);
-      }
-    } else {
-      // If the accessor is from a packed row store, we can optimize the
-      // memcpy by avoiding iterating over each attribute.
-      const bool fast_copy =
-          (accessor->getImplementationType() ==
-              ValueAccessor::Implementation::kCompressedPackedRowStore);
-      const std::size_t attrs_total_size = relation_.getMaximumByteLength();
-      while (this->hasSpaceToInsert<false>(1) && accessor->next()) {
-        if (fast_copy) {
-          memcpy(dest_addr,
-                 accessor->template getUntypedValue<false>(0),
-                 attrs_total_size);
-        } else {
-          for (std::size_t curr_attr = 0; curr_attr < num_attrs; ++curr_attr) {
-            const std::size_t attr_size = attrs_max_size[curr_attr];
-            memcpy(dest_addr,
-                   accessor->template getUntypedValue<false>(curr_attr),
-                   attr_size);
-            dest_addr += attr_size;
-          }
-        }
-        ++(header_->num_tuples);
-      }
-    }
-  });
-
-  return header_->num_tuples - original_num_tuples;
-}
-
-tuple_id PackedRowStoreTupleStorageSubBlock::bulkInsertTuplesWithRemappedAttributes(
-    const std::vector<attribute_id> &attribute_map,
-    ValueAccessor *accessor) {
-  DEBUG_ASSERT(attribute_map.size() == relation_.size());
-
-  const tuple_id original_num_tuples = header_->num_tuples;
-  char *dest_addr = static_cast<char*>(tuple_storage_)
-                      + header_->num_tuples * relation_.getFixedByteLength();
-  const unsigned num_nullable_attrs = relation_.numNullableAttributes();
-
-  InvokeOnAnyValueAccessor(
-      accessor,
-      [this, &num_nullable_attrs, &attribute_map, &dest_addr](auto *accessor) -> void {  // NOLINT(build/c++11)
-    const std::size_t num_attrs = relation_.size();
-    const std::vector<std::size_t> &attrs_max_size =
-        relation_.getMaximumAttributeByteLengths();
-
-    if (num_nullable_attrs != 0) {
-      while (this->hasSpaceToInsert<true>(1) && accessor->next()) {
-        for (std::size_t curr_attr = 0; curr_attr < num_attrs; ++curr_attr) {
-          const std::size_t attr_size = attrs_max_size[curr_attr];
-          const attribute_id nullable_idx = relation_.getNullableAttributeIndex(curr_attr);
-          // If this attribute is nullable, check for a returned null value.
-          if (nullable_idx != kInvalidCatalogId) {
-            const void *attr_value
-                = accessor->template getUntypedValue<true>(attribute_map[curr_attr]);
-            if (attr_value == nullptr) {
-              null_bitmap_->setBit(
-                  header_->num_tuples * num_nullable_attrs + nullable_idx,
-                  true);
-            } else {
-              memcpy(dest_addr, attr_value, attr_size);
-            }
-          } else {
-            memcpy(dest_addr,
-                   accessor->template getUntypedValue<false>(attribute_map[curr_attr]),
-                   attr_size);
-          }
-          dest_addr += attr_size;
-        }
-        ++(header_->num_tuples);
-      }
-    } else {
-      while (this->hasSpaceToInsert<false>(1) && accessor->next()) {
-        for (std::size_t curr_attr = 0; curr_attr < num_attrs; ++curr_attr) {
-          const std::size_t attr_size = attrs_max_size[curr_attr];
-          memcpy(dest_addr,
-                 accessor->template getUntypedValue<false>(attribute_map[curr_attr]),
-                 attr_size);
-          dest_addr += attr_size;
-        }
-        ++(header_->num_tuples);
-      }
-    }
-  });
-
-  return header_->num_tuples - original_num_tuples;
-}
-
-const void* PackedRowStoreTupleStorageSubBlock::getAttributeValue(
-    const tuple_id tuple,
-    const attribute_id attr) const {
-  DEBUG_ASSERT(hasTupleWithID(tuple));
-  DEBUG_ASSERT(relation_.hasAttributeWithId(attr));
-
-  const int nullable_idx = relation_.getNullableAttributeIndex(attr);
-  if ((nullable_idx != -1)
-      && null_bitmap_->getBit(tuple * relation_.numNullableAttributes() + nullable_idx)) {
-    return nullptr;
-  }
-
-  return static_cast<char*>(tuple_storage_)                // Start of actual tuple storage.
-         + (tuple * relation_.getFixedByteLength())        // Tuples prior to 'tuple'.
-         + relation_.getFixedLengthAttributeOffset(attr);  // Attribute offset within tuple.
-}
-
-TypedValue PackedRowStoreTupleStorageSubBlock::getAttributeValueTyped(
-    const tuple_id tuple,
-    const attribute_id attr) const {
-  const Type &attr_type = relation_.getAttributeById(attr)->getType();
-  const void *untyped_value = getAttributeValue(tuple, attr);
-  return (untyped_value == nullptr)
-      ? attr_type.makeNullValue()
-      : attr_type.makeValue(untyped_value, attr_type.maximumByteLength());
-}
-
-ValueAccessor* PackedRowStoreTupleStorageSubBlock::createValueAccessor(
-    const TupleIdSequence *sequence) const {
-  PackedRowStoreValueAccessor *base_accessor
-      = new PackedRowStoreValueAccessor(relation_,
-                                        relation_,
-                                        header_->num_tuples,
-                                        tuple_storage_,
-                                        null_bitmap_.get());
-  if (sequence == nullptr) {
-    return base_accessor;
-  } else {
-    return new TupleIdSequenceAdapterValueAccessor<PackedRowStoreValueAccessor>(
-        base_accessor,
-        *sequence);
-  }
-}
-
-void PackedRowStoreTupleStorageSubBlock::setAttributeValueInPlaceTyped(const tuple_id tuple,
-                                                                       const attribute_id attr,
-                                                                       const TypedValue &value) {
-  DEBUG_ASSERT(hasTupleWithID(tuple));
-  DEBUG_ASSERT(relation_.hasAttributeWithId(attr));
-  DEBUG_ASSERT(value.isPlausibleInstanceOf(relation_.getAttributeById(attr)->getType().getSignature()));
-
-  const int nullable_idx = relation_.getNullableAttributeIndex(attr);
-  if (nullable_idx != -1) {
-    if (value.isNull()) {
-      null_bitmap_->setBit(tuple * relation_.numNullableAttributes() + nullable_idx, true);
-      return;
-    } else {
-      null_bitmap_->setBit(tuple * relation_.numNullableAttributes() + nullable_idx, false);
-    }
-  }
-
-  char *base_addr = static_cast<char*>(tuple_storage_)                // Start of actual tuple storage.
-                    + (tuple * relation_.getFixedByteLength())        // Tuples prior to 'tuple'.
-                    + relation_.getFixedLengthAttributeOffset(attr);  // Attribute offset within tuple.
-
-  value.copyInto(base_addr);
-}
-
-bool PackedRowStoreTupleStorageSubBlock::deleteTuple(const tuple_id tuple) {
-  DEBUG_ASSERT(hasTupleWithID(tuple));
-
-  if (tuple == header_->num_tuples - 1) {
-    // If deleting the last tuple, simply truncate.
-    --(header_->num_tuples);
-    if (null_bitmap_.get() != nullptr) {
-      null_bitmap_->setBitRange(tuple * relation_.numNullableAttributes(),
-                                relation_.numNullableAttributes(),
-                                false);
-    }
-    return false;
-  } else {
-    const size_t tuple_length = relation_.getFixedByteLength();
-
-    char *dest_addr = static_cast<char*>(tuple_storage_)  // Start of actual tuple storage.
-                      + (tuple * tuple_length);           // Prior tuples.
-    char *src_addr = dest_addr + tuple_length;  // Start of subsequent tuples.
-    const size_t copy_bytes = (header_->num_tuples - tuple - 1) * tuple_length;  // Bytes in subsequent tuples.
-    memmove(dest_addr, src_addr, copy_bytes);
-
-    if (null_bitmap_.get() != nullptr) {
-      null_bitmap_->shiftTailForward(tuple * relation_.numNullableAttributes(),
-                                     relation_.numNullableAttributes());
-    }
-
-    --(header_->num_tuples);
-
-    return true;
-  }
-}
-
-bool PackedRowStoreTupleStorageSubBlock::bulkDeleteTuples(TupleIdSequence *tuples) {
-  if (tuples->empty()) {
-    // Nothing to do.
-    return false;
-  }
-
-  const tuple_id front = tuples->front();
-  const tuple_id back = tuples->back();
-  const tuple_id num_tuples = tuples->numTuples();
-
-  if ((back == header_->num_tuples - 1)
-       && (back - front == num_tuples - 1)) {
-    // Just truncate the back.
-    header_->num_tuples = front;
-    if (null_bitmap_.get() != nullptr) {
-      null_bitmap_->setBitRange(header_->num_tuples * relation_.numNullableAttributes(),
-                                num_tuples * relation_.numNullableAttributes(),
-                                false);
-    }
-    return false;
-  }
-
-  // Pack the non-deleted tuples.
-  const size_t tuple_length = relation_.getFixedByteLength();
-  tuple_id dest_tid = front;
-  tuple_id src_tid = dest_tid;
-
-  TupleIdSequence::const_iterator it = tuples->begin();
-  for (tuple_id current_id = front;
-       current_id < header_->num_tuples;
-       ++current_id, ++src_tid) {
-    if (current_id == *it) {
-      // Don't copy a deleted tuple.
-
-      if (null_bitmap_.get() != nullptr) {
-        // Erase the deleted tuple's entries in the null bitmap.
-        null_bitmap_->shiftTailForward(dest_tid * relation_.numNullableAttributes(),
-                                       relation_.numNullableAttributes());
-      }
-
-      ++it;
-      if (it == tuples->end()) {
-        // No more to delete, so copy all the remaining tuples in one go.
-        memmove(static_cast<char*>(tuple_storage_) + dest_tid * tuple_length,
-                static_cast<char*>(tuple_storage_) + (src_tid + 1) * tuple_length,
-                (header_->num_tuples - back - 1) * tuple_length);
-        break;
-      }
-    } else {
-      // Copy the next tuple to the packed region.
-      memmove(static_cast<char*>(tuple_storage_) + dest_tid * tuple_length,
-              static_cast<char*>(tuple_storage_) + src_tid * tuple_length,
-              tuple_length);
-      ++dest_tid;
-    }
-  }
-
-  header_->num_tuples -= static_cast<tuple_id>(num_tuples);
-
-  return true;
-}
-
-template <bool nullable_attrs>
-bool PackedRowStoreTupleStorageSubBlock::hasSpaceToInsert(const tuple_id num_tuples) const {
-  if (sizeof(PackedRowStoreHeader)
-          + null_bitmap_bytes_
-          + (header_->num_tuples + num_tuples) * relation_.getFixedByteLength()
-      <= sub_block_memory_size_) {
-    if (nullable_attrs) {
-      return static_cast<std::size_t>(header_->num_tuples + num_tuples) < null_bitmap_->size();
-    } else {
-      return true;
-    }
-  } else {
-    return false;
-  }
-}
-
-// Make sure both versions get compiled in.
-template bool PackedRowStoreTupleStorageSubBlock::hasSpaceToInsert<false>(
-    const tuple_id num_tuples) const;
-template bool PackedRowStoreTupleStorageSubBlock::hasSpaceToInsert<true>(
-    const tuple_id num_tuples) const;
-
-template <bool nullable_attrs>
-TupleStorageSubBlock::InsertResult PackedRowStoreTupleStorageSubBlock::insertTupleImpl(
-    const Tuple &tuple) {
-#ifdef QUICKSTEP_DEBUG
-  paranoidInsertTypeCheck(tuple);
-#endif
-  if (!hasSpaceToInsert<nullable_attrs>(1)) {
-    return InsertResult(-1, false);
-  }
-
-  char *base_addr = static_cast<char*>(tuple_storage_)                       // Start of actual tuple-storage region.
-                    + header_->num_tuples * relation_.getFixedByteLength();  // Existing tuples.
-
-  Tuple::const_iterator value_it = tuple.begin();
-  CatalogRelationSchema::const_iterator attr_it = relation_.begin();
-
-  while (value_it != tuple.end()) {
-    if (nullable_attrs) {
-      const int nullable_idx = relation_.getNullableAttributeIndex(attr_it->getID());
-      if ((nullable_idx != -1) && value_it->isNull()) {
-        null_bitmap_->setBit(header_->num_tuples * relation_.numNullableAttributes()
-                                 + nullable_idx,
-                             true);
-      } else {
-        value_it->copyInto(base_addr);
-      }
-    } else {
-      value_it->copyInto(base_addr);
-    }
-
-    base_addr += attr_it->getType().maximumByteLength();
-
-    ++value_it;
-    ++attr_it;
-  }
-
-  ++(header_->num_tuples);
-
-  return InsertResult(header_->num_tuples - 1, false);
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/PackedRowStoreTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreTupleStorageSubBlock.hpp b/storage/PackedRowStoreTupleStorageSubBlock.hpp
deleted file mode 100644
index 0cd41f4..0000000
--- a/storage/PackedRowStoreTupleStorageSubBlock.hpp
+++ /dev/null
@@ -1,216 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#ifndef QUICKSTEP_STORAGE_PACKED_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_
-#define QUICKSTEP_STORAGE_PACKED_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_
-
-#include <memory>
-#include <unordered_map>
-#include <vector>
-
-#include "expressions/predicate/PredicateCost.hpp"
-#include "storage/SubBlockTypeRegistryMacros.hpp"
-#include "storage/TupleStorageSubBlock.hpp"
-#include "types/TypedValue.hpp"
-#include "utility/BitVector.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-class CatalogRelationSchema;
-class ComparisonPredicate;
-class TupleStorageSubBlockDescription;
-class ValueAccessor;
-
-QUICKSTEP_DECLARE_SUB_BLOCK_TYPE_REGISTERED(PackedRowStoreTupleStorageSubBlock);
-
-/** \addtogroup Storage
- *  @{
- */
-
-/**
- * @brief An implementation of TupleStorageSubBlock as a packed row-store (i.e.
- *        an array of fixed-length values with no holes).
- * @warning This implementation does NOT support variable-length attributes. It
- *          is an error to attempt to construct a
- *          PackedRowStoreTupleStorageSubBlock for a relation with any
- *          variable-length attributes.
- **/
-class PackedRowStoreTupleStorageSubBlock: public TupleStorageSubBlock {
- public:
-  PackedRowStoreTupleStorageSubBlock(const CatalogRelationSchema &relation,
-                                     const TupleStorageSubBlockDescription &description,
-                                     const bool new_block,
-                                     void *sub_block_memory,
-                                     const std::size_t sub_block_memory_size);
-
-  ~PackedRowStoreTupleStorageSubBlock() override {
-  }
-
-  /**
-   * @brief Determine whether a TupleStorageSubBlockDescription is valid for
-   *        this type of TupleStorageSubBlock.
-   *
-   * @param relation The relation a tuple store described by description would
-   *        belong to.
-   * @param description A description of the parameters for this type of
-   *        TupleStorageSubBlock, which will be checked for validity.
-   * @return Whether description is well-formed and valid for this type of
-   *         TupleStorageSubBlock belonging to relation (i.e. whether a
-   *         TupleStorageSubBlock of this type, belonging to relation, can be
-   *         constructed according to description).
-   **/
-  static bool DescriptionIsValid(const CatalogRelationSchema &relation,
-                                 const TupleStorageSubBlockDescription &description);
-
-  /**
-   * @brief Estimate the average number of bytes (including any applicable
-   *        overhead) used to store a single tuple in this type of
-   *        TupleStorageSubBlock. Used by StorageBlockLayout::finalize() to
-   *        divide block memory amongst sub-blocks.
-   * @warning description must be valid. DescriptionIsValid() should be called
-   *          first if necessary.
-   *
-   * @param relation The relation tuples belong to.
-   * @param description A description of the parameters for this type of
-   *        TupleStorageSubBlock.
-   * @return The average/ammortized number of bytes used to store a single
-   *         tuple of relation in a TupleStorageSubBlock of this type described
-   *         by description.
-   **/
-  static std::size_t EstimateBytesPerTuple(const CatalogRelationSchema &relation,
-                                           const TupleStorageSubBlockDescription &description);
-
-  bool supportsUntypedGetAttributeValue(const attribute_id attr) const override {
-    return true;
-  }
-
-  bool supportsAdHocInsert() const override {
-    return true;
-  }
-
-  bool adHocInsertIsEfficient() const override {
-    return true;
-  }
-
-  TupleStorageSubBlockType getTupleStorageSubBlockType() const override {
-    return kPackedRowStore;
-  }
-
-  bool isEmpty() const override {
-    return (header_->num_tuples == 0);
-  }
-
-  bool isPacked() const override {
-    return true;
-  }
-
-  tuple_id getMaxTupleID() const override {
-    return header_->num_tuples - 1;
-  }
-
-  bool hasTupleWithID(const tuple_id tuple) const override {
-    return ((tuple >=0) && (tuple < header_->num_tuples));
-  }
-
-  InsertResult insertTuple(const Tuple &tuple) override {
-    if (null_bitmap_.get() == nullptr) {
-      return insertTupleImpl<false>(tuple);
-    } else {
-      return insertTupleImpl<true>(tuple);
-    }
-  }
-
-  inline bool insertTupleInBatch(const Tuple &tuple) override {
-    const InsertResult result = insertTuple(tuple);
-    return (result.inserted_id >= 0);
-  }
-
-  tuple_id bulkInsertTuples(ValueAccessor *accessor) override;
-
-  tuple_id bulkInsertTuplesWithRemappedAttributes(
-      const std::vector<attribute_id> &attribute_map,
-      ValueAccessor *accessor) override;
-
-  const void* getAttributeValue(const tuple_id tuple,
-                                const attribute_id attr) const override;
-
-  TypedValue getAttributeValueTyped(const tuple_id tuple,
-                                    const attribute_id attr) const override;
-
-  ValueAccessor* createValueAccessor(
-      const TupleIdSequence *sequence = nullptr) const override;
-
-  bool canSetAttributeValuesInPlaceTyped(
-      const tuple_id tuple,
-      const std::unordered_map<attribute_id, TypedValue> &new_values) const override {
-    return true;
-  }
-
-  void setAttributeValueInPlaceTyped(const tuple_id tuple,
-                                     const attribute_id attr,
-                                     const TypedValue &value) override;
-
-  bool deleteTuple(const tuple_id tuple) override;
-  bool bulkDeleteTuples(TupleIdSequence *tuples) override;
-
-  predicate_cost_t estimatePredicateEvaluationCost(
-      const ComparisonPredicate &predicate) const override {
-    return predicate_cost::kRowScan;
-  }
-
-  void rebuild() override {
-  }
-
-  bool isInsertOrderPreserving() const override {
-    return true;
-  }
-
- private:
-  struct PackedRowStoreHeader {
-    tuple_id num_tuples;
-  };
-
-  // If 'nullable_attrs' is true, extra branches are enabled to deal with NULLs
-  // and set bits in '*null_bitmap_' appropriately. If no attributes of the
-  // relation are nullable, then 'nullable_attrs' is false and this function
-  // has a simpler and faster implementation with no NULL-checking.
-  template <bool nullable_attrs>
-  InsertResult insertTupleImpl(const Tuple &tuple);
-
-  // Similar to insertTupleImpl(), the 'nullable_attrs' template parameter
-  // distinguishes between a version of this function that includes a check for
-  // space in '*null_bitmap_' and a simpler version that skips the check for
-  // relations that have no nullable attributes (and therefore no NULL-bitmap).
-  template <bool nullable_attrs>
-  bool hasSpaceToInsert(const tuple_id num_tuples) const;
-
-  PackedRowStoreHeader *header_;
-  std::unique_ptr<BitVector<false>> null_bitmap_;
-  std::size_t null_bitmap_bytes_;
-  void *tuple_storage_;
-
-  DISALLOW_COPY_AND_ASSIGN(PackedRowStoreTupleStorageSubBlock);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_STORAGE_PACKED_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/PackedRowStoreValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedRowStoreValueAccessor.hpp b/storage/PackedRowStoreValueAccessor.hpp
deleted file mode 100644
index 9d43955..0000000
--- a/storage/PackedRowStoreValueAccessor.hpp
+++ /dev/null
@@ -1,150 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#ifndef QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_
-#define QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_
-
-#include "catalog/CatalogRelationSchema.hpp"
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "types/Type.hpp"
-#include "types/TypedValue.hpp"
-#include "utility/BitVector.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-class PackedRowStoreTupleStorageSubBlock;
-
-class PackedRowStoreValueAccessorHelper {
- public:
-  PackedRowStoreValueAccessorHelper(const CatalogRelationSchema &relation,
-                                    const tuple_id num_tuples,
-                                    const void *tuple_storage,
-                                    const BitVector<false> *null_bitmap)
-      : relation_(relation),
-        num_tuples_(num_tuples),
-        tuple_storage_(tuple_storage),
-        null_bitmap_(null_bitmap) {
-  }
-
-  inline tuple_id numPackedTuples() const {
-    return num_tuples_;
-  }
-
-  /**
-   * @brief Returns whether this accessor has a fast strided ColumnAccessor available
-   *        that can be used to optimize memory access in a tight loop iteration
-   *        over the underlying storage block.
-   *
-   * @return true if fast ColumnAccessor is supported, otherwise false.
-   */
-  inline bool isColumnAccessorSupported() const {
-    return true;
-  }
-
-  /**
-   * @brief Get a pointer to a ColumnAccessor object that provides a fast strided memory
-   *        access on the underlying storage block.
-   * @note The ownership of the returned object lies with the caller.
-   * @warning This method should only be called if isColumnAccessorSupported() method
-   *          returned true. If ColumnAccessor is not supported this method will return a nullptr.
-   *
-   * @param current_tuple_position A constant reference to the tuple position in the containing
-   *        ValueAccessor. This reference value is shared between the containing ValueAccessor &
-   *        a ColumnAccessor. However, a ColumnAccessor *CANNOT* modify this tuple position.
-   * @param attr_id The attribute id on which this ColumnAccessor will be created.
-   *
-   * @return A pointer to a ColumnAccessor object with specific properties set that can be used
-   *         in a tight loop iterations over the underlying storage block.
-   **/
-  template <bool check_null = true>
-  inline const ColumnAccessor<check_null>* getColumnAccessor(const tuple_id &current_tuple_position,
-                                                             const attribute_id attr_id) const {
-    DCHECK(relation_.hasAttributeWithId(attr_id));
-    const void* base_location = static_cast<const char*>(tuple_storage_)
-        + relation_.getFixedLengthAttributeOffset(attr_id);
-    const std::size_t stride = relation_.getFixedByteLength();
-
-    std::unique_ptr<ColumnAccessor<check_null>> column_accessor;
-    if (check_null) {
-      const int nullable_base = relation_.getNullableAttributeIndex(attr_id);
-      const unsigned nullable_stride = relation_.numNullableAttributes();
-      column_accessor.reset(new ColumnAccessor<check_null>(current_tuple_position,
-                                                           num_tuples_,
-                                                           base_location,
-                                                           stride,
-                                                           null_bitmap_,
-                                                           nullable_base,
-                                                           nullable_stride));
-    } else {
-      column_accessor.reset(new ColumnAccessor<check_null>(current_tuple_position,
-                                                           num_tuples_,
-                                                           base_location,
-                                                           stride));
-    }
-    return column_accessor.release();
-  }
-
-  template <bool check_null>
-  inline const void* getAttributeValue(const tuple_id tuple,
-                                       const attribute_id attr) const {
-    DEBUG_ASSERT(tuple < num_tuples_);
-    DEBUG_ASSERT(relation_.hasAttributeWithId(attr));
-    if (check_null) {
-      const int nullable_idx = relation_.getNullableAttributeIndex(attr);
-      if ((nullable_idx != -1)
-          && null_bitmap_->getBit(tuple * relation_.numNullableAttributes() + nullable_idx)) {
-        return nullptr;
-      }
-    }
-
-    return static_cast<const char*>(tuple_storage_)          // Start of actual tuple storage.
-           + (tuple * relation_.getFixedByteLength())        // Tuples prior to 'tuple'.
-           + relation_.getFixedLengthAttributeOffset(attr);  // Attribute offset within tuple.
-  }
-
-  inline TypedValue getAttributeValueTyped(const tuple_id tuple,
-                                           const attribute_id attr) const {
-    const Type &attr_type = relation_.getAttributeById(attr)->getType();
-    const void *untyped_value = getAttributeValue<true>(tuple, attr);
-    return (untyped_value == nullptr)
-        ? attr_type.makeNullValue()
-        : attr_type.makeValue(untyped_value, attr_type.maximumByteLength());
-  }
-
- private:
-  const CatalogRelationSchema &relation_;
-  const tuple_id num_tuples_;
-  const void *tuple_storage_;
-  const BitVector<false> *null_bitmap_;
-
-  DISALLOW_COPY_AND_ASSIGN(PackedRowStoreValueAccessorHelper);
-};
-
-typedef PackedTupleStorageSubBlockValueAccessor<
-    PackedRowStoreTupleStorageSubBlock,
-    PackedRowStoreValueAccessorHelper,
-    ValueAccessor::Implementation::kPackedRowStore>
-        PackedRowStoreValueAccessor;
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_STORAGE_PACKED_ROW_STORE_VALUE_ACCESSOR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/SplitRowStoreTupleStorageSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreTupleStorageSubBlock.cpp b/storage/SplitRowStoreTupleStorageSubBlock.cpp
index 1e6f7ff..ad583eb 100644
--- a/storage/SplitRowStoreTupleStorageSubBlock.cpp
+++ b/storage/SplitRowStoreTupleStorageSubBlock.cpp
@@ -245,9 +245,7 @@ tuple_id SplitRowStoreTupleStorageSubBlock::bulkInsertDispatcher(
   CopyGroupList copy_groups;
   getCopyGroupsForAttributeMap(attribute_map, &copy_groups);
   auto impl = accessor->getImplementationType();
-  const bool is_rowstore_source =
-    (impl == ValueAccessor::Implementation::kPackedRowStore ||
-     impl == ValueAccessor::Implementation::kSplitRowStore);
+  const bool is_rowstore_source = impl == ValueAccessor::Implementation::kSplitRowStore;
   if (is_rowstore_source) {
     copy_groups.merge_contiguous();
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/SplitRowStoreTupleStorageSubBlock.hpp
----------------------------------------------------------------------
diff --git a/storage/SplitRowStoreTupleStorageSubBlock.hpp b/storage/SplitRowStoreTupleStorageSubBlock.hpp
index 89c756d..67d36fe 100644
--- a/storage/SplitRowStoreTupleStorageSubBlock.hpp
+++ b/storage/SplitRowStoreTupleStorageSubBlock.hpp
@@ -421,4 +421,4 @@ class SplitRowStoreTupleStorageSubBlock: public TupleStorageSubBlock {
 
 }  // namespace quickstep
 
-#endif  // QUICKSTEP_STORAGE_PACKED_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_
+#endif  // QUICKSTEP_STORAGE_SPLIT_ROW_STORE_TUPLE_STORAGE_SUB_BLOCK_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index 6267d6b..de2d25b 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -40,7 +40,6 @@
 #include "storage/HashTableBase.hpp"
 #include "storage/IndexSubBlock.hpp"
 #include "storage/InsertDestinationInterface.hpp"
-#include "storage/PackedRowStoreTupleStorageSubBlock.hpp"
 #include "storage/SMAIndexSubBlock.hpp"
 #include "storage/SplitRowStoreTupleStorageSubBlock.hpp"
 #include "storage/StorageBlockBase.hpp"
@@ -958,12 +957,6 @@ TupleStorageSubBlock* StorageBlock::CreateTupleStorageSubBlock(
     const std::size_t sub_block_memory_size) {
   DEBUG_ASSERT(description.IsInitialized());
   switch (description.sub_block_type()) {
-    case TupleStorageSubBlockDescription::PACKED_ROW_STORE:
-      return new PackedRowStoreTupleStorageSubBlock(relation,
-                                                    description,
-                                                    new_block,
-                                                    sub_block_memory,
-                                                    sub_block_memory_size);
     case TupleStorageSubBlockDescription::BASIC_COLUMN_STORE:
       return new BasicColumnStoreTupleStorageSubBlock(relation,
                                                       description,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/StorageBlockInfo.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlockInfo.cpp b/storage/StorageBlockInfo.cpp
index 8c40be6..2646e5f 100644
--- a/storage/StorageBlockInfo.cpp
+++ b/storage/StorageBlockInfo.cpp
@@ -37,7 +37,6 @@ string BlockIdUtil::ToString(const block_id block) {
 }
 
 const char *kTupleStorageSubBlockTypeNames[] = {
-  "PackedRowStore",
   "BasicColumnStore",
   "CompressedPackedRowStore",
   "CompressedColumnStore",
@@ -46,7 +45,7 @@ const char *kTupleStorageSubBlockTypeNames[] = {
 
 const char *kIndexSubBlockTypeNames[] = {
   "CSBTree",
+  "SMA",
 };
 
 }  // namespace quickstep
-

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/StorageBlockInfo.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlockInfo.hpp b/storage/StorageBlockInfo.hpp
index 8d443d5..361648f 100644
--- a/storage/StorageBlockInfo.hpp
+++ b/storage/StorageBlockInfo.hpp
@@ -118,7 +118,6 @@ const tuple_id kMaxTupleID = INT_MAX;
  * @brief Codes for the different implementations of TupleStorageSubBlock.
  **/
 enum TupleStorageSubBlockType {
-  kPackedRowStore = 0,
   kBasicColumnStore,
   kCompressedPackedRowStore,
   kCompressedColumnStore,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/StorageBlockLayout.hpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlockLayout.hpp b/storage/StorageBlockLayout.hpp
index 9827a4c..30a00f5 100644
--- a/storage/StorageBlockLayout.hpp
+++ b/storage/StorageBlockLayout.hpp
@@ -73,9 +73,7 @@ class StorageBlockLayout {
    * @brief Static method to generate a default layout for a particular
    *        relation.
    * @note The current policy is that a default layout takes up one slot, uses
-   *       PackedRowStoreTupleStorageSubBlock for fixed-length relations or
-   *       SplitRowStoreTupleStorageSubBlock for variable-length relations, and
-   *       has no indices.
+   *       SplitRowStoreTupleStorageSubBlock and uses no indices.
    *
    * @param relation The relation to generate a layout for.
    * @param relation_variable_length Whether relation is variable-length.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/StorageBlockLayout.proto
----------------------------------------------------------------------
diff --git a/storage/StorageBlockLayout.proto b/storage/StorageBlockLayout.proto
index 6573470..8919505 100644
--- a/storage/StorageBlockLayout.proto
+++ b/storage/StorageBlockLayout.proto
@@ -22,11 +22,10 @@ package quickstep;
 // Options for TupleStorageSubBlocks.
 message TupleStorageSubBlockDescription {
   enum TupleStorageSubBlockType {
-    PACKED_ROW_STORE = 0;
-    BASIC_COLUMN_STORE = 1;
-    COMPRESSED_PACKED_ROW_STORE = 2;
-    COMPRESSED_COLUMN_STORE = 3;
-    SPLIT_ROW_STORE = 4;
+    BASIC_COLUMN_STORE = 0;
+    COMPRESSED_PACKED_ROW_STORE = 1;
+    COMPRESSED_COLUMN_STORE = 2;
+    SPLIT_ROW_STORE = 3;
   }
 
   required TupleStorageSubBlockType sub_block_type = 1;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/SubBlockTypeRegistry.hpp
----------------------------------------------------------------------
diff --git a/storage/SubBlockTypeRegistry.hpp b/storage/SubBlockTypeRegistry.hpp
index c362d9f..71e0629 100644
--- a/storage/SubBlockTypeRegistry.hpp
+++ b/storage/SubBlockTypeRegistry.hpp
@@ -47,7 +47,7 @@ class CatalogRelationSchema;
  *        (.cpp) file, and put an invocation of the
  *        QUICKSTEP_REGISTER_TUPLE_STORE() macro in the quickstep namespace.
  *        For example:
- *        QUICKSTEP_REGISTER_TUPLE_STORE(PackedRowStoreTupleStorageSubBlock, PACKED_ROW_STORE);
+ *        QUICKSTEP_REGISTER_TUPLE_STORE(SplitRowStoreTupleStorageSubBlock, SPLIT_ROW_STORE);
  *        The first argument to the macro is the name of the class, the second
  *        is the name of the class' corresponding case in the
  *        TupleStorageSubBlockDescription::TupleStorageSubBlockType enum.
@@ -55,7 +55,7 @@ class CatalogRelationSchema;
  *        file, and put an invocation of the
  *        QUICKSTEP_DECLARE_SUB_BLOCK_TYPE_REGISTERED() macro in the quickstep
  *        namespace like so:
- *        QUICKSTEP_DECLARE_SUB_BLOCK_TYPE_REGISTERED(PackedRowStoreTupleStorageSubBlock);
+ *        QUICKSTEP_DECLARE_SUB_BLOCK_TYPE_REGISTERED(SplitRowStoreTupleStorageSubBlock);
  *
  * Registration of IndexSubBlock implementations works the same way, except the
  * first macro used should be QUICKSTEP_REGISTER_INDEX() instead of

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/ValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessor.hpp b/storage/ValueAccessor.hpp
index e4a2906..654bbf9 100644
--- a/storage/ValueAccessor.hpp
+++ b/storage/ValueAccessor.hpp
@@ -78,7 +78,6 @@ class ValueAccessor {
     kBasicColumnStore,
     kCompressedColumnStore,
     kCompressedPackedRowStore,
-    kPackedRowStore,
     kSplitRowStore,
     // Accessor for a group of ColumnVectors:
     kColumnVectors

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/ValueAccessorUtil.hpp
----------------------------------------------------------------------
diff --git a/storage/ValueAccessorUtil.hpp b/storage/ValueAccessorUtil.hpp
index 8be3785..4969156 100644
--- a/storage/ValueAccessorUtil.hpp
+++ b/storage/ValueAccessorUtil.hpp
@@ -25,7 +25,6 @@
 #include "storage/BasicColumnStoreValueAccessor.hpp"
 #include "storage/CompressedColumnStoreValueAccessor.hpp"
 #include "storage/CompressedPackedRowStoreValueAccessor.hpp"
-#include "storage/PackedRowStoreValueAccessor.hpp"
 #include "storage/SplitRowStoreValueAccessor.hpp"
 #include "storage/ValueAccessor.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
@@ -70,8 +69,6 @@ auto InvokeOnValueAccessorNotAdapter(
       return functor(static_cast<CompressedColumnStoreValueAccessor*>(accessor));
     case ValueAccessor::Implementation::kCompressedPackedRowStore:
       return functor(static_cast<CompressedPackedRowStoreValueAccessor*>(accessor));
-    case ValueAccessor::Implementation::kPackedRowStore:
-      return functor(static_cast<PackedRowStoreValueAccessor*>(accessor));
     case ValueAccessor::Implementation::kSplitRowStore:
       return functor(static_cast<SplitRowStoreValueAccessor*>(accessor));
     case ValueAccessor::Implementation::kColumnVectors:
@@ -121,10 +118,6 @@ auto InvokeOnTupleIdSequenceAdapterValueAccessor(
       return functor(
           static_cast<TupleIdSequenceAdapterValueAccessor<CompressedPackedRowStoreValueAccessor>*>(
               accessor));
-    case ValueAccessor::Implementation::kPackedRowStore:
-      return functor(
-          static_cast<TupleIdSequenceAdapterValueAccessor<PackedRowStoreValueAccessor>*>(
-              accessor));
     case ValueAccessor::Implementation::kSplitRowStore:
       return functor(
           static_cast<TupleIdSequenceAdapterValueAccessor<SplitRowStoreValueAccessor>*>(
@@ -177,10 +170,6 @@ auto InvokeOnOrderedTupleIdSequenceAdapterValueAccessor(
       return functor(
           static_cast<OrderedTupleIdSequenceAdapterValueAccessor<CompressedPackedRowStoreValueAccessor>*>(
               accessor));
-    case ValueAccessor::Implementation::kPackedRowStore:
-      return functor(
-          static_cast<OrderedTupleIdSequenceAdapterValueAccessor<PackedRowStoreValueAccessor>*>(
-              accessor));
     case ValueAccessor::Implementation::kSplitRowStore:
       return functor(
           static_cast<OrderedTupleIdSequenceAdapterValueAccessor<SplitRowStoreValueAccessor>*>(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp b/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
index a511fcb..d41a457 100644
--- a/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
+++ b/storage/tests/BasicColumnStoreTupleStorageSubBlock_unittest.cpp
@@ -859,7 +859,7 @@ TEST_P(BasicColumnStoreTupleStorageSubBlockTest, DescriptionIsValidTest) {
 
   // A description that specifies the wrong sub_block_type is not valid.
   tuple_store_description_->set_sub_block_type(
-      TupleStorageSubBlockDescription::PACKED_ROW_STORE);
+      TupleStorageSubBlockDescription::SPLIT_ROW_STORE);
   EXPECT_FALSE(BasicColumnStoreTupleStorageSubBlock::DescriptionIsValid(
                    *relation_,
                    *tuple_store_description_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp b/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp
index 530507b..3feee5e 100644
--- a/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp
+++ b/storage/tests/CompressedPackedRowStoreTupleStorageSubBlock_unittest.cpp
@@ -679,7 +679,7 @@ TEST_P(CompressedPackedRowStoreTupleStorageSubBlockTest, DescriptionIsValidTest)
                                                                                 *tuple_store_description_));
 
   // A description that specifies the wrong sub_block_type is not valid.
-  tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::PACKED_ROW_STORE);
+  tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::SPLIT_ROW_STORE);
   EXPECT_FALSE(CompressedPackedRowStoreTupleStorageSubBlock::DescriptionIsValid(*relation_,
                                                                                 *tuple_store_description_));
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5fee8218/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp
----------------------------------------------------------------------
diff --git a/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp b/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp
deleted file mode 100644
index bf3c605..0000000
--- a/storage/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp
+++ /dev/null
@@ -1,584 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#include <cstring>
-#include <memory>
-#include <sstream>
-#include <string>
-#include <unordered_map>
-#include <utility>
-#include <vector>
-
-#include "gtest/gtest.h"
-
-#include "catalog/CatalogAttribute.hpp"
-#include "catalog/CatalogRelation.hpp"
-#include "storage/PackedRowStoreTupleStorageSubBlock.hpp"
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/StorageBlockLayout.hpp"
-#include "storage/StorageBlockLayout.pb.h"
-#include "storage/StorageConstants.hpp"
-#include "storage/StorageErrors.hpp"
-#include "storage/TupleIdSequence.hpp"
-#include "storage/TupleStorageSubBlock.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
-#include "types/CharType.hpp"
-#include "types/DoubleType.hpp"
-#include "types/IntType.hpp"
-#include "types/Type.hpp"
-#include "types/TypeFactory.hpp"
-#include "types/TypedValue.hpp"
-#include "types/TypeID.hpp"
-#include "types/containers/Tuple.hpp"
-#include "types/operations/comparisons/Comparison.hpp"
-#include "types/operations/comparisons/ComparisonFactory.hpp"
-#include "types/operations/comparisons/ComparisonID.hpp"
-#include "utility/BitVector.hpp"
-#include "utility/ScopedBuffer.hpp"
-
-using std::make_pair;
-using std::ostringstream;
-using std::pair;
-
-namespace quickstep {
-
-class PackedRowStoreTupleStorageSubBlockTest : public ::testing::TestWithParam<bool> {
- protected:
-  static const size_t kSubBlockSize = 0x100000;  // 1 MB
-  static const size_t kTupleLength = 24;
-
-  virtual void SetUp() {
-    // Create a sample relation with a variety of attribute types.
-    relation_.reset(new CatalogRelation(nullptr, "TestRelation"));
-
-    // An integer.
-    CatalogAttribute *current_attr = new CatalogAttribute(relation_.get(),
-                                                          "int_attr",
-                                                          TypeFactory::GetType(kInt, GetParam()));
-    ASSERT_EQ(0, relation_->addAttribute(current_attr));
-    eq_comp_int_.reset(ComparisonFactory::GetComparison(ComparisonID::kEqual).makeUncheckedComparatorForTypes(
-        current_attr->getType(),
-        current_attr->getType()));
-
-    // A double.
-    current_attr = new CatalogAttribute(relation_.get(),
-                                        "double_attr",
-                                        TypeFactory::GetType(kDouble, GetParam()));
-    ASSERT_EQ(1, relation_->addAttribute(current_attr));
-    eq_comp_double_.reset(ComparisonFactory::GetComparison(ComparisonID::kEqual).makeUncheckedComparatorForTypes(
-        current_attr->getType(),
-        current_attr->getType()));
-
-    // A string.
-    current_attr = new CatalogAttribute(relation_.get(),
-                                        "char_attr",
-                                        TypeFactory::GetType(kChar, 12, GetParam()));
-    ASSERT_EQ(2, relation_->addAttribute(current_attr));
-    eq_comp_char_.reset(ComparisonFactory::GetComparison(ComparisonID::kEqual).makeUncheckedComparatorForTypes(
-        current_attr->getType(),
-        current_attr->getType()));
-
-    tuple_store_description_.reset(new TupleStorageSubBlockDescription());
-    tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::PACKED_ROW_STORE);
-
-    // Don't initialize the block yet. Different tests will use different
-    // params.
-    tuple_store_memory_.reset();
-    tuple_store_.reset();
-  }
-
-  void initializeNewBlock(const size_t block_size) {
-    tuple_store_memory_.reset(block_size);
-    tuple_store_.reset(new PackedRowStoreTupleStorageSubBlock(*relation_,
-                                                              *tuple_store_description_,
-                                                              true,
-                                                              tuple_store_memory_.get(),
-                                                              block_size));
-  }
-
-  int computeRowCapacity() {
-    if (GetParam()) {
-      // Estimate using the same heuristic as
-      // PackedRowStoreTupleStorageSubBlock's constructor.
-      int row_capacity = ((kSubBlockSize - sizeof(tuple_id)) << 3)
-                         / ((kTupleLength << 3) + relation_->numNullableAttributes());
-      const size_t null_bitmap_bytes_ = BitVector<false>::BytesNeeded(
-          row_capacity * relation_->numNullableAttributes());
-      return static_cast<int>(kSubBlockSize - sizeof(tuple_id) - null_bitmap_bytes_) / kTupleLength;
-    } else {
-      return static_cast<int>(kSubBlockSize - sizeof(tuple_id)) / kTupleLength;
-    }
-  }
-
-  // Caller takes ownership of new heap-created Tuple.
-  Tuple* createSampleTuple(const int base_value) const {
-    std::vector<TypedValue> attrs;
-
-    // int_attr
-    if (GetParam() && (base_value % 6 == 0)) {
-      // Throw in a NULL integer for every sixth value.
-      attrs.emplace_back(kInt);
-    } else {
-      attrs.emplace_back(base_value);
-    }
-
-    // double_attr
-    if (GetParam() && (base_value % 6 == 2)) {
-      // NULL very sixth value.
-      attrs.emplace_back(kDouble);
-    } else {
-      attrs.emplace_back(static_cast<double>(0.25 * base_value));
-    }
-
-    // char_attr
-    if (GetParam() && (base_value % 6 == 4)) {
-      // NULL very sixth value.
-      attrs.emplace_back(CharType::InstanceNullable(12).makeNullValue());
-    } else {
-      ostringstream char_buffer;
-      char_buffer << base_value;
-      std::string string_literal(char_buffer.str());
-      attrs.emplace_back(CharType::InstanceNonNullable(12).makeValue(string_literal.c_str(),
-                                                                     string_literal.size() + 1));
-      attrs.back().ensureNotReference();
-    }
-
-    return new Tuple(std::move(attrs));
-  }
-
-  void fillBlockWithSampleData() {
-    tuple_id current_tid = 0;
-    std::unique_ptr<Tuple> current_tuple(createSampleTuple(current_tid));
-    while (tuple_store_->insertTupleInBatch(*current_tuple)) {
-      ++current_tid;
-      current_tuple.reset(createSampleTuple(current_tid));
-    }
-
-    tuple_store_->rebuild();
-  }
-
-  void checkTupleValuesUntyped(const tuple_id tid,
-                               const int base_value) {
-    ASSERT_TRUE(tuple_store_->hasTupleWithID(tid));
-    ASSERT_TRUE(tuple_store_->supportsUntypedGetAttributeValue(0));
-    ASSERT_TRUE(tuple_store_->supportsUntypedGetAttributeValue(1));
-    ASSERT_TRUE(tuple_store_->supportsUntypedGetAttributeValue(2));
-
-    std::unique_ptr<Tuple> comp_tuple(createSampleTuple(base_value));
-
-    if (comp_tuple->getAttributeValue(0).isNull()) {
-      assert(nullptr == tuple_store_->getAttributeValue(tid, 0));
-      // NULL comparisons are always false (for now).
-      EXPECT_FALSE(eq_comp_int_->compareDataPtrs(nullptr,
-                                                 tuple_store_->getAttributeValue(tid, 0)));
-    } else {
-      assert(eq_comp_int_->compareDataPtrs(comp_tuple->getAttributeValue(0).getDataPtr(),
-                                           tuple_store_->getAttributeValue(tid, 0)));
-    }
-
-    if (comp_tuple->getAttributeValue(1).isNull()) {
-      EXPECT_EQ(nullptr, tuple_store_->getAttributeValue(tid, 1));
-      // NULL comparisons are always false (for now).
-      EXPECT_FALSE(eq_comp_double_->compareDataPtrs(nullptr,
-                                                    tuple_store_->getAttributeValue(tid, 1)));
-    } else {
-      EXPECT_TRUE(eq_comp_double_->compareDataPtrs(comp_tuple->getAttributeValue(1).getDataPtr(),
-                                                   tuple_store_->getAttributeValue(tid, 1)));
-    }
-
-    if (comp_tuple->getAttributeValue(2).isNull()) {
-      EXPECT_EQ(nullptr, tuple_store_->getAttributeValue(tid, 2));
-      // NULL comparisons are always false (for now).
-      EXPECT_FALSE(eq_comp_char_->compareDataPtrs(nullptr,
-                                                  tuple_store_->getAttributeValue(tid, 2)));
-    } else {
-      EXPECT_TRUE(eq_comp_char_->compareDataPtrs(comp_tuple->getAttributeValue(2).getDataPtr(),
-                                                 tuple_store_->getAttributeValue(tid, 2)));
-    }
-  }
-
-  void checkTupleValuesTyped(const tuple_id tid,
-                             const int base_value) {
-    ASSERT_TRUE(tuple_store_->hasTupleWithID(tid));
-
-    std::unique_ptr<Tuple> comp_tuple(createSampleTuple(base_value));
-
-    if (comp_tuple->getAttributeValue(0).isNull()) {
-      EXPECT_TRUE(tuple_store_->getAttributeValueTyped(tid, 0).isNull());
-      // NULL comparisons are always false (for now).
-      EXPECT_FALSE(eq_comp_int_->compareTypedValues(comp_tuple->getAttributeValue(0),
-                                                    tuple_store_->getAttributeValueTyped(tid, 0)));
-    } else {
-      EXPECT_TRUE(eq_comp_int_->compareTypedValues(comp_tuple->getAttributeValue(0),
-                                                   tuple_store_->getAttributeValueTyped(tid, 0)));
-    }
-
-    if (comp_tuple->getAttributeValue(1).isNull()) {
-      EXPECT_TRUE(tuple_store_->getAttributeValueTyped(tid, 1).isNull());
-      // NULL comparisons are always false (for now).
-      EXPECT_FALSE(eq_comp_double_->compareTypedValues(comp_tuple->getAttributeValue(1),
-                                                       tuple_store_->getAttributeValueTyped(tid, 1)));
-    } else {
-      EXPECT_TRUE(eq_comp_double_->compareTypedValues(comp_tuple->getAttributeValue(1),
-                                                      tuple_store_->getAttributeValueTyped(tid, 1)));
-    }
-
-    if (comp_tuple->getAttributeValue(2).isNull()) {
-      EXPECT_TRUE(tuple_store_->getAttributeValueTyped(tid, 2).isNull());
-      // NULL comparisons are always false (for now).
-      EXPECT_FALSE(eq_comp_char_->compareTypedValues(comp_tuple->getAttributeValue(2),
-                                                     tuple_store_->getAttributeValueTyped(tid, 2)));
-    } else {
-      EXPECT_TRUE(eq_comp_char_->compareTypedValues(comp_tuple->getAttributeValue(2),
-                                                    tuple_store_->getAttributeValueTyped(tid, 2)));
-    }
-  }
-
-  template<bool check_null>
-  void checkColumnAccessor() {
-    initializeNewBlock(kSubBlockSize);
-    fillBlockWithSampleData();
-    ASSERT_TRUE(tuple_store_->isPacked());
-    std::unique_ptr<PackedRowStoreValueAccessor> accessor(
-      static_cast<PackedRowStoreValueAccessor*>(tuple_store_->createValueAccessor()));
-    attribute_id  value_accessor_id = 0;
-    tuple_id tid = 0;
-    accessor->beginIteration();
-    ASSERT_TRUE(accessor->isColumnAccessorSupported());
-    std::unique_ptr<const ColumnAccessor<check_null>>
-    column_accessor(accessor->template getColumnAccessor<check_null>(value_accessor_id));
-    ASSERT_TRUE(column_accessor != nullptr);
-    while (accessor->next()) {
-      const void *va_value = column_accessor->getUntypedValue();
-      std::unique_ptr<Tuple> expected_tuple(createSampleTuple(tid));
-
-      if (expected_tuple->getAttributeValue(value_accessor_id).isNull()) {
-        ASSERT_TRUE(va_value == nullptr);
-      } else {
-        ASSERT_TRUE(eq_comp_int_->compareDataPtrs(expected_tuple->getAttributeValue(value_accessor_id).getDataPtr(),
-                                                  va_value));
-      }
-      ++tid;
-    }
-  }
-
-  std::unique_ptr<CatalogRelation> relation_;
-  ScopedBuffer tuple_store_memory_;
-  std::unique_ptr<TupleStorageSubBlockDescription> tuple_store_description_;
-  std::unique_ptr<PackedRowStoreTupleStorageSubBlock> tuple_store_;
-
-  std::unique_ptr<UncheckedComparator> eq_comp_int_;
-  std::unique_ptr<UncheckedComparator> eq_comp_double_;
-  std::unique_ptr<UncheckedComparator> eq_comp_char_;
-};
-
-typedef PackedRowStoreTupleStorageSubBlockTest PackedRowStoreTupleStorageSubBlockDeathTest;
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, DescriptionIsValidTest) {
-  // The descriptions we use for the other tests should be valid.
-  EXPECT_TRUE(PackedRowStoreTupleStorageSubBlock::DescriptionIsValid(*relation_,
-                                                                     *tuple_store_description_));
-
-  // An uninitialized description is not valid.
-  tuple_store_description_.reset(new TupleStorageSubBlockDescription());
-  EXPECT_FALSE(PackedRowStoreTupleStorageSubBlock::DescriptionIsValid(*relation_,
-                                                                      *tuple_store_description_));
-
-  // A description that specifies the wrong sub_block_type is not valid.
-  tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::BASIC_COLUMN_STORE);
-  EXPECT_FALSE(PackedRowStoreTupleStorageSubBlock::DescriptionIsValid(*relation_,
-                                                                      *tuple_store_description_));
-
-  // A relation with a nullable attribute is OK.
-  std::unique_ptr<CatalogRelation> nullable_relation(new CatalogRelation(NULL, "nullable_relation"));
-  CatalogAttribute *nullable_attribute = new CatalogAttribute(nullable_relation.get(),
-                                                              "nullable_attr",
-                                                              TypeFactory::GetType(kInt, true));
-  ASSERT_EQ(0, nullable_relation->addAttribute(nullable_attribute));
-  tuple_store_description_.reset(new TupleStorageSubBlockDescription());
-  tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::PACKED_ROW_STORE);
-  EXPECT_TRUE(PackedRowStoreTupleStorageSubBlock::DescriptionIsValid(*nullable_relation,
-                                                                     *tuple_store_description_));
-
-  // A relation with a variable-length attribute can't be used with this block type.
-  std::unique_ptr<CatalogRelation> variable_length_relation(new CatalogRelation(NULL, "variable_length_relation"));
-  CatalogAttribute *variable_length_attribute = new CatalogAttribute(variable_length_relation.get(),
-                                                                    "variable_length_attr",
-                                                                     TypeFactory::GetType(kVarChar, 20, false));
-  ASSERT_EQ(0, variable_length_relation->addAttribute(variable_length_attribute));
-  EXPECT_FALSE(PackedRowStoreTupleStorageSubBlock::DescriptionIsValid(*variable_length_relation,
-                                                                      *tuple_store_description_));
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockDeathTest, ConstructWithInvalidDescriptionTest) {
-  tuple_store_description_.reset(new TupleStorageSubBlockDescription());
-  tuple_store_description_->set_sub_block_type(TupleStorageSubBlockDescription::BASIC_COLUMN_STORE);
-  EXPECT_DEATH(initializeNewBlock(kSubBlockSize), "");
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, MemoryTooSmallTest) {
-  // 1 byte short.
-  EXPECT_THROW(initializeNewBlock(sizeof(tuple_id) - 1),
-               BlockMemoryTooSmall);
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, InsertTest) {
-  initializeNewBlock(kSubBlockSize);
-
-  int row_capacity = computeRowCapacity();
-
-  EXPECT_TRUE(tuple_store_->supportsAdHocInsert());
-  EXPECT_TRUE(tuple_store_->adHocInsertIsEfficient());
-  EXPECT_FALSE(tuple_store_->isCompressed());
-  EXPECT_TRUE(tuple_store_->isEmpty());
-
-  std::unique_ptr<Tuple> current_tuple;
-  for (int tuple_num = 0;
-       tuple_num < row_capacity;
-       ++tuple_num) {
-    current_tuple.reset(createSampleTuple(tuple_num));
-    TupleStorageSubBlock::InsertResult result = tuple_store_->insertTuple(*current_tuple);
-    ASSERT_EQ(tuple_num, result.inserted_id);
-    ASSERT_FALSE(result.ids_mutated);
-
-    EXPECT_FALSE(tuple_store_->isEmpty());
-    EXPECT_TRUE(tuple_store_->isPacked());
-    EXPECT_EQ(tuple_num, tuple_store_->getMaxTupleID());
-    EXPECT_EQ(tuple_num + 1, tuple_store_->numTuples());
-  }
-
-  current_tuple.reset(createSampleTuple(0));
-  TupleStorageSubBlock::InsertResult result = tuple_store_->insertTuple(*current_tuple);
-  EXPECT_EQ(-1, result.inserted_id);
-  EXPECT_FALSE(result.ids_mutated);
-
-  EXPECT_TRUE(tuple_store_->isPacked());
-  EXPECT_EQ(row_capacity - 1, tuple_store_->getMaxTupleID());
-  EXPECT_EQ(row_capacity, tuple_store_->numTuples());
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, InsertInBatchTest) {
-  initializeNewBlock(kSubBlockSize);
-
-  int row_capacity = computeRowCapacity();
-
-  EXPECT_TRUE(tuple_store_->supportsAdHocInsert());
-  EXPECT_TRUE(tuple_store_->adHocInsertIsEfficient());
-  EXPECT_FALSE(tuple_store_->isCompressed());
-  EXPECT_TRUE(tuple_store_->isEmpty());
-
-  std::unique_ptr<Tuple> current_tuple;
-  for (int tuple_num = 0;
-       tuple_num < row_capacity;
-       ++tuple_num) {
-    current_tuple.reset(createSampleTuple(tuple_num));
-    EXPECT_TRUE(tuple_store_->insertTupleInBatch(*current_tuple));
-
-    EXPECT_FALSE(tuple_store_->isEmpty());
-    EXPECT_TRUE(tuple_store_->isPacked());
-    EXPECT_EQ(tuple_num, tuple_store_->getMaxTupleID());
-    EXPECT_EQ(tuple_num + 1, tuple_store_->numTuples());
-  }
-
-  current_tuple.reset(createSampleTuple(0));
-  EXPECT_FALSE(tuple_store_->insertTupleInBatch(*current_tuple));
-
-  EXPECT_TRUE(tuple_store_->isPacked());
-  EXPECT_EQ(row_capacity - 1, tuple_store_->getMaxTupleID());
-  EXPECT_EQ(row_capacity, tuple_store_->numTuples());
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, ColumnAccessorTest) {
-  if (GetParam()) {   // when true, the attributes can be nullable.
-    checkColumnAccessor<true>();
-  } else {   // when false, the attributes are non-null.
-    checkColumnAccessor<false>();
-  }
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, GetAttributeValueTest) {
-  initializeNewBlock(kSubBlockSize);
-  fillBlockWithSampleData();
-  ASSERT_TRUE(tuple_store_->isPacked());
-
-  for (tuple_id tid = 0;
-       tid <= tuple_store_->getMaxTupleID();
-       ++tid) {
-    checkTupleValuesUntyped(tid, tid);
-  }
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, GetAttributeValueTypedTest) {
-  initializeNewBlock(kSubBlockSize);
-  fillBlockWithSampleData();
-  ASSERT_TRUE(tuple_store_->isPacked());
-
-  for (tuple_id tid = 0;
-       tid <= tuple_store_->getMaxTupleID();
-       ++tid) {
-    checkTupleValuesTyped(tid, tid);
-  }
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, SetAttributeValueTypedTest) {
-  initializeNewBlock(kSubBlockSize);
-  fillBlockWithSampleData();
-  ASSERT_TRUE(tuple_store_->isPacked());
-
-  // Alter every 16th tuple.
-  for (tuple_id tid = 0;
-       tid <= tuple_store_->getMaxTupleID();
-       tid += 16) {
-    std::unique_ptr<Tuple> mod_tuple(createSampleTuple(-tid));
-
-    std::unordered_map<attribute_id, TypedValue> new_values;
-    new_values.insert(make_pair(0, mod_tuple->getAttributeValue(0)));
-    new_values.insert(make_pair(1, mod_tuple->getAttributeValue(1)));
-    new_values.insert(make_pair(2, mod_tuple->getAttributeValue(2)));
-    ASSERT_TRUE(tuple_store_->canSetAttributeValuesInPlaceTyped(tid, new_values));
-
-    tuple_store_->setAttributeValueInPlaceTyped(tid, 0, mod_tuple->getAttributeValue(0));
-    tuple_store_->setAttributeValueInPlaceTyped(tid, 1, mod_tuple->getAttributeValue(1));
-    tuple_store_->setAttributeValueInPlaceTyped(tid, 2, mod_tuple->getAttributeValue(2));
-  }
-
-  // Check all values.
-  for (tuple_id tid = 0;
-       tid <= tuple_store_->getMaxTupleID();
-       ++tid) {
-    if (tid & 0xF) {
-      checkTupleValuesTyped(tid, tid);
-    } else {
-      checkTupleValuesTyped(tid, -tid);
-    }
-  }
-}
-
-TEST_P(PackedRowStoreTupleStorageSubBlockTest, DeleteTest) {
-  initializeNewBlock(kSubBlockSize);
-  fillBlockWithSampleData();
-  ASSERT_TRUE(tuple_store_->isPacked());
-
-  tuple_id original_num_tuples = tuple_store_->numTuples();
-
-  // Delete the last tuple.
-  EXPECT_FALSE(tuple_store_->deleteTuple(original_num_tuples - 1));
-
-  // Delete the first tuple.
-  EXPECT_TRUE(tuple_store_->deleteTuple(0));
-
-  // Delete a sequence of tuples.
-  TupleIdSequence delete_sequence(tuple_store_->getMaxTupleID() + 1);
-  for (tuple_id tid = 63;
-       tid <= tuple_store_->getMaxTupleID();
-       tid += 64) {
-    delete_sequence.set(tid, true);
-  }
-  EXPECT_TRUE(tuple_store_->bulkDeleteTuples(&delete_sequence));
-
-  EXPECT_EQ(static_cast<tuple_id>(original_num_tuples - 2 - delete_sequence.numTuples()),
-            tuple_store_->numTuples());
-
-  tuple_id remaining_tid = 0;
-  for (tuple_id original_tid = 0;
-       original_tid < (original_num_tuples - 1);
-       ++original_tid) {
-    if (original_tid & 0x3F) {
-      checkTupleValuesUntyped(remaining_tid, original_tid);
-      ++remaining_tid;
-    }
-  }
-}
-
-TEST(PackedRowStoreTupleStorageSubBlockNullTypeTest, NullTypeTest) {
-  // Set up a relation with a single NullType attribute.
-  CatalogRelation test_relation(nullptr, "TestRelation");
-  CatalogAttribute *nulltype_attr = new CatalogAttribute(&test_relation,
-                                                         "nulltype_attr",
-                                                         TypeFactory::GetType(kNullType, true));
-  ASSERT_EQ(0, test_relation.addAttribute(nulltype_attr));
-
-  // Set up a minimal StorageBlockLayoutDescription.
-  StorageBlockLayoutDescription layout_desc;
-  layout_desc.set_num_slots(1);
-  layout_desc.mutable_tuple_store_description()->set_sub_block_type(
-      TupleStorageSubBlockDescription::PACKED_ROW_STORE);
-
-  // Check that the description is considered valid.
-  EXPECT_TRUE(StorageBlockLayout::DescriptionIsValid(test_relation, layout_desc));
-
-  StorageBlockLayout layout(test_relation, layout_desc);
-
-  // Construct an actual PackedRowStoreTupleStorageSubBlock.
-  ScopedBuffer tuple_store_memory(kSlotSizeBytes);
-  PackedRowStoreTupleStorageSubBlock tuple_store(test_relation,
-                                                 layout_desc.tuple_store_description(),
-                                                 true,
-                                                 tuple_store_memory.get(),
-                                                 kSlotSizeBytes);
-
-  // Insert some NullType values.
-  std::vector<TypedValue> attr_values;
-  attr_values.emplace_back(kNullType);
-  Tuple tuple(std::move(attr_values));
-
-  for (tuple_id tid = 0; tid < 100; ++tid) {
-    tuple_store.insertTuple(tuple);
-  }
-
-  EXPECT_EQ(100, tuple_store.numTuples());
-
-  // Delete some values.
-  TupleIdSequence delete_sequence(100);
-  delete_sequence.set(5, true);
-  delete_sequence.set(25, true);
-  delete_sequence.set(45, true);
-  delete_sequence.set(65, true);
-  delete_sequence.set(85, true);
-
-  EXPECT_TRUE(tuple_store.bulkDeleteTuples(&delete_sequence));
-  EXPECT_EQ(95, tuple_store.numTuples());
-  ASSERT_EQ(94, tuple_store.getMaxTupleID());
-
-  // Read out values.
-  for (tuple_id tid = 0; tid < 95; ++tid) {
-    ASSERT_TRUE(tuple_store.hasTupleWithID(tid));
-    EXPECT_EQ(nullptr, tuple_store.getAttributeValue(tid, 0));
-
-    TypedValue value = tuple_store.getAttributeValueTyped(tid, 0);
-    EXPECT_TRUE(value.isNull());
-    EXPECT_EQ(kNullType, value.getTypeID());
-  }
-}
-
-// Note: INSTANTIATE_TEST_CASE_P has variadic arguments part. If the variable argument part
-//       is empty, C++11 standard says it should produce a warning. A warning is converted
-//       to an error since we use -Werror as a compiler parameter. It causes Travis to build.
-//       This is the reason that we must give an empty string argument as a last parameter
-//       to supress warning that clang gives.
-INSTANTIATE_TEST_CASE_P(WithAndWithoutNullableAttributes,
-                        PackedRowStoreTupleStorageSubBlockTest,
-                        ::testing::Bool(),);  // NOLINT(whitespace/comma)
-
-INSTANTIATE_TEST_CASE_P(WithAndWithoutNullableAttributes,
-                        PackedRowStoreTupleStorageSubBlockDeathTest,
-                        ::testing::Bool(),);  // NOLINT(whitespace/comma)
-
-}  // namespace quickstep



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

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

  - CreateTable
  - Insert / InsertSelection
  - UpdateTable


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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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



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

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


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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


[04/22] incubator-quickstep git commit: Remove PackedRowStoreBlock

Posted by ji...@apache.org.
Remove PackedRowStoreBlock


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

Branch: refs/heads/exact-filter
Commit: c394405abdf6f6fbb3c4f6cdcb3400da7d43fafe
Parents: c0d510e
Author: cramja <ma...@gmail.com>
Authored: Mon Nov 7 13:33:21 2016 -0600
Committer: cramja <ma...@gmail.com>
Committed: Sun Jan 8 14:55:21 2017 -0600

----------------------------------------------------------------------
 storage/CMakeLists.txt | 70 ---------------------------------------------
 1 file changed, 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c394405a/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 61a8a99..534630a 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -229,12 +229,6 @@ add_library(quickstep_storage_InsertDestination_proto
 add_library(quickstep_storage_LinearOpenAddressingHashTable
             ../empty_src.cpp
             LinearOpenAddressingHashTable.hpp)
-add_library(quickstep_storage_PackedRowStoreTupleStorageSubBlock
-            PackedRowStoreTupleStorageSubBlock.cpp
-            PackedRowStoreTupleStorageSubBlock.hpp)
-add_library(quickstep_storage_PackedRowStoreValueAccessor
-            ../empty_src.cpp
-            PackedRowStoreValueAccessor.hpp)
 add_library(quickstep_storage_PartitionedHashTablePool ../empty_src.cpp PartitionedHashTablePool.hpp)
 add_library(quickstep_storage_PreloaderThread PreloaderThread.cpp PreloaderThread.hpp)
 add_library(quickstep_storage_SMAIndexSubBlock SMAIndexSubBlock.cpp SMAIndexSubBlock.hpp)
@@ -827,35 +821,6 @@ target_link_libraries(quickstep_storage_LinearOpenAddressingHashTable
                       quickstep_utility_Alignment
                       quickstep_utility_Macros
                       quickstep_utility_PrimeNumber)
-target_link_libraries(quickstep_storage_PackedRowStoreTupleStorageSubBlock
-                      quickstep_catalog_CatalogAttribute
-                      quickstep_catalog_CatalogRelationSchema
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_expressions_predicate_PredicateCost
-                      quickstep_storage_PackedRowStoreValueAccessor
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_StorageBlockLayout_proto
-                      quickstep_storage_StorageErrors
-                      quickstep_storage_SubBlockTypeRegistry
-                      quickstep_storage_SubBlockTypeRegistryMacros
-                      quickstep_storage_TupleIdSequence
-                      quickstep_storage_TupleStorageSubBlock
-                      quickstep_storage_ValueAccessor
-                      quickstep_storage_ValueAccessorUtil
-                      quickstep_types_Type
-                      quickstep_types_TypedValue
-                      quickstep_types_containers_Tuple
-                      quickstep_utility_BitVector
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_storage_PackedRowStoreValueAccessor
-                      quickstep_catalog_CatalogRelationSchema
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_ValueAccessor
-                      quickstep_types_Type
-                      quickstep_types_TypedValue
-                      quickstep_utility_BitVector
-                      quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_PartitionedHashTablePool
                       glog
                       quickstep_expressions_aggregation_AggregationHandle
@@ -984,7 +949,6 @@ target_link_libraries(quickstep_storage_StorageBlock
                       quickstep_storage_HashTableBase
                       quickstep_storage_IndexSubBlock
                       quickstep_storage_InsertDestinationInterface
-                      quickstep_storage_PackedRowStoreTupleStorageSubBlock
                       quickstep_storage_SMAIndexSubBlock
                       quickstep_storage_SplitRowStoreTupleStorageSubBlock
                       quickstep_storage_StorageBlockBase
@@ -1109,7 +1073,6 @@ target_link_libraries(quickstep_storage_ValueAccessorUtil
                       quickstep_storage_BasicColumnStoreValueAccessor
                       quickstep_storage_CompressedColumnStoreValueAccessor
                       quickstep_storage_CompressedPackedRowStoreValueAccessor
-                      quickstep_storage_PackedRowStoreValueAccessor
                       quickstep_storage_SplitRowStoreValueAccessor
                       quickstep_storage_ValueAccessor
                       quickstep_types_containers_ColumnVectorsValueAccessor
@@ -1179,8 +1142,6 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_InsertDestinationInterface
                       quickstep_storage_InsertDestination_proto
                       quickstep_storage_LinearOpenAddressingHashTable
-                      quickstep_storage_PackedRowStoreTupleStorageSubBlock
-                      quickstep_storage_PackedRowStoreValueAccessor
                       quickstep_storage_PartitionedHashTablePool
                       quickstep_storage_PreloaderThread
                       quickstep_storage_SMAIndexSubBlock
@@ -1600,37 +1561,6 @@ target_link_libraries(LinearOpenAddressingHashTable_unittest
                       ${LIBS})
 add_test(LinearOpenAddressingHashTable_unittest LinearOpenAddressingHashTable_unittest)
 
-add_executable(PackedRowStoreTupleStorageSubBlock_unittest "${CMAKE_CURRENT_SOURCE_DIR}/tests/PackedRowStoreTupleStorageSubBlock_unittest.cpp")
-target_link_libraries(PackedRowStoreTupleStorageSubBlock_unittest
-                      gtest
-                      gtest_main
-                      quickstep_catalog_CatalogAttribute
-                      quickstep_catalog_CatalogRelation
-                      quickstep_storage_PackedRowStoreTupleStorageSubBlock
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_StorageBlockLayout
-                      quickstep_storage_StorageBlockLayout_proto
-                      quickstep_storage_StorageConstants
-                      quickstep_storage_StorageErrors
-                      quickstep_storage_TupleIdSequence
-                      quickstep_storage_TupleStorageSubBlock
-                      quickstep_types_CharType
-                      quickstep_types_DoubleType
-                      quickstep_types_IntType
-                      quickstep_types_Type
-                      quickstep_types_TypeFactory
-                      quickstep_types_TypeID
-                      quickstep_types_TypedValue
-                      quickstep_types_containers_Tuple
-                      quickstep_types_operations_comparisons_Comparison
-                      quickstep_types_operations_comparisons_ComparisonFactory
-                      quickstep_types_operations_comparisons_ComparisonID
-                      quickstep_utility_BitVector
-                      quickstep_utility_PtrMap
-                      quickstep_utility_ScopedBuffer
-                      ${LIBS})
-add_test(PackedRowStoreTupleStorageSubBlock_unittest PackedRowStoreTupleStorageSubBlock_unittest)
-
 add_executable(SMAIndexSubBlock_unittest "${CMAKE_CURRENT_SOURCE_DIR}/tests/SMAIndexSubBlock_unittest.cpp")
 target_link_libraries(SMAIndexSubBlock_unittest
                       gtest


[09/22] incubator-quickstep git commit: Refactored the method to reconstruct PartitionSchemeHeader.

Posted by ji...@apache.org.
Refactored the method to reconstruct PartitionSchemeHeader.


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

Branch: refs/heads/exact-filter
Commit: c8c5d45bd3553e05ef3f87996925c49f964e8682
Parents: 8d25d43
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Jan 10 14:10:24 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Jan 11 13:36:57 2017 -0800

----------------------------------------------------------------------
 catalog/CMakeLists.txt                     |  3 +++
 catalog/Catalog.proto                      |  3 ++-
 catalog/CatalogRelation.cpp                |  8 ++------
 catalog/PartitionScheme.cpp                |  5 ++---
 catalog/PartitionScheme.hpp                |  6 +-----
 catalog/PartitionSchemeHeader.cpp          | 15 +++++++++++++--
 catalog/PartitionSchemeHeader.hpp          |  7 ++++---
 catalog/tests/PartitionScheme_unittest.cpp |  4 ++--
 storage/InsertDestination.cpp              |  7 +------
 9 files changed, 30 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8c5d45b/catalog/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/catalog/CMakeLists.txt b/catalog/CMakeLists.txt
index 143d12f..7de9a67 100644
--- a/catalog/CMakeLists.txt
+++ b/catalog/CMakeLists.txt
@@ -168,6 +168,9 @@ target_link_libraries(quickstep_catalog_PartitionSchemeHeader
                       glog
                       quickstep_catalog_CatalogTypedefs
                       quickstep_catalog_Catalog_proto
+                      quickstep_types_Type
+                      quickstep_types_TypeFactory
+                      quickstep_types_Type_proto
                       quickstep_types_TypedValue
                       quickstep_types_TypedValue_proto
                       quickstep_types_operations_comparisons_Comparison

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8c5d45b/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index a54862f..89cb7e5 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -49,7 +49,8 @@ message PartitionSchemeHeader {
 message RangePartitionSchemeHeader {
   extend PartitionSchemeHeader {
     // All required.
-    repeated TypedValue partition_range_boundaries = 32;
+    optional Type partition_attr_type = 32;
+    repeated TypedValue partition_range_boundaries = 33;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8c5d45b/catalog/CatalogRelation.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelation.cpp b/catalog/CatalogRelation.cpp
index c288532..41c503c 100644
--- a/catalog/CatalogRelation.cpp
+++ b/catalog/CatalogRelation.cpp
@@ -43,8 +43,6 @@
 
 namespace quickstep {
 
-class Type;
-
 bool CatalogRelation::ProtoIsValid(const serialization::CatalogRelationSchema &proto) {
   // Check that proto is fully initialized.
   if (!CatalogRelationSchema::ProtoIsValid(proto) ||
@@ -90,11 +88,9 @@ CatalogRelation::CatalogRelation(const serialization::CatalogRelationSchema &pro
     const serialization::PartitionScheme &proto_partition_scheme =
         proto.GetExtension(serialization::CatalogRelation::partition_scheme);
 
-    const attribute_id partition_attribute_id = proto_partition_scheme.header().partition_attribute_id();
-    DCHECK(hasAttributeWithId(partition_attribute_id));
-    const Type &attr_type = attr_vec_[partition_attribute_id].getType();
+    DCHECK(hasAttributeWithId(proto_partition_scheme.header().partition_attribute_id()));
 
-    setPartitionScheme(PartitionScheme::ReconstructFromProto(proto_partition_scheme, attr_type));
+    setPartitionScheme(PartitionScheme::ReconstructFromProto(proto_partition_scheme));
 
     // Deserializing the NUMA placement scheme for the relation.
 #ifdef QUICKSTEP_HAVE_LIBNUMA

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8c5d45b/catalog/PartitionScheme.cpp
----------------------------------------------------------------------
diff --git a/catalog/PartitionScheme.cpp b/catalog/PartitionScheme.cpp
index d6e3469..1d7dce0 100644
--- a/catalog/PartitionScheme.cpp
+++ b/catalog/PartitionScheme.cpp
@@ -55,8 +55,7 @@ bool PartitionScheme::ProtoIsValid(
   return true;
 }
 
-PartitionScheme* PartitionScheme::ReconstructFromProto(const serialization::PartitionScheme &proto,
-                                                       const Type &attr_type) {
+PartitionScheme* PartitionScheme::ReconstructFromProto(const serialization::PartitionScheme &proto) {
   DCHECK(ProtoIsValid(proto))
       << "Attempted to create PartitionScheme from an invalid proto description:\n"
       << proto.DebugString();
@@ -74,7 +73,7 @@ PartitionScheme* PartitionScheme::ReconstructFromProto(const serialization::Part
   }
 
   return new PartitionScheme(
-      PartitionSchemeHeader::ReconstructFromProto(proto.header(), attr_type),
+      PartitionSchemeHeader::ReconstructFromProto(proto.header()),
       move(blocks_in_partition));
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8c5d45b/catalog/PartitionScheme.hpp
----------------------------------------------------------------------
diff --git a/catalog/PartitionScheme.hpp b/catalog/PartitionScheme.hpp
index c882a5e..b69a33f 100644
--- a/catalog/PartitionScheme.hpp
+++ b/catalog/PartitionScheme.hpp
@@ -39,8 +39,6 @@
 
 namespace quickstep {
 
-class Type;
-
 /** \addtogroup Catalog
  *  @{
  */
@@ -82,11 +80,9 @@ class PartitionScheme {
    *
    * @param proto The Protocol Buffer serialization of a Partition Scheme,
    *        previously produced by getProto().
-   * @param attr_type The attribute type of the partitioning attribute.
    * @return The deserialized partition scheme object.
    **/
-  static PartitionScheme* ReconstructFromProto(const serialization::PartitionScheme &proto,
-                                               const Type &attr_type);
+  static PartitionScheme* ReconstructFromProto(const serialization::PartitionScheme &proto);
 
   /**
    * @brief Check whether a serialization::PartitionScheme is fully-formed and

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8c5d45b/catalog/PartitionSchemeHeader.cpp
----------------------------------------------------------------------
diff --git a/catalog/PartitionSchemeHeader.cpp b/catalog/PartitionSchemeHeader.cpp
index 26c317d..a4a9063 100644
--- a/catalog/PartitionSchemeHeader.cpp
+++ b/catalog/PartitionSchemeHeader.cpp
@@ -24,6 +24,9 @@
 #include <vector>
 
 #include "catalog/Catalog.pb.h"
+#include "types/Type.hpp"
+#include "types/Type.pb.h"
+#include "types/TypeFactory.hpp"
 #include "types/TypedValue.hpp"
 #include "types/TypedValue.pb.h"
 
@@ -59,7 +62,8 @@ bool PartitionSchemeHeader::ProtoIsValid(
     case serialization::PartitionSchemeHeader::RANGE: {
       const std::size_t num_ranges =
           proto.ExtensionSize(serialization::RangePartitionSchemeHeader::partition_range_boundaries);
-      return num_ranges == proto.num_partitions() - 1;
+      return num_ranges == proto.num_partitions() - 1 &&
+             proto.HasExtension(serialization::RangePartitionSchemeHeader::partition_attr_type);
     }
     default:
       // Partition type is unknown.
@@ -68,7 +72,7 @@ bool PartitionSchemeHeader::ProtoIsValid(
 }
 
 PartitionSchemeHeader* PartitionSchemeHeader::ReconstructFromProto(
-    const serialization::PartitionSchemeHeader &proto, const Type &attr_type) {
+    const serialization::PartitionSchemeHeader &proto) {
   DCHECK(ProtoIsValid(proto))
       << "Attempted to create PartitionSchemeHeader from an invalid proto description:\n"
       << proto.DebugString();
@@ -78,6 +82,10 @@ PartitionSchemeHeader* PartitionSchemeHeader::ReconstructFromProto(
       return new HashPartitionSchemeHeader(proto.num_partitions(), proto.partition_attribute_id());
     }
     case serialization::PartitionSchemeHeader::RANGE: {
+      const Type &attr_type =
+          TypeFactory::ReconstructFromProto(proto.GetExtension(
+              serialization::RangePartitionSchemeHeader::partition_attr_type));
+
       std::vector<TypedValue> partition_ranges;
       for (int i = 0;
            i < proto.ExtensionSize(serialization::RangePartitionSchemeHeader::partition_range_boundaries);
@@ -123,6 +131,9 @@ serialization::PartitionSchemeHeader RangePartitionSchemeHeader::getProto() cons
   proto.set_num_partitions(num_partitions_);
   proto.set_partition_attribute_id(partition_attribute_id_);
 
+  proto.MutableExtension(serialization::RangePartitionSchemeHeader::partition_attr_type)
+      ->MergeFrom(partition_attr_type_->getProto());
+
   for (std::size_t i = 0; i < partition_range_boundaries_.size(); ++i) {
     proto.AddExtension(serialization::RangePartitionSchemeHeader::partition_range_boundaries)
         ->MergeFrom(partition_range_boundaries_[i].getProto());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8c5d45b/catalog/PartitionSchemeHeader.hpp
----------------------------------------------------------------------
diff --git a/catalog/PartitionSchemeHeader.hpp b/catalog/PartitionSchemeHeader.hpp
index 89efc6d..c1d65d1 100644
--- a/catalog/PartitionSchemeHeader.hpp
+++ b/catalog/PartitionSchemeHeader.hpp
@@ -65,12 +65,10 @@ class PartitionSchemeHeader {
    *
    * @param proto The Protocol Buffer serialization of a PartitionSchemeHeader,
    *        previously produced by getProto().
-   * @param attr_type The attribute type of the partitioning attribute.
    * @return The reconstructed PartitionSchemeHeader object.
    **/
   static PartitionSchemeHeader* ReconstructFromProto(
-      const serialization::PartitionSchemeHeader &proto,
-      const Type &attr_type);
+      const serialization::PartitionSchemeHeader &proto);
 
   /**
    * @brief Check whether a serialization::PartitionSchemeHeader is fully-formed
@@ -227,6 +225,7 @@ class RangePartitionSchemeHeader : public PartitionSchemeHeader {
                              const attribute_id attribute,
                              std::vector<TypedValue> &&partition_range)
       : PartitionSchemeHeader(PartitionType::kRange, num_partitions, attribute),
+        partition_attr_type_(&partition_attribute_type),
         partition_range_boundaries_(std::move(partition_range)) {
     DCHECK_EQ(num_partitions - 1, partition_range_boundaries_.size());
 
@@ -298,6 +297,8 @@ class RangePartitionSchemeHeader : public PartitionSchemeHeader {
     }
   }
 
+  const Type* partition_attr_type_;
+
   // The boundaries for each range in the RangePartitionSchemeHeader.
   // The upper bound of the range is stored here.
   const std::vector<TypedValue> partition_range_boundaries_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8c5d45b/catalog/tests/PartitionScheme_unittest.cpp
----------------------------------------------------------------------
diff --git a/catalog/tests/PartitionScheme_unittest.cpp b/catalog/tests/PartitionScheme_unittest.cpp
index d10b26e..97a9092 100644
--- a/catalog/tests/PartitionScheme_unittest.cpp
+++ b/catalog/tests/PartitionScheme_unittest.cpp
@@ -511,7 +511,7 @@ TEST(PartitionSchemeTest, CheckHashPartitionSchemeSerialization) {
   }
   std::unique_ptr<PartitionScheme> part_scheme_from_proto;
   part_scheme_from_proto.reset(
-      PartitionScheme::ReconstructFromProto(part_scheme->getProto(), TypeFactory::GetType(kInt)));
+      PartitionScheme::ReconstructFromProto(part_scheme->getProto()));
 
   const PartitionSchemeHeader &header = part_scheme->getPartitionSchemeHeader();
   const PartitionSchemeHeader &header_from_proto = part_scheme_from_proto->getPartitionSchemeHeader();
@@ -561,7 +561,7 @@ TEST(PartitionSchemeTest, CheckRangePartitionSchemeSerialization) {
   std::unique_ptr<PartitionScheme> part_scheme_from_proto;
 
   part_scheme_from_proto.reset(
-      PartitionScheme::ReconstructFromProto(part_scheme->getProto(), type));
+      PartitionScheme::ReconstructFromProto(part_scheme->getProto()));
 
   const PartitionSchemeHeader &header = part_scheme->getPartitionSchemeHeader();
   const PartitionSchemeHeader &header_from_proto = part_scheme_from_proto->getPartitionSchemeHeader();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8c5d45b/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index 067edf6..19bb356 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -56,8 +56,6 @@ using std::vector;
 
 namespace quickstep {
 
-class Type;
-
 InsertDestination::InsertDestination(const CatalogRelationSchema &relation,
                                      const StorageBlockLayout *layout,
                                      StorageManager *storage_manager,
@@ -132,11 +130,8 @@ InsertDestination* InsertDestination::ReconstructFromProto(
         partitions.push_back(move(partition));
       }
 
-      const serialization::PartitionSchemeHeader &proto_partition_scheme_header = proto_partition_scheme.header();
-      const Type &attr_type =
-          relation.getAttributeById(proto_partition_scheme_header.partition_attribute_id())->getType();
       return new PartitionAwareInsertDestination(
-          PartitionSchemeHeader::ReconstructFromProto(proto_partition_scheme_header, attr_type),
+          PartitionSchemeHeader::ReconstructFromProto(proto_partition_scheme.header()),
           relation,
           layout,
           storage_manager,


[22/22] incubator-quickstep git commit: Add BitVectorExactFilter as a LIP filter and supports Join-to-Semijoin transformation.

Posted by ji...@apache.org.
Add BitVectorExactFilter as a LIP filter and supports Join-to-Semijoin transformation.


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

Branch: refs/heads/exact-filter
Commit: d52b912651f9ac2559379e8fb7c975869b200f2b
Parents: 968ce3f
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Oct 27 14:16:32 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Wed Jan 25 20:33:53 2017 -0600

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   4 +
 query_optimizer/ExecutionGenerator.cpp          |  55 +++
 query_optimizer/ExecutionGenerator.hpp          |   8 +
 query_optimizer/LIPFilterGenerator.cpp          |  80 ++--
 query_optimizer/LIPFilterGenerator.hpp          |  47 ++-
 query_optimizer/PhysicalGenerator.cpp           |   6 +
 query_optimizer/cost_model/CMakeLists.txt       |   5 +
 query_optimizer/cost_model/SimpleCostModel.cpp  |   9 +
 query_optimizer/cost_model/SimpleCostModel.hpp  |   5 +
 .../cost_model/StarSchemaSimpleCostModel.cpp    | 156 +++++++-
 .../cost_model/StarSchemaSimpleCostModel.hpp    |  84 ++++
 query_optimizer/expressions/ExpressionUtil.hpp  |   8 +-
 query_optimizer/physical/CMakeLists.txt         |  14 +
 query_optimizer/physical/FilterJoin.cpp         | 115 ++++++
 query_optimizer/physical/FilterJoin.hpp         | 156 ++++++++
 .../physical/LIPFilterConfiguration.hpp         |  65 ++-
 query_optimizer/physical/PatternMatcher.hpp     |   2 +
 query_optimizer/physical/PhysicalType.hpp       |   1 +
 query_optimizer/physical/TopLevelPlan.hpp       |   3 +-
 query_optimizer/rules/AttachLIPFilters.cpp      |  14 +-
 query_optimizer/rules/CMakeLists.txt            |  22 +
 query_optimizer/rules/InjectJoinFilters.cpp     | 399 +++++++++++++++++++
 query_optimizer/rules/InjectJoinFilters.hpp     |  99 +++++
 relational_operators/BuildLIPFilterOperator.cpp | 125 ++++++
 relational_operators/BuildLIPFilterOperator.hpp | 165 ++++++++
 relational_operators/CMakeLists.txt             |  23 ++
 utility/CMakeLists.txt                          |   1 +
 utility/PlanVisualizer.cpp                      |  20 +-
 utility/lip_filter/BitVectorExactFilter.hpp     | 180 +++++++++
 utility/lip_filter/CMakeLists.txt               |  11 +
 utility/lip_filter/LIPFilter.hpp                |   2 +-
 utility/lip_filter/LIPFilter.proto              |  20 +-
 utility/lip_filter/LIPFilterDeployment.cpp      |  70 ++--
 utility/lip_filter/LIPFilterDeployment.hpp      |  28 +-
 utility/lip_filter/LIPFilterFactory.cpp         |  46 +++
 35 files changed, 1928 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 1a15271..1a11b7b 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -100,6 +100,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_queryoptimizer_physical_CreateTable
                       quickstep_queryoptimizer_physical_DeleteTuples
                       quickstep_queryoptimizer_physical_DropTable
+                      quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_InsertSelection
                       quickstep_queryoptimizer_physical_InsertTuple
@@ -119,6 +120,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_queryoptimizer_physical_WindowAggregate
                       quickstep_relationaloperators_AggregationOperator
                       quickstep_relationaloperators_BuildHashOperator
+                      quickstep_relationaloperators_BuildLIPFilterOperator
                       quickstep_relationaloperators_CreateIndexOperator
                       quickstep_relationaloperators_CreateTableOperator
                       quickstep_relationaloperators_DeleteOperator
@@ -165,6 +167,7 @@ target_link_libraries(quickstep_queryoptimizer_LIPFilterGenerator
                       quickstep_queryoptimizer_QueryPlan
                       quickstep_queryoptimizer_expressions_ExprId
                       quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_LIPFilterConfiguration
                       quickstep_queryoptimizer_physical_Physical
@@ -210,6 +213,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_logical_Logical
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_rules_AttachLIPFilters
+                      quickstep_queryoptimizer_rules_InjectJoinFilters
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_rules_SwapProbeBuild

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index e25b8ad..8249beb 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -76,6 +76,7 @@
 #include "query_optimizer/physical/CreateTable.hpp"
 #include "query_optimizer/physical/DeleteTuples.hpp"
 #include "query_optimizer/physical/DropTable.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/InsertSelection.hpp"
 #include "query_optimizer/physical/InsertTuple.hpp"
@@ -95,6 +96,7 @@
 #include "query_optimizer/physical/WindowAggregate.hpp"
 #include "relational_operators/AggregationOperator.hpp"
 #include "relational_operators/BuildHashOperator.hpp"
+#include "relational_operators/BuildLIPFilterOperator.hpp"
 #include "relational_operators/CreateIndexOperator.hpp"
 #include "relational_operators/CreateTableOperator.hpp"
 #include "relational_operators/DeleteOperator.hpp"
@@ -271,6 +273,9 @@ void ExecutionGenerator::generatePlanInternal(
     case P::PhysicalType::kDropTable:
       return convertDropTable(
           std::static_pointer_cast<const P::DropTable>(physical_plan));
+    case P::PhysicalType::kFilterJoin:
+      return convertFilterJoin(
+          std::static_pointer_cast<const P::FilterJoin>(physical_plan));
     case P::PhysicalType::kHashJoin:
       return convertHashJoin(
           std::static_pointer_cast<const P::HashJoin>(physical_plan));
@@ -608,6 +613,56 @@ void ExecutionGenerator::convertSharedSubplanReference(const physical::SharedSub
   }
 }
 
+void ExecutionGenerator::convertFilterJoin(const P::FilterJoinPtr &physical_plan) {
+  P::PhysicalPtr probe_physical = physical_plan->left();
+  P::PhysicalPtr build_physical = physical_plan->right();
+
+  P::FilterJoinPtr filter_join;
+  if (P::SomeFilterJoin::MatchesWithConditionalCast(build_physical, &filter_join)) {
+    build_physical = filter_join->left();
+    DCHECK(build_physical->getPhysicalType() != P::PhysicalType::kFilterJoin);
+  }
+
+  // Convert the predicate proto.
+  QueryContext::predicate_id build_side_predicate_index = QueryContext::kInvalidPredicateId;
+  if (physical_plan->build_side_filter_predicate()) {
+    build_side_predicate_index = query_context_proto_->predicates_size();
+
+    std::unique_ptr<const Predicate> build_side_predicate(
+        convertPredicate(physical_plan->build_side_filter_predicate()));
+    query_context_proto_->add_predicates()->CopyFrom(build_side_predicate->getProto());
+  }
+
+  const CatalogRelationInfo *probe_relation_info =
+      findRelationInfoOutputByPhysical(probe_physical);
+  const CatalogRelationInfo *build_relation_info =
+      findRelationInfoOutputByPhysical(build_physical);
+
+  const QueryPlan::DAGNodeIndex build_filter_operator_index =
+      execution_plan_->addRelationalOperator(
+          new BuildLIPFilterOperator(
+              query_handle_->query_id(),
+              *build_relation_info->relation,
+              build_side_predicate_index,
+              build_relation_info->isStoredRelation()));
+
+  if (!build_relation_info->isStoredRelation()) {
+    execution_plan_->addDirectDependency(build_filter_operator_index,
+                                         build_relation_info->producer_operator_index,
+                                         false /* is_pipeline_breaker */);
+  }
+
+  physical_to_output_relation_map_.emplace(
+      std::piecewise_construct,
+      std::forward_as_tuple(physical_plan),
+      std::forward_as_tuple(probe_relation_info->producer_operator_index,
+                            probe_relation_info->relation));
+
+  DCHECK(lip_filter_generator_ != nullptr);
+  lip_filter_generator_->addFilterJoinInfo(physical_plan,
+                                           build_filter_operator_index);
+}
+
 void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   // HashJoin is converted to three operators:
   //     BuildHash, HashJoin, DestroyHash. The second is the primary operator.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/ExecutionGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.hpp b/query_optimizer/ExecutionGenerator.hpp
index 55197c9..eba6eee 100644
--- a/query_optimizer/ExecutionGenerator.hpp
+++ b/query_optimizer/ExecutionGenerator.hpp
@@ -46,6 +46,7 @@
 #include "query_optimizer/physical/CreateTable.hpp"
 #include "query_optimizer/physical/DeleteTuples.hpp"
 #include "query_optimizer/physical/DropTable.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/InsertSelection.hpp"
 #include "query_optimizer/physical/InsertTuple.hpp"
@@ -248,6 +249,13 @@ class ExecutionGenerator {
   void convertSharedSubplanReference(const physical::SharedSubplanReferencePtr &physical_plan);
 
   /**
+   * @brief Converts a FilterJoin to a BuildLIPFilter operator.
+   *
+   * @param physical_plan The FilterJoin to be converted.
+   */
+  void convertFilterJoin(const physical::FilterJoinPtr &physical_plan);
+
+  /**
    * @brief Converts a HashJoin to BuildHash, HashJoin and
    *        DestroyHash operators.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/LIPFilterGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/LIPFilterGenerator.cpp b/query_optimizer/LIPFilterGenerator.cpp
index 404037e..a80c261 100644
--- a/query_optimizer/LIPFilterGenerator.cpp
+++ b/query_optimizer/LIPFilterGenerator.cpp
@@ -30,6 +30,8 @@
 #include "utility/lip_filter/LIPFilter.hpp"
 #include "utility/lip_filter/LIPFilter.pb.h"
 
+#include "google/protobuf/text_format.h"
+
 #include "glog/logging.h"
 
 namespace quickstep {
@@ -64,8 +66,9 @@ void LIPFilterGenerator::registerAttributeMap(
 }
 
 void LIPFilterGenerator::deployLIPFilters(QueryPlan *execution_plan,
-                                          serialization::QueryContext *query_context_proto) const {
-  LIPFilterBuilderMap lip_filter_builder_map;
+                                          serialization::QueryContext *query_context_proto) {
+  lip_filter_builder_map_.clear();
+  lip_filter_deployment_protos_.clear();
 
   // Deploy builders
   const auto &build_info_map = lip_filter_configuration_->getBuildInfoMap();
@@ -76,8 +79,7 @@ void LIPFilterGenerator::deployLIPFilters(QueryPlan *execution_plan,
                             query_context_proto,
                             info.builder_node,
                             info.builder_operator_index,
-                            build_it->second,
-                            &lip_filter_builder_map);
+                            build_it->second);
     }
   }
 
@@ -90,10 +92,16 @@ void LIPFilterGenerator::deployLIPFilters(QueryPlan *execution_plan,
                           query_context_proto,
                           info.prober_node,
                           info.prober_operator_index,
-                          probe_it->second,
-                          lip_filter_builder_map);
+                          probe_it->second);
     }
   }
+
+  // Attach LIPFilterDeployment information to the RelationalOperators.
+  for (const auto &entry : lip_filter_deployment_protos_) {
+    RelationalOperator *relop =
+        execution_plan->getQueryPlanDAGMutable()->getNodePayloadMutable(entry.first);
+    relop->deployLIPFilters(entry.second.first);
+  }
 }
 
 void LIPFilterGenerator::deployBuilderInternal(
@@ -101,12 +109,9 @@ void LIPFilterGenerator::deployBuilderInternal(
     serialization::QueryContext *query_context_proto,
     const physical::PhysicalPtr &builder_node,
     const QueryPlan::DAGNodeIndex builder_operator_index,
-    const std::vector<physical::LIPFilterBuildInfo> &build_info_vec,
-    LIPFilterBuilderMap *lip_filter_builder_map) const {
-  const auto lip_deployment_index = query_context_proto->lip_filter_deployments_size();
+    const std::vector<physical::LIPFilterBuildInfo> &build_info_vec) {
   auto *lip_filter_deployment_info_proto =
-      query_context_proto->add_lip_filter_deployments();
-  lip_filter_deployment_info_proto->set_action_type(serialization::LIPFilterActionType::BUILD);
+      getLIPFilterDeploymentProto(builder_operator_index, query_context_proto);
 
   const auto &builder_attribute_map = attribute_map_.at(builder_node);
   for (const auto &info : build_info_vec) {
@@ -119,6 +124,7 @@ void LIPFilterGenerator::deployBuilderInternal(
     switch (info.filter_type) {
       case LIPFilterType::kSingleIdentityHashFilter: {
         DCHECK(!attr_type.isVariableLength());
+        DCHECK(!info.is_anti_filter);
         lip_filter_proto->set_lip_filter_type(
             serialization::LIPFilterType::SINGLE_IDENTITY_HASH_FILTER);
         lip_filter_proto->SetExtension(
@@ -127,27 +133,34 @@ void LIPFilterGenerator::deployBuilderInternal(
             serialization::SingleIdentityHashFilter::attribute_size, attr_type.minimumByteLength());
         break;
       }
+      case LIPFilterType::kBitVectorExactFilter: {
+        DCHECK(!attr_type.isVariableLength());
+        lip_filter_proto->set_lip_filter_type(
+            serialization::LIPFilterType::BIT_VECTOR_EXACT_FILTER);
+        lip_filter_proto->SetExtension(
+            serialization::BitVectorExactFilter::filter_cardinality, info.filter_cardinality);
+        lip_filter_proto->SetExtension(
+            serialization::BitVectorExactFilter::attribute_size, attr_type.minimumByteLength());
+        lip_filter_proto->SetExtension(
+            serialization::BitVectorExactFilter::is_anti_filter, info.is_anti_filter);
+        break;
+      }
       default:
         LOG(FATAL) << "Unsupported LIPFilter type";
         break;
     }
 
     // Register the builder information which is needed later by the probers.
-    lip_filter_builder_map->emplace(
+    lip_filter_builder_map_.emplace(
         std::make_pair(info.build_attribute->id(), builder_node),
         std::make_pair(lip_filter_id, builder_operator_index));
 
     // Add the builder deployment information into query context.
-    auto *lip_filter_entry_proto = lip_filter_deployment_info_proto->add_entries();
+    auto *lip_filter_entry_proto = lip_filter_deployment_info_proto->add_build_entries();
     lip_filter_entry_proto->set_lip_filter_id(lip_filter_id);
     lip_filter_entry_proto->set_attribute_id(target_attr->getID());
     lip_filter_entry_proto->mutable_attribute_type()->CopyFrom(attr_type.getProto());
   }
-
-  // Attach the LIPFilterDeployment information to the RelationalOperator.
-  RelationalOperator *relop =
-      execution_plan->getQueryPlanDAGMutable()->getNodePayloadMutable(builder_operator_index);
-  relop->deployLIPFilters(lip_deployment_index);
 }
 
 void LIPFilterGenerator::deployProberInteral(
@@ -155,23 +168,20 @@ void LIPFilterGenerator::deployProberInteral(
     serialization::QueryContext *query_context_proto,
     const physical::PhysicalPtr &prober_node,
     const QueryPlan::DAGNodeIndex prober_operator_index,
-    const std::vector<physical::LIPFilterProbeInfo> &probe_info_vec,
-    const LIPFilterBuilderMap &lip_filter_builder_map) const {
-  const auto lip_deployment_index = query_context_proto->lip_filter_deployments_size();
+    const std::vector<physical::LIPFilterProbeInfo> &probe_info_vec) {
   auto *lip_filter_deployment_info_proto =
-      query_context_proto->add_lip_filter_deployments();
-  lip_filter_deployment_info_proto->set_action_type(serialization::LIPFilterActionType::PROBE);
+      getLIPFilterDeploymentProto(prober_operator_index, query_context_proto);
 
   const auto &prober_attribute_map = attribute_map_.at(prober_node);
   for (const auto &info : probe_info_vec) {
     // Find the corresponding builder for the to-be-probed LIPFilter.
     const auto &builder_info =
-        lip_filter_builder_map.at(
+        lip_filter_builder_map_.at(
             std::make_pair(info.build_attribute->id(), info.builder));
     const CatalogAttribute *target_attr = prober_attribute_map.at(info.probe_attribute->id());
 
     // Add the prober deployment information into query context.
-    auto *lip_filter_entry_proto = lip_filter_deployment_info_proto->add_entries();
+    auto *lip_filter_entry_proto = lip_filter_deployment_info_proto->add_probe_entries();
     lip_filter_entry_proto->set_lip_filter_id(builder_info.first);
     lip_filter_entry_proto->set_attribute_id(target_attr->getID());
     lip_filter_entry_proto->mutable_attribute_type()->CopyFrom(
@@ -183,11 +193,23 @@ void LIPFilterGenerator::deployProberInteral(
                                                  builder_info.second,
                                                  true /* is_pipeline_breaker */);
   }
+}
 
-  // Attach the LIPFilterDeployment information to the RelationalOperator.
-  RelationalOperator *relop =
-      execution_plan->getQueryPlanDAGMutable()->getNodePayloadMutable(prober_operator_index);
-  relop->deployLIPFilters(lip_deployment_index);
+serialization::LIPFilterDeployment* LIPFilterGenerator::getLIPFilterDeploymentProto(
+    const QueryPlan::DAGNodeIndex op_index,
+    serialization::QueryContext *query_context_proto) {
+  const auto proto_it = lip_filter_deployment_protos_.find(op_index);
+  if (proto_it == lip_filter_deployment_protos_.end()) {
+    const int lip_deployment_index =
+        query_context_proto->lip_filter_deployments_size();
+    auto *lip_filter_deployment_proto =
+        query_context_proto->add_lip_filter_deployments();
+    lip_filter_deployment_protos_.emplace(
+        op_index, std::make_pair(lip_deployment_index, lip_filter_deployment_proto));
+    return lip_filter_deployment_proto;
+  } else {
+    return proto_it->second.second;
+  }
 }
 
 }  // namespace optimizer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/LIPFilterGenerator.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/LIPFilterGenerator.hpp b/query_optimizer/LIPFilterGenerator.hpp
index 9d191a1..43973cb 100644
--- a/query_optimizer/LIPFilterGenerator.hpp
+++ b/query_optimizer/LIPFilterGenerator.hpp
@@ -30,6 +30,7 @@
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/physical/LIPFilterConfiguration.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/Selection.hpp"
@@ -39,7 +40,12 @@
 
 namespace quickstep {
 
-namespace serialization { class QueryContext; }
+namespace serialization {
+
+class QueryContext;
+class LIPFilterDeployment;
+
+}
 
 class CatalogAttribute;
 
@@ -93,6 +99,20 @@ class LIPFilterGenerator {
 
   /**
    * @brief Add physical-to-execution mapping information for deploying LIPFilters
+   *        to a FilterJoin node.
+   *
+   * @param filter_join A physical FilterJoin node.
+   * @param build_filter_operator_index The index of the BuildLIPFilterOperator
+   *        that corresponds to \p filter_join in the execution plan.
+   */
+  void addFilterJoinInfo(const physical::FilterJoinPtr &filter_join,
+                         const QueryPlan::DAGNodeIndex build_filter_operator_index) {
+    builder_infos_.emplace_back(filter_join, build_filter_operator_index);
+    prober_infos_.emplace_back(filter_join, build_filter_operator_index);
+  }
+
+  /**
+   * @brief Add physical-to-execution mapping information for deploying LIPFilters
    *        to a hash-join.
    *
    * @param hash_join A physical HashJoin node.
@@ -128,7 +148,7 @@ class LIPFilterGenerator {
    * @param query_context_proto QueryContext protobuf for the execution plan.
    */
   void deployLIPFilters(QueryPlan *execution_plan,
-                        serialization::QueryContext *query_context_proto) const;
+                        serialization::QueryContext *query_context_proto);
 
  private:
   /**
@@ -157,24 +177,21 @@ class LIPFilterGenerator {
     const QueryPlan::DAGNodeIndex prober_operator_index;
   };
 
-  // Maps each LIPFilter's building attribute to the LIPFilter's id in QueryContext
-  // as well as the LIPFilter's building relational operator's index.
-  typedef std::map<std::pair<expressions::ExprId, physical::PhysicalPtr>,
-                   std::pair<QueryContext::lip_filter_id, QueryPlan::DAGNodeIndex>> LIPFilterBuilderMap;
-
   void deployBuilderInternal(QueryPlan *execution_plan,
                              serialization::QueryContext *query_context_proto,
                              const physical::PhysicalPtr &builder_node,
                              const QueryPlan::DAGNodeIndex builder_operator_index,
-                             const std::vector<physical::LIPFilterBuildInfo> &build_info_vec,
-                             LIPFilterBuilderMap *lip_filter_builder_map) const;
+                             const std::vector<physical::LIPFilterBuildInfo> &build_info_vec);
 
   void deployProberInteral(QueryPlan *execution_plan,
                            serialization::QueryContext *query_context_proto,
                            const physical::PhysicalPtr &prober_node,
                            const QueryPlan::DAGNodeIndex prober_operator_index,
-                           const std::vector<physical::LIPFilterProbeInfo> &probe_info_vec,
-                           const LIPFilterBuilderMap &lip_filter_builder_map) const;
+                           const std::vector<physical::LIPFilterProbeInfo> &probe_info_vec);
+
+  serialization::LIPFilterDeployment* getLIPFilterDeploymentProto(
+      const QueryPlan::DAGNodeIndex op_index,
+      serialization::QueryContext *query_context_proto);
 
   const physical::LIPFilterConfigurationPtr lip_filter_configuration_;
 
@@ -183,6 +200,14 @@ class LIPFilterGenerator {
 
   std::map<physical::PhysicalPtr, std::map<expressions::ExprId, const CatalogAttribute *>> attribute_map_;
 
+  // Maps each LIPFilter's building attribute to the LIPFilter's id in QueryContext
+  // as well as the LIPFilter's building relational operator's index.
+  std::map<std::pair<expressions::ExprId, physical::PhysicalPtr>,
+           std::pair<QueryContext::lip_filter_id, QueryPlan::DAGNodeIndex>> lip_filter_builder_map_;
+
+  std::map<QueryPlan::DAGNodeIndex,
+           std::pair<int, serialization::LIPFilterDeployment *>> lip_filter_deployment_protos_;
+
   DISALLOW_COPY_AND_ASSIGN(LIPFilterGenerator);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 7cb97dc..9df9b9a 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -30,6 +30,7 @@
 #include "query_optimizer/rules/PruneColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/rules/SwapProbeBuild.hpp"
+#include "query_optimizer/rules/InjectJoinFilters.hpp"
 #include "query_optimizer/strategy/Aggregate.hpp"
 #include "query_optimizer/strategy/Join.hpp"
 #include "query_optimizer/strategy/OneToOne.hpp"
@@ -50,6 +51,8 @@ DEFINE_bool(reorder_hash_joins, true,
             "cardinality and selective tables to be joined first, which is suitable "
             "for queries on star-schema tables.");
 
+DEFINE_bool(use_filter_join, true, "Transform HashJoin to FilterJoin.");
+
 DEFINE_bool(use_lip_filters, true,
             "If true, use LIP (Lookahead Information Passing) filters to accelerate "
             "query processing. LIP filters are effective for queries on star schema "
@@ -109,6 +112,9 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   } else {
     rules.emplace_back(new SwapProbeBuild());
   }
+  if (FLAGS_use_filter_join) {
+    rules.emplace_back(new InjectJoinFilters());
+  }
   if (FLAGS_use_lip_filters) {
     rules.emplace_back(new AttachLIPFilters());
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/cost_model/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/CMakeLists.txt b/query_optimizer/cost_model/CMakeLists.txt
index 90133e7..5f28bb3 100644
--- a/query_optimizer/cost_model/CMakeLists.txt
+++ b/query_optimizer/cost_model/CMakeLists.txt
@@ -33,6 +33,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_SimpleCostModel
                       quickstep_catalog_CatalogRelationStatistics
                       quickstep_queryoptimizer_costmodel_CostModel
                       quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_NestedLoopsJoin
                       quickstep_queryoptimizer_physical_Physical
@@ -49,6 +50,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       glog
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationStatistics
+                      quickstep_catalog_CatalogTypedefs
                       quickstep_queryoptimizer_costmodel_CostModel
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ComparisonExpression
@@ -60,6 +62,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       quickstep_queryoptimizer_expressions_PatternMatcher
                       quickstep_queryoptimizer_expressions_Predicate
                       quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_NestedLoopsJoin
                       quickstep_queryoptimizer_physical_PatternMatcher
@@ -72,6 +75,8 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_physical_WindowAggregate
+                      quickstep_types_NullType
+                      quickstep_types_TypedValue
                       quickstep_utility_Macros)
 
 # Module all-in-one library:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index 7808898..e9d2e3a 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -27,6 +27,7 @@
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/PhysicalType.hpp"
@@ -61,6 +62,9 @@ std::size_t SimpleCostModel::estimateCardinality(
     case P::PhysicalType::kTableGenerator:
       return estimateCardinalityForTableGenerator(
           std::static_pointer_cast<const P::TableGenerator>(physical_plan));
+    case P::PhysicalType::kFilterJoin:
+      return estimateCardinalityForFilterJoin(
+          std::static_pointer_cast<const P::FilterJoin>(physical_plan));
     case P::PhysicalType::kHashJoin:
       return estimateCardinalityForHashJoin(
           std::static_pointer_cast<const P::HashJoin>(physical_plan));
@@ -119,6 +123,11 @@ std::size_t SimpleCostModel::estimateCardinalityForTableGenerator(
   return physical_plan->generator_function_handle()->getEstimatedCardinality();
 }
 
+std::size_t SimpleCostModel::estimateCardinalityForFilterJoin(
+    const P::FilterJoinPtr &physical_plan) {
+  return estimateCardinality(physical_plan->left());
+}
+
 std::size_t SimpleCostModel::estimateCardinalityForHashJoin(
     const P::HashJoinPtr &physical_plan) {
   return std::max(estimateCardinality(physical_plan->left()),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/cost_model/SimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.hpp b/query_optimizer/cost_model/SimpleCostModel.hpp
index 16366cd..4edc2fe 100644
--- a/query_optimizer/cost_model/SimpleCostModel.hpp
+++ b/query_optimizer/cost_model/SimpleCostModel.hpp
@@ -26,6 +26,7 @@
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/Selection.hpp"
@@ -80,6 +81,10 @@ class SimpleCostModel : public CostModel {
   std::size_t estimateCardinalityForSort(
       const physical::SortPtr &physical_plan);
 
+  // Returns the left child's cardinality
+  std::size_t estimateCardinalityForFilterJoin(
+      const physical::FilterJoinPtr &physical_plan);
+
   // Returns the larger value of the estimated cardinalities of two
   // input plans.
   std::size_t estimateCardinalityForHashJoin(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index 75b1b2b..8b91ee6 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -26,6 +26,7 @@
 
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogRelationStatistics.hpp"
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ComparisonExpression.hpp"
@@ -38,6 +39,7 @@
 #include "query_optimizer/expressions/PatternMatcher.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/PatternMatcher.hpp"
 #include "query_optimizer/physical/Physical.hpp"
@@ -48,6 +50,8 @@
 #include "query_optimizer/physical/TableGenerator.hpp"
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "types/TypedValue.hpp"
+#include "types/NullType.hpp"
 
 #include "glog/logging.h"
 
@@ -73,6 +77,9 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinality(
     case P::PhysicalType::kTableGenerator:
       return estimateCardinalityForTableGenerator(
           std::static_pointer_cast<const P::TableGenerator>(physical_plan));
+    case P::PhysicalType::kFilterJoin:
+      return estimateCardinalityForFilterJoin(
+          std::static_pointer_cast<const P::FilterJoin>(physical_plan));
     case P::PhysicalType::kHashJoin:
       return estimateCardinalityForHashJoin(
           std::static_pointer_cast<const P::HashJoin>(physical_plan));
@@ -134,6 +141,17 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinalityForTableGenerator(
   return physical_plan->generator_function_handle()->getEstimatedCardinality();
 }
 
+std::size_t StarSchemaSimpleCostModel::estimateCardinalityForFilterJoin(
+    const P::FilterJoinPtr &physical_plan) {
+  double build_side_filter_selectivity =
+      estimateSelectivityForPredicate(physical_plan->build_side_filter_predicate(),
+                                      physical_plan->right());
+  std::size_t left_cardinality = estimateCardinality(physical_plan->left());
+  double right_selectivity = estimateSelectivity(physical_plan->right());
+  return static_cast<std::size_t>(
+      left_cardinality * build_side_filter_selectivity * right_selectivity + 0.5);
+}
+
 std::size_t StarSchemaSimpleCostModel::estimateCardinalityForHashJoin(
     const P::HashJoinPtr &physical_plan) {
   std::size_t left_cardinality = estimateCardinality(physical_plan->left());
@@ -216,6 +234,18 @@ std::size_t StarSchemaSimpleCostModel::estimateNumDistinctValues(
       }
       break;
     }
+    case P::PhysicalType::kFilterJoin: {
+      const P::FilterJoinPtr &filter_join =
+          std::static_pointer_cast<const P::FilterJoin>(physical_plan);
+      if (E::ContainsExprId(filter_join->left()->getOutputAttributes(), attribute_id)) {
+        std::size_t left_child_num_distinct_values =
+            estimateNumDistinctValues(attribute_id, filter_join->left());
+        double right_child_selectivity =
+            estimateSelectivity(filter_join->right());
+        return static_cast<std::size_t>(
+            left_child_num_distinct_values * right_child_selectivity + 0.5);
+      }
+    }
     case P::PhysicalType::kHashJoin: {
       const P::HashJoinPtr &hash_join =
           std::static_pointer_cast<const P::HashJoin>(physical_plan);
@@ -254,6 +284,16 @@ double StarSchemaSimpleCostModel::estimateSelectivity(
       double child_selectivity = estimateSelectivity(selection->input());
       return filter_selectivity * child_selectivity;
     }
+    case P::PhysicalType::kFilterJoin: {
+      const P::FilterJoinPtr &filter_join =
+          std::static_pointer_cast<const P::FilterJoin>(physical_plan);
+      double left_selectivity = estimateSelectivity(filter_join->left());
+      double right_selectivity = estimateSelectivity(filter_join->right());
+      double build_side_filter_selectivity =
+          estimateSelectivityForPredicate(filter_join->build_side_filter_predicate(),
+                                          filter_join->right());
+      return left_selectivity * right_selectivity * build_side_filter_selectivity;
+    }
     case P::PhysicalType::kHashJoin: {
       const P::HashJoinPtr &hash_join =
           std::static_pointer_cast<const P::HashJoin>(physical_plan);
@@ -383,18 +423,118 @@ double StarSchemaSimpleCostModel::estimateSelectivityForPredicate(
 std::size_t StarSchemaSimpleCostModel::getNumDistinctValues(
     const E::ExprId attribute_id,
     const P::TableReferencePtr &table_reference) {
-  const CatalogRelation &relation = *table_reference->relation();
-  const std::vector<E::AttributeReferencePtr> &attributes = table_reference->attribute_list();
-  for (std::size_t i = 0; i < attributes.size(); ++i) {
-    if (attributes[i]->id() == attribute_id) {
-      const CatalogRelationStatistics &stat = relation.getStatistics();
-      if (stat.hasNumDistinctValues(i)) {
-        return stat.getNumDistinctValues(i);
+  const auto rel_attr_id =
+      findCatalogRelationAttributeId(table_reference, attribute_id);
+  if (rel_attr_id != kInvalidAttributeID) {
+    const CatalogRelationStatistics &stat =
+        table_reference->relation()->getStatistics();
+    if (stat.hasNumDistinctValues(rel_attr_id)) {
+      return stat.getNumDistinctValues(rel_attr_id);
+    }
+  }
+  return estimateCardinalityForTableReference(table_reference);
+}
+
+bool StarSchemaSimpleCostModel::impliesUniqueAttributes(
+    const P::PhysicalPtr &physical_plan,
+    const std::vector<E::AttributeReferencePtr> &attributes) {
+  switch (physical_plan->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      const P::AggregatePtr &aggregate =
+          std::static_pointer_cast<const P::Aggregate>(physical_plan);
+      return E::SubsetOfExpressions(aggregate->grouping_expressions(), attributes);
+    }
+    case P::PhysicalType::kHashJoin: {
+      const P::HashJoinPtr &hash_join =
+          std::static_pointer_cast<const P::HashJoin>(physical_plan);
+      bool unique_from_left =
+          impliesUniqueAttributes(hash_join->right(), hash_join->right_join_attributes())
+              && impliesUniqueAttributes(hash_join->left(), attributes);
+      bool unique_from_right =
+          impliesUniqueAttributes(hash_join->left(), hash_join->left_join_attributes())
+              && impliesUniqueAttributes(hash_join->right(), attributes);
+      return unique_from_left || unique_from_right;
+    }
+    case P::PhysicalType::kTableReference: {
+      const P::TableReferencePtr &table_reference =
+          std::static_pointer_cast<const P::TableReference>(physical_plan);
+      const CatalogRelationStatistics &stat =
+          table_reference->relation()->getStatistics();
+      if (stat.hasNumTuples()) {
+        const std::size_t num_tuples = stat.getNumTuples();
+        for (const auto &attr : attributes) {
+          const attribute_id rel_attr_id =
+              findCatalogRelationAttributeId(table_reference, attr->id());
+          if (rel_attr_id != kInvalidAttributeID &&
+              stat.hasNumDistinctValues(rel_attr_id) &&
+              stat.getNumDistinctValues(rel_attr_id) == num_tuples) {
+            return true;
+          }
+        }
       }
+      return false;
+    }
+    case P::PhysicalType::kSample:  // Fall through
+    case P::PhysicalType::kSelection:
+    case P::PhysicalType::kSort: {
+      DCHECK_EQ(physical_plan->getNumChildren(), 1u);
+      return impliesUniqueAttributes(physical_plan->children()[0], attributes);
+    }
+    default:
       break;
+  }
+  return false;
+}
+
+TypedValue StarSchemaSimpleCostModel::findCatalogRelationStat(
+    const P::PhysicalPtr &physical_plan,
+    const E::ExprId attr_id,
+    const StatType stat_type) {
+  P::TableReferencePtr table_reference;
+  if (P::SomeTableReference::MatchesWithConditionalCast(physical_plan, &table_reference)) {
+    const attribute_id rel_attr_id =
+        findCatalogRelationAttributeId(table_reference, attr_id);
+    if (rel_attr_id != kInvalidAttributeID) {
+      const CatalogRelationStatistics &stat =
+          table_reference->relation()->getStatistics();
+      switch (stat_type) {
+        case StatType::kMin: {
+          if (stat.hasMinValue(rel_attr_id)) {
+            return stat.getMinValue(rel_attr_id);
+          }
+          break;
+        }
+        case StatType::kMax: {
+          if (stat.hasMaxValue(rel_attr_id)) {
+            return stat.getMaxValue(rel_attr_id);
+          }
+          break;
+        }
+        default:
+          break;
+      }
+      return NullType::InstanceNullable().makeNullValue();
     }
   }
-  return estimateCardinalityForTableReference(table_reference);
+
+  for (const auto &child : physical_plan->children()) {
+    if (E::ContainsExprId(child->getOutputAttributes(), attr_id)) {
+      return findCatalogRelationStat(child, attr_id, stat_type);
+    }
+  }
+  return NullType::InstanceNullable().makeNullValue();
+}
+
+attribute_id StarSchemaSimpleCostModel::findCatalogRelationAttributeId(
+    const physical::TableReferencePtr &table_reference,
+    const expressions::ExprId expr_id) {
+  const auto &attribute_list = table_reference->attribute_list();
+  for (std::size_t i = 0; i < attribute_list.size(); ++i) {
+    if (attribute_list[i]->id() == expr_id) {
+      return i;
+    }
+  }
+  return kInvalidAttributeID;
 }
 
 }  // namespace cost

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
index 6f6aa29..e65e353 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
@@ -23,11 +23,13 @@
 #include <cstddef>
 #include <vector>
 
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/Predicate.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
 #include "query_optimizer/physical/NestedLoopsJoin.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
 #include "query_optimizer/physical/HashJoin.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/physical/Selection.hpp"
@@ -36,6 +38,7 @@
 #include "query_optimizer/physical/TableReference.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
 #include "query_optimizer/physical/WindowAggregate.hpp"
+#include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -105,10 +108,38 @@ class StarSchemaSimpleCostModel : public CostModel {
   double estimateSelectivityForFilterPredicate(
       const physical::PhysicalPtr &physical_plan);
 
+  bool impliesUniqueAttributes(
+      const physical::PhysicalPtr &physical_plan,
+      const std::vector<expressions::AttributeReferencePtr> &attributes);
+
+  TypedValue findMinValueStat(
+      const physical::PhysicalPtr &physical_plan,
+      const expressions::AttributeReferencePtr &attribute) {
+    return findCatalogRelationStat(
+        physical_plan, attribute->id(), StatType::kMin);
+  }
+
+  TypedValue findMaxValueStat(
+      const physical::PhysicalPtr &physical_plan,
+      const expressions::AttributeReferencePtr &attribute) {
+    return findCatalogRelationStat(
+        physical_plan, attribute->id(), StatType::kMax);
+  }
+
+  template <typename CppType>
+  bool findMinMaxStatsCppValue(
+      const physical::PhysicalPtr &physical_plan,
+      const expressions::AttributeReferencePtr &attribute,
+      CppType *min_cpp_value,
+      CppType *max_cpp_value);
+
  private:
   std::size_t estimateCardinalityForAggregate(
       const physical::AggregatePtr &physical_plan);
 
+  std::size_t estimateCardinalityForFilterJoin(
+      const physical::FilterJoinPtr &physical_plan);
+
   std::size_t estimateCardinalityForHashJoin(
       const physical::HashJoinPtr &physical_plan);
 
@@ -144,10 +175,63 @@ class StarSchemaSimpleCostModel : public CostModel {
   std::size_t getNumDistinctValues(const expressions::ExprId attribute_id,
                                    const physical::TableReferencePtr &table_reference);
 
+  enum class StatType {
+    kMax = 0,
+    kMin
+  };
+
+  TypedValue findCatalogRelationStat(
+      const physical::PhysicalPtr &physical_plan,
+      const expressions::ExprId expr_id,
+      const StatType stat_type);
+
+  attribute_id findCatalogRelationAttributeId(
+      const physical::TableReferencePtr &table_reference,
+      const expressions::ExprId expr_id);
 
   DISALLOW_COPY_AND_ASSIGN(StarSchemaSimpleCostModel);
 };
 
+template <typename CppType>
+bool StarSchemaSimpleCostModel::findMinMaxStatsCppValue(
+    const physical::PhysicalPtr &physical_plan,
+    const expressions::AttributeReferencePtr &attribute,
+    CppType *min_cpp_value,
+    CppType *max_cpp_value) {
+  const TypedValue min_value =
+      findMinValueStat(physical_plan, attribute);
+  const TypedValue max_value =
+      findMaxValueStat(physical_plan, attribute);
+  if (min_value.isNull() || max_value.isNull()) {
+    return false;
+  }
+
+  switch (attribute->getValueType().getTypeID()) {
+    case TypeID::kInt: {
+      *min_cpp_value = min_value.getLiteral<int>();
+      *max_cpp_value = max_value.getLiteral<int>();
+      return true;
+    }
+    case TypeID::kLong: {
+      *min_cpp_value = min_value.getLiteral<std::int64_t>();
+      *max_cpp_value = max_value.getLiteral<std::int64_t>();
+      return true;
+    }
+    case TypeID::kFloat: {
+      *min_cpp_value = min_value.getLiteral<float>();
+      *max_cpp_value = max_value.getLiteral<float>();
+      return true;
+    }
+    case TypeID::kDouble: {
+      *min_cpp_value = min_value.getLiteral<double>();
+      *max_cpp_value = max_value.getLiteral<double>();
+      return true;
+    }
+    default:
+      return false;
+  }
+}
+
 /** @} */
 
 }  // namespace cost

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/physical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/CMakeLists.txt b/query_optimizer/physical/CMakeLists.txt
index 7f26943..f68ed39 100644
--- a/query_optimizer/physical/CMakeLists.txt
+++ b/query_optimizer/physical/CMakeLists.txt
@@ -23,6 +23,7 @@ add_library(quickstep_queryoptimizer_physical_CreateIndex CreateIndex.cpp Create
 add_library(quickstep_queryoptimizer_physical_CreateTable CreateTable.cpp CreateTable.hpp)
 add_library(quickstep_queryoptimizer_physical_DeleteTuples DeleteTuples.cpp DeleteTuples.hpp)
 add_library(quickstep_queryoptimizer_physical_DropTable DropTable.cpp DropTable.hpp)
+add_library(quickstep_queryoptimizer_physical_FilterJoin FilterJoin.cpp FilterJoin.hpp)
 add_library(quickstep_queryoptimizer_physical_HashJoin HashJoin.cpp HashJoin.hpp)
 add_library(quickstep_queryoptimizer_physical_InsertSelection InsertSelection.cpp InsertSelection.hpp)
 add_library(quickstep_queryoptimizer_physical_InsertTuple InsertTuple.cpp InsertTuple.hpp)
@@ -115,6 +116,18 @@ target_link_libraries(quickstep_queryoptimizer_physical_DropTable
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_physical_PhysicalType
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_physical_FilterJoin
+                      glog
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_expressions_Predicate
+                      quickstep_queryoptimizer_physical_BinaryJoin
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_utility_Cast
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_physical_HashJoin
                       glog
                       quickstep_queryoptimizer_OptimizerTree
@@ -282,6 +295,7 @@ target_link_libraries(quickstep_queryoptimizer_physical
                       quickstep_queryoptimizer_physical_CreateTable
                       quickstep_queryoptimizer_physical_DeleteTuples
                       quickstep_queryoptimizer_physical_DropTable
+                      quickstep_queryoptimizer_physical_FilterJoin
                       quickstep_queryoptimizer_physical_HashJoin
                       quickstep_queryoptimizer_physical_InsertSelection
                       quickstep_queryoptimizer_physical_InsertTuple

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/physical/FilterJoin.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/FilterJoin.cpp b/query_optimizer/physical/FilterJoin.cpp
new file mode 100644
index 0000000..1817a1c
--- /dev/null
+++ b/query_optimizer/physical/FilterJoin.cpp
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "query_optimizer/physical/FilterJoin.hpp"
+
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "utility/Cast.hpp"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+namespace E = ::quickstep::optimizer::expressions;
+
+std::vector<E::AttributeReferencePtr> FilterJoin::getReferencedAttributes() const {
+  std::vector<E::AttributeReferencePtr> referenced_attributes;
+  for (const auto &project_expression : project_expressions()) {
+    const auto referenced_attributes_in_expression =
+        project_expression->getReferencedAttributes();
+    referenced_attributes.insert(referenced_attributes.end(),
+                                 referenced_attributes_in_expression.begin(),
+                                 referenced_attributes_in_expression.end());
+  }
+  referenced_attributes.insert(referenced_attributes.end(),
+                               probe_attributes_.begin(),
+                               probe_attributes_.end());
+  referenced_attributes.insert(referenced_attributes.end(),
+                               build_attributes_.begin(),
+                               build_attributes_.end());
+  if (build_side_filter_predicate_ != nullptr) {
+    const auto referenced_attributes_in_predicate =
+        build_side_filter_predicate_->getReferencedAttributes();
+    referenced_attributes.insert(referenced_attributes.end(),
+                                 referenced_attributes_in_predicate.begin(),
+                                 referenced_attributes_in_predicate.end());
+  }
+  return referenced_attributes;
+}
+
+bool FilterJoin::maybeCopyWithPrunedExpressions(
+    const expressions::UnorderedNamedExpressionSet &referenced_expressions,
+    PhysicalPtr *output) const {
+  std::vector<E::NamedExpressionPtr> new_project_expressions;
+  const auto &current_project_expressions = project_expressions();
+  for (const auto &project_expression : current_project_expressions) {
+    if (referenced_expressions.find(project_expression) != referenced_expressions.end()) {
+      new_project_expressions.emplace_back(project_expression);
+    }
+  }
+  if (new_project_expressions.size() != current_project_expressions.size()) {
+    *output = Create(left(),
+                     right(),
+                     probe_attributes_,
+                     build_attributes_,
+                     new_project_expressions,
+                     build_side_filter_predicate_,
+                     is_anti_join_);
+    return true;
+  }
+  return false;
+}
+
+void FilterJoin::getFieldStringItems(
+    std::vector<std::string> *inline_field_names,
+    std::vector<std::string> *inline_field_values,
+    std::vector<std::string> *non_container_child_field_names,
+    std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
+  BinaryJoin::getFieldStringItems(inline_field_names,
+                                  inline_field_values,
+                                  non_container_child_field_names,
+                                  non_container_child_fields,
+                                  container_child_field_names,
+                                  container_child_fields);
+
+  inline_field_names->push_back("is_anti_join");
+  inline_field_values->push_back(std::to_string(is_anti_join_));
+
+  if (build_side_filter_predicate_ != nullptr) {
+    non_container_child_field_names->emplace_back("build_side_filter_predicate");
+    non_container_child_fields->emplace_back(build_side_filter_predicate_);
+  }
+
+  container_child_field_names->push_back("probe_attributes");
+  container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(probe_attributes_));
+  container_child_field_names->push_back("build_attributes");
+  container_child_fields->push_back(CastSharedPtrVector<OptimizerTreeBase>(build_attributes_));
+}
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/physical/FilterJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/FilterJoin.hpp b/query_optimizer/physical/FilterJoin.hpp
new file mode 100644
index 0000000..3d3fc39
--- /dev/null
+++ b/query_optimizer/physical/FilterJoin.hpp
@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_FILTER_JOIN_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_FILTER_JOIN_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/physical/BinaryJoin.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace physical {
+
+/** \addtogroup OptimizerPhysical
+ *  @{
+ */
+
+class FilterJoin;
+typedef std::shared_ptr<const FilterJoin> FilterJoinPtr;
+
+/**
+ * @brief Physical filter join node.
+ */
+class FilterJoin : public BinaryJoin {
+ public:
+  PhysicalType getPhysicalType() const override { return PhysicalType::kFilterJoin; }
+
+  std::string getName() const override {
+    if (is_anti_join_) {
+      return "FilterJoin(Anti)";
+    } else {
+      return "FilterJoin";
+    }
+  }
+
+  const std::vector<expressions::AttributeReferencePtr>& probe_attributes() const {
+    return probe_attributes_;
+  }
+
+  const std::vector<expressions::AttributeReferencePtr>& build_attributes() const {
+    return build_attributes_;
+  }
+
+  const expressions::PredicatePtr& build_side_filter_predicate() const {
+    return build_side_filter_predicate_;
+  }
+
+  const bool is_anti_join() const {
+    return is_anti_join_;
+  }
+
+  PhysicalPtr copyWithNewChildren(
+      const std::vector<PhysicalPtr> &new_children) const override {
+    DCHECK_EQ(children().size(), new_children.size());
+    return Create(new_children[0],
+                  new_children[1],
+                  probe_attributes_,
+                  build_attributes_,
+                  project_expressions(),
+                  build_side_filter_predicate_,
+                  is_anti_join_);
+  }
+
+  std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override;
+
+  bool maybeCopyWithPrunedExpressions(
+      const expressions::UnorderedNamedExpressionSet &referenced_expressions,
+      PhysicalPtr *output) const override;
+
+  static FilterJoinPtr Create(
+      const PhysicalPtr &probe_child,
+      const PhysicalPtr &build_child,
+      const std::vector<expressions::AttributeReferencePtr> &probe_attributes,
+      const std::vector<expressions::AttributeReferencePtr> &build_attributes,
+      const std::vector<expressions::NamedExpressionPtr> &project_expressions,
+      const expressions::PredicatePtr &build_side_filter_predicate,
+      const bool is_anti_join) {
+    return FilterJoinPtr(
+        new FilterJoin(probe_child,
+                       build_child,
+                       probe_attributes,
+                       build_attributes,
+                       project_expressions,
+                       build_side_filter_predicate,
+                       is_anti_join));
+  }
+
+ protected:
+  void getFieldStringItems(
+      std::vector<std::string> *inline_field_names,
+      std::vector<std::string> *inline_field_values,
+      std::vector<std::string> *non_container_child_field_names,
+      std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const override;
+
+ private:
+  FilterJoin(
+      const PhysicalPtr &probe_child,
+      const PhysicalPtr &build_child,
+      const std::vector<expressions::AttributeReferencePtr> &probe_attributes,
+      const std::vector<expressions::AttributeReferencePtr> &build_attributes,
+      const std::vector<expressions::NamedExpressionPtr> &project_expressions,
+      const expressions::PredicatePtr &build_side_filter_predicate,
+      const bool is_anti_join)
+      : BinaryJoin(probe_child, build_child, project_expressions),
+        probe_attributes_(probe_attributes),
+        build_attributes_(build_attributes),
+        build_side_filter_predicate_(build_side_filter_predicate),
+        is_anti_join_(is_anti_join) {
+  }
+
+  std::vector<expressions::AttributeReferencePtr> probe_attributes_;
+  std::vector<expressions::AttributeReferencePtr> build_attributes_;
+  expressions::PredicatePtr build_side_filter_predicate_;
+  bool is_anti_join_;
+
+  DISALLOW_COPY_AND_ASSIGN(FilterJoin);
+};
+
+/** @} */
+
+}  // namespace physical
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_FILTER_JOIN_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/physical/LIPFilterConfiguration.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/LIPFilterConfiguration.hpp b/query_optimizer/physical/LIPFilterConfiguration.hpp
index 62a6149..b4ae548 100644
--- a/query_optimizer/physical/LIPFilterConfiguration.hpp
+++ b/query_optimizer/physical/LIPFilterConfiguration.hpp
@@ -50,17 +50,32 @@ struct LIPFilterBuildInfo {
    * @param build_attribute_in The attribute to build the LIP filter with.
    * @param filter_cardinality_in The LIP filter's cardinality.
    * @param filter_type_in The LIP filter's type.
+   * @param is_anti_filter_in Whether this LIPFilter is an anti-filter.
    */
   LIPFilterBuildInfo(const expressions::AttributeReferencePtr &build_attribute_in,
                      const std::size_t filter_cardinality_in,
-                     const LIPFilterType &filter_type_in)
+                     const LIPFilterType &filter_type_in,
+                     const bool is_anti_filter_in)
       : build_attribute(build_attribute_in),
         filter_cardinality(filter_cardinality_in),
-        filter_type(filter_type_in) {
-  }
+        filter_type(filter_type_in),
+        is_anti_filter(is_anti_filter_in) {}
+
+  /**
+   * @brief Copy constructor.
+   *
+   * @param info The LIPFilter build info to copy.
+   */
+  LIPFilterBuildInfo(const LIPFilterBuildInfo &info)
+      : build_attribute(info.build_attribute),
+        filter_cardinality(info.filter_cardinality),
+        filter_type(info.filter_type),
+        is_anti_filter(info.is_anti_filter) {}
+
   const expressions::AttributeReferencePtr build_attribute;
   const std::size_t filter_cardinality;
   const LIPFilterType filter_type;
+  const bool is_anti_filter;
 };
 
 /**
@@ -79,8 +94,18 @@ struct LIPFilterProbeInfo {
                      const PhysicalPtr &builder_in)
       : probe_attribute(probe_attribute_in),
         build_attribute(build_attribute_in),
-        builder(builder_in) {
-  }
+        builder(builder_in) {}
+
+  /**
+   * @brief Copy constructor.
+   *
+   * @param info The LIPFilter probe info to copy.
+   */
+  LIPFilterProbeInfo(const LIPFilterProbeInfo &info)
+      : probe_attribute(info.probe_attribute),
+        build_attribute(info.build_attribute),
+        builder(info.builder) {}
+
   const expressions::AttributeReferencePtr probe_attribute;
   const expressions::AttributeReferencePtr build_attribute;
   const PhysicalPtr builder;
@@ -112,9 +137,10 @@ class LIPFilterConfiguration {
   void addBuildInfo(const expressions::AttributeReferencePtr &build_attribute,
                     const PhysicalPtr &builder,
                     const std::size_t filter_size,
-                    const LIPFilterType &filter_type) {
+                    const LIPFilterType &filter_type,
+                    const bool is_anti_filter) {
     build_info_map_[builder].emplace_back(
-        build_attribute, filter_size, filter_type);
+        build_attribute, filter_size, filter_type, is_anti_filter);
   }
 
   /**
@@ -155,6 +181,31 @@ class LIPFilterConfiguration {
     return probe_info_map_;
   }
 
+  /**
+   * @brief Clone a copy of this configuration.
+   *
+   * @return A copy of this confiugration. Caller should take ownership of the
+   *         returned object.
+   */
+  LIPFilterConfiguration* clone() const {
+    LIPFilterConfiguration *new_conf = new LIPFilterConfiguration();
+    for (const auto &build_pair : build_info_map_) {
+      auto &new_build_vec = new_conf->build_info_map_[build_pair.first];
+      const auto &build_vec = build_pair.second;
+      for (const auto &info : build_vec) {
+        new_build_vec.emplace_back(info);
+      }
+    }
+    for (const auto &probe_pair : probe_info_map_) {
+      auto &new_probe_vec = new_conf->probe_info_map_[probe_pair.first];
+      const auto &probe_vec = probe_pair.second;
+      for (const auto &info : probe_vec) {
+        new_probe_vec.emplace_back(info);
+      }
+    }
+    return new_conf;
+  }
+
  private:
   std::map<PhysicalPtr, std::vector<LIPFilterBuildInfo>> build_info_map_;
   std::map<PhysicalPtr, std::vector<LIPFilterProbeInfo>> probe_info_map_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/physical/PatternMatcher.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/PatternMatcher.hpp b/query_optimizer/physical/PatternMatcher.hpp
index 5cd6fd3..4336767 100644
--- a/query_optimizer/physical/PatternMatcher.hpp
+++ b/query_optimizer/physical/PatternMatcher.hpp
@@ -35,6 +35,7 @@ class CopyFrom;
 class CreateTable;
 class DeleteTuples;
 class DropTable;
+class FilterJoin;
 class HashJoin;
 class InsertTuple;
 class Join;
@@ -113,6 +114,7 @@ using SomeCopyFrom = SomePhysicalNode<CopyFrom, PhysicalType::kCopyFrom>;
 using SomeCreateTable = SomePhysicalNode<CreateTable, PhysicalType::kCreateTable>;
 using SomeDeleteTuples = SomePhysicalNode<DeleteTuples, PhysicalType::kDeleteTuples>;
 using SomeDropTable = SomePhysicalNode<DropTable, PhysicalType::kDropTable>;
+using SomeFilterJoin = SomePhysicalNode<FilterJoin, PhysicalType::kFilterJoin>;
 using SomeHashJoin = SomePhysicalNode<HashJoin, PhysicalType::kHashJoin>;
 using SomeInsertTuple = SomePhysicalNode<InsertTuple, PhysicalType::kInsertTuple>;
 using SomeJoin = SomePhysicalNode<Join, PhysicalType::kHashJoin, PhysicalType::kNestedLoopsJoin>;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/physical/PhysicalType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/PhysicalType.hpp b/query_optimizer/physical/PhysicalType.hpp
index f5f35a1..1da5929 100644
--- a/query_optimizer/physical/PhysicalType.hpp
+++ b/query_optimizer/physical/PhysicalType.hpp
@@ -38,6 +38,7 @@ enum class PhysicalType {
   kCreateTable,
   kDeleteTuples,
   kDropTable,
+  kFilterJoin,
   kHashJoin,
   kInsertSelection,
   kInsertTuple,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/physical/TopLevelPlan.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/TopLevelPlan.hpp b/query_optimizer/physical/TopLevelPlan.hpp
index 7dfc2b6..9e567e1 100644
--- a/query_optimizer/physical/TopLevelPlan.hpp
+++ b/query_optimizer/physical/TopLevelPlan.hpp
@@ -126,7 +126,8 @@ class TopLevelPlan : public Physical {
     }
     return TopLevelPlan::Create(new_children[0],
                                 new_shared_subplans,
-                                uncorrelated_subquery_map_);
+                                uncorrelated_subquery_map_,
+                                lip_filter_configuration_);
   }
 
   std::vector<expressions::AttributeReferencePtr> getOutputAttributes() const override {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/rules/AttachLIPFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/AttachLIPFilters.cpp b/query_optimizer/rules/AttachLIPFilters.cpp
index b3c57ab..c4551bc 100644
--- a/query_optimizer/rules/AttachLIPFilters.cpp
+++ b/query_optimizer/rules/AttachLIPFilters.cpp
@@ -55,7 +55,14 @@ P::PhysicalPtr AttachLIPFilters::apply(const P::PhysicalPtr &input) {
   cost_model_.reset(
       new cost::StarSchemaSimpleCostModel(
           top_level_plan->shared_subplans()));
-  lip_filter_configuration_.reset(new P::LIPFilterConfiguration());
+
+  const P::LIPFilterConfigurationPtr &existing_configuration =
+      top_level_plan->lip_filter_configuration();
+  if (existing_configuration != nullptr) {
+    lip_filter_configuration_.reset(existing_configuration->clone());
+  } else {
+    lip_filter_configuration_.reset(new P::LIPFilterConfiguration());
+  }
 
   std::set<E::ExprId> already_filtered_attributes;
   attachLIPFilters(NodeList(input), &already_filtered_attributes);
@@ -101,7 +108,7 @@ void AttachLIPFilters::attachLIPFilters(
   }
 
   if (probe_child != nullptr &&
-      cost_model_->estimateCardinality(probe_child) > 10000000) {
+      cost_model_->estimateCardinality(probe_child) >= 100000) {
     const auto &candidate_lip_filters = getProbeSideInfo(path.cons(probe_child));
     if (!candidate_lip_filters.empty()) {
       std::map<E::AttributeReferencePtr, LIPFilterInfoPtr> selected_filters;
@@ -122,7 +129,8 @@ void AttachLIPFilters::attachLIPFilters(
               pair.second->source_attribute,
               pair.second->source,
               pair.second->estimated_cardinality * 8,
-              LIPFilterType::kSingleIdentityHashFilter);
+              LIPFilterType::kSingleIdentityHashFilter,
+              false);
           lip_filter_configuration_->addProbeInfo(
               pair.first,
               node,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 7fffadc..b4d0f20 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -22,6 +22,7 @@ add_library(quickstep_queryoptimizer_rules_AttachLIPFilters AttachLIPFilters.cpp
 add_library(quickstep_queryoptimizer_rules_BottomUpRule ../../empty_src.cpp BottomUpRule.hpp)
 add_library(quickstep_queryoptimizer_rules_CollapseProject CollapseProject.cpp CollapseProject.hpp)
 add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp GenerateJoins.hpp)
+add_library(quickstep_queryoptimizer_rules_InjectJoinFilters InjectJoinFilters.cpp InjectJoinFilters.hpp)
 add_library(quickstep_queryoptimizer_rules_PruneColumns PruneColumns.cpp PruneColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownFilter PushDownFilter.cpp PushDownFilter.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin PushDownSemiAntiJoin.cpp PushDownSemiAntiJoin.hpp)
@@ -160,6 +161,26 @@ target_link_libraries(quickstep_queryoptimizer_rules_SwapProbeBuild
 target_link_libraries(quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_InjectJoinFilters
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExprId
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_FilterJoin
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_LIPFilterConfiguration
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_queryoptimizer_rules_PruneColumns
+                      quickstep_types_TypeID
+                      quickstep_types_TypedValue
+                      quickstep_utility_Macros
+                      quickstep_utility_lipfilter_LIPFilter)
 target_link_libraries(quickstep_queryoptimizer_rules_UnnestSubqueries
                       quickstep_queryoptimizer_OptimizerContext
                       quickstep_queryoptimizer_expressions_AttributeReference
@@ -210,6 +231,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_BottomUpRule
                       quickstep_queryoptimizer_rules_CollapseProject
                       quickstep_queryoptimizer_rules_GenerateJoins
+                      quickstep_queryoptimizer_rules_InjectJoinFilters
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownFilter
                       quickstep_queryoptimizer_rules_PushDownSemiAntiJoin

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d52b9126/query_optimizer/rules/InjectJoinFilters.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/InjectJoinFilters.cpp b/query_optimizer/rules/InjectJoinFilters.cpp
new file mode 100644
index 0000000..3d35382
--- /dev/null
+++ b/query_optimizer/rules/InjectJoinFilters.cpp
@@ -0,0 +1,399 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "query_optimizer/rules/InjectJoinFilters.hpp"
+
+#include <cstddef>
+#include <cstdint>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/physical/LIPFilterConfiguration.hpp"
+#include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/FilterJoin.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/PatternMatcher.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "query_optimizer/physical/Selection.hpp"
+#include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/rules/PruneColumns.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "utility/lip_filter/LIPFilter.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr InjectJoinFilters::apply(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  const P::TopLevelPlanPtr top_level_plan =
+     std::static_pointer_cast<const P::TopLevelPlan>(input);
+  cost_model_.reset(
+      new cost::StarSchemaSimpleCostModel(
+          top_level_plan->shared_subplans()));
+  lip_filter_configuration_.reset(new P::LIPFilterConfiguration());
+
+  P::PhysicalPtr output = transformHashJoinToFilters(input);
+  output = pushDownFilters(output);
+  output = addFilterAnchors(output, false);
+  output = PruneColumns().apply(output);
+
+  concretizeAsLIPFilters(output, nullptr);
+
+  if (!lip_filter_configuration_->getBuildInfoMap().empty() ||
+      !lip_filter_configuration_->getProbeInfoMap().empty()) {
+    output = std::static_pointer_cast<const P::TopLevelPlan>(output)
+        ->copyWithLIPFilterConfiguration(
+              P::LIPFilterConfigurationPtr(lip_filter_configuration_.release()));
+  }
+
+  return output;
+}
+
+bool InjectJoinFilters::isTransformable(
+    const physical::HashJoinPtr &hash_join) const {
+  if (hash_join->residual_predicate() != nullptr) {
+    return false;
+  }
+  if (hash_join->right_join_attributes().size() > 1) {
+    return false;
+  }
+  if (!E::SubsetOfExpressions(hash_join->getOutputAttributes(),
+                              hash_join->left()->getOutputAttributes())) {
+    return false;
+  }
+  switch (hash_join->join_type()) {
+    case P::HashJoin::JoinType::kInnerJoin: {
+      if (!cost_model_->impliesUniqueAttributes(hash_join->right(),
+                                                hash_join->right_join_attributes())) {
+        return false;
+      }
+      break;
+    }
+    case P::HashJoin::JoinType::kLeftSemiJoin:  // Fall through
+    case P::HashJoin::JoinType::kLeftAntiJoin:
+      break;
+    default:
+      return false;
+  }
+
+  std::int64_t min_cpp_value;
+  std::int64_t max_cpp_value;
+  const bool has_min_max_stats =
+      findMinMaxValuesForAttributeHelper(hash_join->right(),
+                                         hash_join->right_join_attributes().front(),
+                                         &min_cpp_value,
+                                         &max_cpp_value);
+  if (!has_min_max_stats) {
+    return false;
+  }
+
+  // TODO(jianqiao): implement SimpleHashSetExactFilter to relax this requirement.
+  // Note that 1G bits = 128MB.
+  if (min_cpp_value < 0 || max_cpp_value > kMaxFilterSize) {
+    return false;
+  }
+
+  return true;
+}
+
+P::PhysicalPtr InjectJoinFilters::transformHashJoinToFilters(
+    const P::PhysicalPtr &input) const {
+  std::vector<P::PhysicalPtr> new_children;
+  bool has_changed_children = false;
+  for (const P::PhysicalPtr &child : input->children()) {
+    const P::PhysicalPtr new_child = transformHashJoinToFilters(child);
+    if (child != new_child && !has_changed_children) {
+      has_changed_children = true;
+    }
+    new_children.push_back(new_child);
+  }
+
+  P::HashJoinPtr hash_join;
+  if (P::SomeHashJoin::MatchesWithConditionalCast(input, &hash_join) &&
+      isTransformable(hash_join)) {
+    const bool is_anti_join =
+        hash_join->join_type() == P::HashJoin::JoinType::kLeftAntiJoin;
+
+    P::PhysicalPtr build_child = new_children[1];
+    E::PredicatePtr build_side_filter_predicate = nullptr;
+    P::SelectionPtr selection;
+    if (P::SomeSelection::MatchesWithConditionalCast(build_child, &selection) &&
+        E::SubsetOfExpressions(hash_join->right_join_attributes(),
+                               selection->input()->getOutputAttributes())) {
+      build_child = selection->input();
+      build_side_filter_predicate = selection->filter_predicate();
+    }
+
+    return P::FilterJoin::Create(new_children[0],
+                                 build_child,
+                                 hash_join->left_join_attributes(),
+                                 hash_join->right_join_attributes(),
+                                 hash_join->project_expressions(),
+                                 build_side_filter_predicate,
+                                 is_anti_join);
+  }
+
+  if (has_changed_children) {
+    return input->copyWithNewChildren(new_children);
+  } else {
+    return input;
+  }
+}
+
+physical::PhysicalPtr InjectJoinFilters::pushDownFilters(
+    const physical::PhysicalPtr &input) const {
+  std::vector<P::PhysicalPtr> new_children;
+  bool has_changed_children = false;
+  for (const P::PhysicalPtr &child : input->children()) {
+    const P::PhysicalPtr new_child = pushDownFilters(child);
+    if (child != new_child && !has_changed_children) {
+      has_changed_children = true;
+    }
+    new_children.push_back(new_child);
+  }
+
+  P::FilterJoinPtr filter_join;
+  if (P::SomeFilterJoin::MatchesWithConditionalCast(input, &filter_join)) {
+    DCHECK_EQ(2u, new_children.size());
+    return pushDownFiltersInternal(
+        new_children[0], new_children[1], filter_join);
+  }
+
+  if (has_changed_children) {
+    return input->copyWithNewChildren(new_children);
+  } else {
+    return input;
+  }
+}
+
+physical::PhysicalPtr InjectJoinFilters::pushDownFiltersInternal(
+    const physical::PhysicalPtr &probe_child,
+    const physical::PhysicalPtr &build_child,
+    const physical::FilterJoinPtr &filter_join) const {
+  switch (probe_child->getPhysicalType()) {
+    case P::PhysicalType::kAggregate:  // Fall through
+    case P::PhysicalType::kHashJoin:
+    case P::PhysicalType::kSample:
+    case P::PhysicalType::kSelection:
+    case P::PhysicalType::kSort:
+    case P::PhysicalType::kWindowAggregate: {
+      DCHECK_GE(probe_child->getNumChildren(), 1u);
+      const P::PhysicalPtr child = probe_child->children()[0];
+      if (E::SubsetOfExpressions(filter_join->probe_attributes(),
+                                 child->getOutputAttributes())) {
+        const P::PhysicalPtr new_child =
+            pushDownFiltersInternal(child, build_child, filter_join);
+        if (new_child != child) {
+          std::vector<P::PhysicalPtr> new_children = probe_child->children();
+          new_children[0] = new_child;
+          return probe_child->copyWithNewChildren(new_children);
+        }
+      }
+    }
+    default:
+      break;
+  }
+
+  if (probe_child != filter_join->left()) {
+    // TODO(jianqiao): may need to update probe_attributes.
+    return P::FilterJoin::Create(probe_child,
+                                 build_child,
+                                 filter_join->probe_attributes(),
+                                 filter_join->build_attributes(),
+                                 E::ToNamedExpressions(probe_child->getOutputAttributes()),
+                                 filter_join->build_side_filter_predicate(),
+                                 filter_join->is_anti_join());
+  } else {
+    return filter_join;
+  }
+}
+
+
+physical::PhysicalPtr InjectJoinFilters::addFilterAnchors(
+    const physical::PhysicalPtr &input,
+    const bool ancestor_can_anchor_filter) const {
+  std::vector<P::PhysicalPtr> new_children;
+
+  switch (input->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      const P::AggregatePtr &aggregate =
+          std::static_pointer_cast<const P::Aggregate>(input);
+      new_children.emplace_back(
+          addFilterAnchors(aggregate->input(), true));
+      break;
+    }
+    case P::PhysicalType::kSelection: {
+      const P::SelectionPtr &selection =
+          std::static_pointer_cast<const P::Selection>(input);
+      new_children.emplace_back(
+          addFilterAnchors(selection->input(), true));
+      break;
+    }
+//    case P::PhysicalType::kHashJoin: {
+//      const P::HashJoinPtr &hash_join =
+//          std::static_pointer_cast<const P::HashJoin>(input);
+//      new_children.emplace_back(
+//          addFilterAnchors(hash_join->left(), true));
+//      new_children.emplace_back(
+//          addFilterAnchors(hash_join->right(), false));
+//      break;
+//    }
+    case P::PhysicalType::kFilterJoin: {
+      const P::FilterJoinPtr &filter_join =
+          std::static_pointer_cast<const P::FilterJoin>(input);
+      new_children.emplace_back(
+          addFilterAnchors(filter_join->left(), true));
+      new_children.emplace_back(
+          addFilterAnchors(filter_join->right(), true));
+      break;
+    }
+    default: {
+      for (const P::PhysicalPtr &child : input->children()) {
+        new_children.emplace_back(addFilterAnchors(child, false));
+      }
+    }
+  }
+
+  DCHECK_EQ(new_children.size(), input->children().size());
+  const P::PhysicalPtr output_with_new_children =
+      new_children == input->children()
+          ? input
+          : input->copyWithNewChildren(new_children);
+
+  if (input->getPhysicalType() == P::PhysicalType::kFilterJoin &&
+      !ancestor_can_anchor_filter) {
+    const P::FilterJoinPtr &filter_join =
+        std::static_pointer_cast<const P::FilterJoin>(output_with_new_children);
+    return P::Selection::Create(filter_join,
+                                filter_join->project_expressions(),
+                                nullptr);
+  } else {
+    return output_with_new_children;
+  }
+}
+
+void InjectJoinFilters::concretizeAsLIPFilters(
+    const P::PhysicalPtr &input,
+    const P::PhysicalPtr &anchor_node) const {
+  switch (input->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      const P::AggregatePtr &aggregate =
+          std::static_pointer_cast<const P::Aggregate>(input);
+      concretizeAsLIPFilters(aggregate->input(), aggregate);
+      break;
+    }
+    case P::PhysicalType::kSelection: {
+      const P::SelectionPtr &selection =
+          std::static_pointer_cast<const P::Selection>(input);
+      concretizeAsLIPFilters(selection->input(), selection);
+      break;
+    }
+//    case P::PhysicalType::kHashJoin: {
+//      const P::HashJoinPtr &hash_join =
+//          std::static_pointer_cast<const P::HashJoin>(input);
+//      concretizeAsLIPFilters(hash_join->left(), hash_join);
+//      concretizeAsLIPFilters(hash_join->right(), nullptr);
+//      break;
+//    }
+    case P::PhysicalType::kFilterJoin: {
+      const P::FilterJoinPtr &filter_join =
+          std::static_pointer_cast<const P::FilterJoin>(input);
+      DCHECK_EQ(1u, filter_join->build_attributes().size());
+      const E::AttributeReferencePtr &build_attr =
+          filter_join->build_attributes().front();
+
+      std::int64_t min_cpp_value;
+      std::int64_t max_cpp_value;
+      const bool has_min_max_stats =
+          findMinMaxValuesForAttributeHelper(filter_join,
+                                             build_attr,
+                                             &min_cpp_value,
+                                             &max_cpp_value);
+      DCHECK(has_min_max_stats);
+      DCHECK_GE(min_cpp_value, 0);
+      DCHECK_GE(max_cpp_value, 0);
+      DCHECK_LE(max_cpp_value, kMaxFilterSize);
+      CHECK(anchor_node != nullptr);
+
+      lip_filter_configuration_->addBuildInfo(
+          build_attr,
+          filter_join,
+          static_cast<std::size_t>(max_cpp_value),
+          LIPFilterType::kBitVectorExactFilter,
+          filter_join->is_anti_join());
+      lip_filter_configuration_->addProbeInfo(
+          filter_join->probe_attributes().front(),
+          anchor_node,
+          build_attr,
+          filter_join);
+
+      concretizeAsLIPFilters(filter_join->left(), anchor_node);
+      concretizeAsLIPFilters(filter_join->right(), filter_join);
+      break;
+    }
+    default: {
+      for (const P::PhysicalPtr &child : input->children()) {
+        concretizeAsLIPFilters(child, nullptr);
+      }
+    }
+  }
+}
+
+bool InjectJoinFilters::findMinMaxValuesForAttributeHelper(
+    const physical::PhysicalPtr &physical_plan,
+    const expressions::AttributeReferencePtr &attribute,
+    std::int64_t *min_cpp_value,
+    std::int64_t *max_cpp_value) const {
+  const TypedValue min_value =
+      cost_model_->findMinValueStat(physical_plan, attribute);
+  const TypedValue max_value =
+      cost_model_->findMaxValueStat(physical_plan, attribute);
+  if (min_value.isNull() || max_value.isNull()) {
+    return false;
+  }
+
+  switch (attribute->getValueType().getTypeID()) {
+    case TypeID::kInt: {
+      *min_cpp_value = min_value.getLiteral<int>();
+      *max_cpp_value = max_value.getLiteral<int>();
+      return true;
+    }
+    case TypeID::kLong: {
+      *min_cpp_value = min_value.getLiteral<std::int64_t>();
+      *max_cpp_value = max_value.getLiteral<std::int64_t>();
+      return true;
+    }
+    default:
+      return false;
+  }
+}
+
+
+}  // namespace optimizer
+}  // namespace quickstep



[08/22] incubator-quickstep git commit: Fixed the bug in addBlockToPartition.

Posted by ji...@apache.org.
Fixed the bug in addBlockToPartition.


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

Branch: refs/heads/exact-filter
Commit: 8d25d43b46ffb4b7b43efc0ed4f49a74fb9607b6
Parents: 86411b9
Author: Zuyu Zhang <zu...@apache.org>
Authored: Wed Jan 11 13:36:39 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Wed Jan 11 13:36:39 2017 -0800

----------------------------------------------------------------------
 query_execution/PolicyEnforcerBase.cpp | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8d25d43b/query_execution/PolicyEnforcerBase.cpp
----------------------------------------------------------------------
diff --git a/query_execution/PolicyEnforcerBase.cpp b/query_execution/PolicyEnforcerBase.cpp
index b799d5f..1a2ab46 100644
--- a/query_execution/PolicyEnforcerBase.cpp
+++ b/query_execution/PolicyEnforcerBase.cpp
@@ -105,8 +105,7 @@ void PolicyEnforcerBase::processMessage(const TaggedMessage &tagged_message) {
       relation->addBlock(block);
 
       if (proto.has_partition_id()) {
-        relation->getPartitionSchemeMutable()->addBlockToPartition(
-            proto.partition_id(), block);
+        relation->getPartitionSchemeMutable()->addBlockToPartition(block, proto.partition_id());
       }
       return;
     }


[06/22] incubator-quickstep git commit: Refactored partition_type in parser.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/86411b9c/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index 283bcbe..23cbf1b 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -1,8 +1,8 @@
-/* A Bison parser, made by GNU Bison 3.0.2.  */
+/* A Bison parser, made by GNU Bison 3.0.4.  */
 
 /* Bison implementation for Yacc-like parsers in C
 
-   Copyright (C) 1984, 1989-1990, 2000-2013 Free Software Foundation, Inc.
+   Copyright (C) 1984, 1989-1990, 2000-2015 Free Software Foundation, Inc.
 
    This program is free software: you can redistribute it and/or modify
    it under the terms of the GNU General Public License as published by
@@ -44,7 +44,7 @@
 #define YYBISON 1
 
 /* Bison version.  */
-#define YYBISON_VERSION "3.0.2"
+#define YYBISON_VERSION "3.0.4"
 
 /* Skeleton name.  */
 #define YYSKELETON_NAME "yacc.c"
@@ -323,7 +323,7 @@ extern int quickstep_yydebug;
 
 /* Value type.  */
 #if ! defined YYSTYPE && ! defined YYSTYPE_IS_DECLARED
-typedef union YYSTYPE YYSTYPE;
+
 union YYSTYPE
 {
 #line 120 "../SqlParser.ypp" /* yacc.c:355  */
@@ -428,6 +428,8 @@ union YYSTYPE
 
 #line 430 "SqlParser_gen.cpp" /* yacc.c:355  */
 };
+
+typedef union YYSTYPE YYSTYPE;
 # define YYSTYPE_IS_TRIVIAL 1
 # define YYSTYPE_IS_DECLARED 1
 #endif
@@ -459,7 +461,7 @@ int quickstep_yyparse (yyscan_t yyscanner, quickstep::ParseStatement **parsedSta
 #include "SqlLexer_gen.hpp"
 void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string &feature);
 
-#line 463 "SqlParser_gen.cpp" /* yacc.c:358  */
+#line 465 "SqlParser_gen.cpp" /* yacc.c:358  */
 
 #ifdef short
 # undef short
@@ -778,28 +780,28 @@ static const yytype_uint16 yyrline[] =
      794,   797,   800,   812,   815,   818,   836,   856,   859,   862,
      867,   872,   878,   884,   893,   897,   903,   906,   911,   916,
      921,   928,   935,   939,   945,   948,   953,   956,   961,   964,
-     969,   972,   991,   995,  1001,  1005,  1011,  1014,  1017,  1022,
-    1025,  1032,  1037,  1048,  1053,  1057,  1061,  1067,  1070,  1076,
-    1084,  1087,  1090,  1096,  1101,  1104,  1109,  1113,  1117,  1121,
-    1127,  1132,  1137,  1141,  1147,  1153,  1156,  1161,  1164,  1183,
-    1188,  1192,  1198,  1204,  1210,  1213,  1217,  1223,  1226,  1231,
-    1235,  1241,  1244,  1247,  1252,  1257,  1262,  1265,  1268,  1273,
-    1276,  1279,  1282,  1285,  1288,  1291,  1294,  1299,  1302,  1307,
-    1311,  1315,  1318,  1322,  1325,  1330,  1333,  1338,  1341,  1346,
-    1350,  1356,  1359,  1364,  1367,  1372,  1375,  1380,  1383,  1402,
-    1405,  1410,  1414,  1420,  1426,  1431,  1434,  1439,  1442,  1447,
-    1450,  1455,  1458,  1463,  1464,  1467,  1472,  1473,  1476,  1481,
-    1485,  1491,  1498,  1501,  1504,  1509,  1512,  1515,  1521,  1524,
-    1529,  1534,  1543,  1548,  1557,  1562,  1565,  1570,  1573,  1578,
-    1584,  1590,  1593,  1596,  1599,  1602,  1605,  1611,  1620,  1623,
-    1628,  1631,  1636,  1639,  1644,  1647,  1650,  1653,  1657,  1661,
-    1664,  1667,  1670,  1673,  1678,  1682,  1686,  1689,  1694,  1699,
-    1703,  1709,  1712,  1717,  1721,  1727,  1732,  1736,  1742,  1747,
-    1750,  1755,  1759,  1765,  1768,  1771,  1774,  1786,  1790,  1809,
-    1822,  1837,  1840,  1843,  1846,  1849,  1852,  1857,  1861,  1867,
-    1870,  1875,  1879,  1886,  1889,  1892,  1895,  1898,  1901,  1904,
-    1907,  1910,  1913,  1918,  1929,  1932,  1937,  1940,  1943,  1949,
-    1953,  1959,  1962,  1970,  1973,  1976,  1979,  1985,  1990,  1995
+     969,   972,   991,   994,   999,  1003,  1009,  1012,  1015,  1020,
+    1023,  1030,  1035,  1046,  1051,  1055,  1059,  1065,  1068,  1074,
+    1082,  1085,  1088,  1094,  1099,  1102,  1107,  1111,  1115,  1119,
+    1125,  1130,  1135,  1139,  1145,  1151,  1154,  1159,  1162,  1181,
+    1186,  1190,  1196,  1202,  1208,  1211,  1215,  1221,  1224,  1229,
+    1233,  1239,  1242,  1245,  1250,  1255,  1260,  1263,  1266,  1271,
+    1274,  1277,  1280,  1283,  1286,  1289,  1292,  1297,  1300,  1305,
+    1309,  1313,  1316,  1320,  1323,  1328,  1331,  1336,  1339,  1344,
+    1348,  1354,  1357,  1362,  1365,  1370,  1373,  1378,  1381,  1400,
+    1403,  1408,  1412,  1418,  1424,  1429,  1432,  1437,  1440,  1445,
+    1448,  1453,  1456,  1461,  1462,  1465,  1470,  1471,  1474,  1479,
+    1483,  1489,  1496,  1499,  1502,  1507,  1510,  1513,  1519,  1522,
+    1527,  1532,  1541,  1546,  1555,  1560,  1563,  1568,  1571,  1576,
+    1582,  1588,  1591,  1594,  1597,  1600,  1603,  1609,  1618,  1621,
+    1626,  1629,  1634,  1637,  1642,  1645,  1648,  1651,  1655,  1659,
+    1662,  1665,  1668,  1671,  1676,  1680,  1684,  1687,  1692,  1697,
+    1701,  1707,  1710,  1715,  1719,  1725,  1730,  1734,  1740,  1745,
+    1748,  1753,  1757,  1763,  1766,  1769,  1772,  1784,  1788,  1807,
+    1820,  1835,  1838,  1841,  1844,  1847,  1850,  1855,  1859,  1865,
+    1868,  1873,  1877,  1884,  1887,  1890,  1893,  1896,  1899,  1902,
+    1905,  1908,  1911,  1916,  1927,  1930,  1935,  1938,  1941,  1947,
+    1951,  1957,  1960,  1968,  1971,  1974,  1977,  1983,  1988,  1993
 };
 #endif
 
@@ -1974,7 +1976,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1978 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1980 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 4: /* TOKEN_NAME  */
@@ -1984,7 +1986,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1988 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 1990 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 5: /* TOKEN_STRING_SINGLE_QUOTED  */
@@ -1994,7 +1996,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 1998 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2000 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 6: /* TOKEN_STRING_DOUBLE_QUOTED  */
@@ -2004,7 +2006,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2008 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2010 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 7: /* TOKEN_UNSIGNED_NUMVAL  */
@@ -2014,7 +2016,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).numeric_literal_value_);
   }
 }
-#line 2018 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2020 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 147: /* sql_statement  */
@@ -2024,7 +2026,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).statement_);
   }
 }
-#line 2028 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2030 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 148: /* quit_statement  */
@@ -2034,7 +2036,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).quit_statement_);
   }
 }
-#line 2038 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2040 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 149: /* alter_table_statement  */
@@ -2044,7 +2046,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).statement_);
   }
 }
-#line 2048 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2050 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 150: /* create_table_statement  */
@@ -2054,7 +2056,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).create_table_statement_);
   }
 }
-#line 2058 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2060 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 151: /* create_index_statement  */
@@ -2064,7 +2066,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).statement_);
   }
 }
-#line 2068 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2070 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 152: /* drop_table_statement  */
@@ -2074,7 +2076,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).drop_table_statement_);
   }
 }
-#line 2078 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2080 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 153: /* column_def  */
@@ -2084,7 +2086,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).attribute_definition_);
   }
 }
-#line 2088 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2090 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 154: /* column_def_commalist  */
@@ -2094,7 +2096,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).attribute_definition_list_);
   }
 }
-#line 2098 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2100 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 155: /* data_type  */
@@ -2104,7 +2106,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).data_type_);
   }
 }
-#line 2108 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2110 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 156: /* column_constraint_def  */
@@ -2114,7 +2116,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).column_constraint_);
   }
 }
-#line 2118 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2120 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 157: /* column_constraint_def_list  */
@@ -2124,7 +2126,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).column_constraint_list_);
   }
 }
-#line 2128 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2130 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 158: /* opt_column_constraint_def_list  */
@@ -2134,7 +2136,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).column_constraint_list_);
   }
 }
-#line 2138 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2140 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 162: /* opt_column_list  */
@@ -2144,7 +2146,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).attribute_list_);
   }
 }
-#line 2148 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2150 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 163: /* opt_block_properties  */
@@ -2154,7 +2156,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).block_properties_);
   }
 }
-#line 2158 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2160 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 164: /* opt_partition_clause  */
@@ -2164,7 +2166,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).partition_clause_);
   }
 }
-#line 2168 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2170 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 165: /* partition_type  */
@@ -2174,7 +2176,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2178 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2180 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 166: /* key_value_list  */
@@ -2184,7 +2186,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2188 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2190 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 167: /* key_value  */
@@ -2194,7 +2196,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).key_value_);
   }
 }
-#line 2198 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2200 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 168: /* key_string_value  */
@@ -2204,7 +2206,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).key_string_value_);
   }
 }
-#line 2208 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2210 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 169: /* key_string_list  */
@@ -2214,7 +2216,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).key_string_list_);
   }
 }
-#line 2218 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2220 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 170: /* key_integer_value  */
@@ -2224,7 +2226,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).key_integer_value_);
   }
 }
-#line 2228 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2230 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 171: /* index_type  */
@@ -2234,7 +2236,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2238 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2240 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 172: /* opt_index_properties  */
@@ -2244,7 +2246,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).key_value_list_);
   }
 }
-#line 2248 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2250 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 173: /* insert_statement  */
@@ -2254,7 +2256,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).insert_statement_);
   }
 }
-#line 2258 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2260 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 174: /* copy_from_statement  */
@@ -2264,7 +2266,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).copy_from_statement_);
   }
 }
-#line 2268 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2270 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 175: /* opt_copy_from_params  */
@@ -2274,7 +2276,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).copy_from_params_);
   }
 }
-#line 2278 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2280 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 176: /* copy_from_params  */
@@ -2284,7 +2286,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).copy_from_params_);
   }
 }
-#line 2288 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2290 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 177: /* update_statement  */
@@ -2294,7 +2296,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).update_statement_);
   }
 }
-#line 2298 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2300 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 178: /* delete_statement  */
@@ -2304,7 +2306,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).delete_statement_);
   }
 }
-#line 2308 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2310 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 179: /* assignment_list  */
@@ -2314,7 +2316,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).assignment_list_);
   }
 }
-#line 2318 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2320 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 180: /* assignment_item  */
@@ -2324,7 +2326,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).assignment_);
   }
 }
-#line 2328 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2330 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 181: /* select_statement  */
@@ -2334,7 +2336,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).select_statement_);
   }
 }
-#line 2338 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2340 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 182: /* opt_priority_clause  */
@@ -2344,7 +2346,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_priority_clause_);
   }
 }
-#line 2348 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2350 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 183: /* with_clause  */
@@ -2354,7 +2356,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).with_list_);
   }
 }
-#line 2358 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2360 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 184: /* with_list  */
@@ -2364,7 +2366,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).with_list_);
   }
 }
-#line 2368 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2370 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 185: /* with_list_element  */
@@ -2374,7 +2376,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).with_list_element_);
   }
 }
-#line 2378 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2380 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 186: /* select_query  */
@@ -2384,7 +2386,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).select_query_);
   }
 }
-#line 2388 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2390 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 188: /* selection  */
@@ -2394,7 +2396,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).selection_);
   }
 }
-#line 2398 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2400 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 189: /* selection_item_commalist  */
@@ -2404,7 +2406,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).selection_list_);
   }
 }
-#line 2408 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2410 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 190: /* selection_item  */
@@ -2414,7 +2416,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).selection_item_);
   }
 }
-#line 2418 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2420 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 191: /* from_clause  */
@@ -2424,7 +2426,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).table_reference_list_);
   }
 }
-#line 2428 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2430 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 192: /* subquery_expression  */
@@ -2434,7 +2436,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).subquery_expression_);
   }
 }
-#line 2438 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2440 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 193: /* opt_sample_clause  */
@@ -2444,13 +2446,13 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_sample_clause_);
   }
 }
-#line 2448 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2450 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 194: /* join_type  */
 #line 618 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2454 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2456 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 195: /* joined_table_reference  */
@@ -2460,7 +2462,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).table_reference_);
   }
 }
-#line 2464 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2466 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 196: /* table_reference  */
@@ -2470,7 +2472,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).table_reference_);
   }
 }
-#line 2474 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2476 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 197: /* table_reference_signature  */
@@ -2480,7 +2482,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).table_reference_signature_);
   }
 }
-#line 2484 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2486 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 198: /* table_reference_signature_primary  */
@@ -2490,7 +2492,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).table_reference_signature_);
   }
 }
-#line 2494 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2496 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 199: /* joined_table_reference_commalist  */
@@ -2500,7 +2502,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).table_reference_list_);
   }
 }
-#line 2504 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2506 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 200: /* opt_group_by_clause  */
@@ -2510,7 +2512,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_group_by_clause_);
   }
 }
-#line 2514 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2516 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 201: /* opt_having_clause  */
@@ -2520,7 +2522,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_having_clause_);
   }
 }
-#line 2524 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2526 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 202: /* opt_order_by_clause  */
@@ -2530,7 +2532,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_order_by_clause_);
   }
 }
-#line 2534 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2536 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 203: /* opt_limit_clause  */
@@ -2540,7 +2542,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_limit_clause_);
   }
 }
-#line 2544 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2546 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 204: /* opt_window_clause  */
@@ -2550,7 +2552,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_window_clause_);
   }
 }
-#line 2554 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2556 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 205: /* window_declaration_list  */
@@ -2560,7 +2562,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).opt_window_clause_);
   }
 }
-#line 2564 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2566 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 206: /* window_declaration  */
@@ -2570,7 +2572,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).window_definition_);
   }
 }
-#line 2574 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2576 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 207: /* window_definition  */
@@ -2580,7 +2582,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).window_definition_);
   }
 }
-#line 2584 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2586 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 208: /* opt_window_partition  */
@@ -2590,7 +2592,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).window_partition_by_list_);
   }
 }
-#line 2594 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2596 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 209: /* opt_window_order  */
@@ -2600,7 +2602,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).window_order_by_list_);
   }
 }
-#line 2604 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2606 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 210: /* opt_window_frame  */
@@ -2610,13 +2612,13 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).window_frame_info_);
   }
 }
-#line 2614 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2616 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 211: /* frame_mode  */
 #line 614 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2620 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2622 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 212: /* frame_preceding  */
@@ -2626,7 +2628,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).numeric_literal_value_);
   }
 }
-#line 2630 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2632 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 213: /* frame_following  */
@@ -2636,7 +2638,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).numeric_literal_value_);
   }
 }
-#line 2640 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2642 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 214: /* order_commalist  */
@@ -2646,7 +2648,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).order_commalist_);
   }
 }
-#line 2650 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2652 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 215: /* order_item  */
@@ -2656,7 +2658,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).order_item_);
   }
 }
-#line 2660 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2662 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 216: /* opt_order_direction  */
@@ -2666,7 +2668,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).order_direction_);
   }
 }
-#line 2670 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2672 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 217: /* opt_nulls_first  */
@@ -2676,7 +2678,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).order_direction_);
   }
 }
-#line 2680 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2682 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 218: /* opt_where_clause  */
@@ -2686,7 +2688,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2690 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2692 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 219: /* where_clause  */
@@ -2696,7 +2698,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2700 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2702 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 220: /* or_expression  */
@@ -2706,7 +2708,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2710 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2712 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 221: /* and_expression  */
@@ -2716,7 +2718,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2720 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2722 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 222: /* not_expression  */
@@ -2726,7 +2728,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2730 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2732 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 223: /* predicate_expression_base  */
@@ -2736,7 +2738,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).predicate_);
   }
 }
-#line 2740 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2742 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 224: /* add_expression  */
@@ -2746,7 +2748,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2750 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2752 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 225: /* multiply_expression  */
@@ -2756,7 +2758,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2760 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2762 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 226: /* unary_expression  */
@@ -2766,7 +2768,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2770 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2772 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 227: /* expression_base  */
@@ -2776,7 +2778,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2780 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2782 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 228: /* function_call  */
@@ -2786,7 +2788,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).function_call_);
   }
 }
-#line 2790 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2792 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 229: /* extract_function  */
@@ -2796,7 +2798,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2800 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2802 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 230: /* substr_function  */
@@ -2806,7 +2808,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2810 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2812 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 231: /* case_expression  */
@@ -2816,7 +2818,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2820 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2822 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 232: /* simple_when_clause_list  */
@@ -2826,7 +2828,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).simple_when_clause_list_);
   }
 }
-#line 2830 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2832 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 233: /* simple_when_clause  */
@@ -2836,7 +2838,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).simple_when_clause_);
   }
 }
-#line 2840 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2842 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 234: /* searched_when_clause_list  */
@@ -2846,7 +2848,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).searched_when_clause_list_);
   }
 }
-#line 2850 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2852 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 235: /* searched_when_clause  */
@@ -2856,7 +2858,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).searched_when_clause_);
   }
 }
-#line 2860 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2862 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 236: /* opt_else_clause  */
@@ -2866,7 +2868,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_);
   }
 }
-#line 2870 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2872 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 237: /* expression_list  */
@@ -2876,7 +2878,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).expression_list_);
   }
 }
-#line 2880 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2882 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 238: /* literal_value  */
@@ -2886,7 +2888,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).literal_value_);
   }
 }
-#line 2890 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2892 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 239: /* datetime_unit  */
@@ -2896,7 +2898,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2900 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2902 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 240: /* literal_value_commalist  */
@@ -2906,7 +2908,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).literal_value_list_);
   }
 }
-#line 2910 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2912 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 241: /* attribute_ref  */
@@ -2916,7 +2918,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).attribute_);
   }
 }
-#line 2920 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2922 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 242: /* attribute_ref_list  */
@@ -2926,31 +2928,31 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).attribute_list_);
   }
 }
-#line 2930 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2932 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 243: /* comparison_operation  */
 #line 615 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2936 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2938 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 244: /* unary_operation  */
 #line 616 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2942 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2944 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 245: /* add_operation  */
 #line 617 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2948 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2950 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 246: /* multiply_operation  */
 #line 617 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2954 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2956 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 247: /* name_commalist  */
@@ -2960,7 +2962,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_list_);
   }
 }
-#line 2964 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2966 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 248: /* any_name  */
@@ -2970,13 +2972,13 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).string_value_);
   }
 }
-#line 2974 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2976 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 249: /* boolean_value  */
 #line 614 "../SqlParser.ypp" /* yacc.c:1257  */
       { }
-#line 2980 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2982 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 250: /* command  */
@@ -2986,7 +2988,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).command_);
   }
 }
-#line 2990 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 2992 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
     case 251: /* command_argument_list  */
@@ -2996,7 +2998,7 @@ yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocatio
     delete ((*yyvaluep).command_argument_list_);
   }
 }
-#line 3000 "SqlParser_gen.cpp" /* yacc.c:1257  */
+#line 3002 "SqlParser_gen.cpp" /* yacc.c:1257  */
         break;
 
 
@@ -3288,148 +3290,148 @@ yyreduce:
   switch (yyn)
     {
         case 2:
-#line 629 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 629 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     *parsedStatement = (yyvsp[-1].statement_);
     YYACCEPT;
   }
-#line 3297 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3299 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 3:
-#line 633 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 633 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     *parsedStatement = (yyvsp[-1].statement_);
     YYACCEPT;
   }
-#line 3306 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3308 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 4:
-#line 637 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 637 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     *parsedStatement = (yyvsp[-1].command_);
     YYACCEPT;
   }
-#line 3315 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3317 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 5:
-#line 641 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 641 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     *parsedStatement = (yyvsp[-1].command_);
     YYACCEPT;
   }
-#line 3324 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3326 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 6:
-#line 645 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 645 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     YYABORT;
   }
-#line 3332 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3334 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 7:
-#line 648 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 648 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     // Regular yyparse() return codes are non-negative, so use a negative one here.
     return -1;
   }
-#line 3341 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3343 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 8:
-#line 655 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 655 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].statement_);
   }
-#line 3349 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3351 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 9:
-#line 658 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 658 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].copy_from_statement_);
   }
-#line 3357 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3359 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 10:
-#line 661 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 661 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].create_table_statement_);
   }
-#line 3365 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3367 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 11:
-#line 664 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 664 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].statement_);
   }
-#line 3373 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3375 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 12:
-#line 667 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 667 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].delete_statement_);
   }
-#line 3381 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3383 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 13:
-#line 670 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 670 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].drop_table_statement_);
   }
-#line 3389 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3391 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 14:
-#line 673 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 673 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].insert_statement_);
   }
-#line 3397 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3399 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 15:
-#line 676 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 676 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].quit_statement_);
   }
-#line 3405 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3407 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 16:
-#line 679 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 679 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].select_statement_);
   }
-#line 3413 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3415 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 17:
-#line 682 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 682 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].update_statement_);
   }
-#line 3421 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3423 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 18:
-#line 688 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 688 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.quit_statement_) = new quickstep::ParseStatementQuit((yylsp[0]).first_line, (yylsp[0]).first_column);
   }
-#line 3429 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3431 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 19:
-#line 694 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 694 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].string_value_);
     delete (yyvsp[0].attribute_definition_);
@@ -3437,22 +3439,22 @@ yyreduce:
     NotSupported(&(yylsp[-5]), yyscanner, "ALTER statements");
     YYERROR;
   }
-#line 3441 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3443 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 20:
-#line 701 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 701 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].string_value_);
     (yyval.statement_) = nullptr;
     NotSupported(&(yylsp[-5]), yyscanner, "ALTER statements");
     YYERROR;
   }
-#line 3452 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3454 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 21:
-#line 707 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 707 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].string_value_);
     delete (yyvsp[0].string_value_);
@@ -3460,11 +3462,11 @@ yyreduce:
     NotSupported(&(yylsp[-5]), yyscanner, "ALTER statements");
     YYERROR;
   }
-#line 3464 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3466 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 22:
-#line 714 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 714 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].string_value_);
     delete (yyvsp[0].string_value_);
@@ -3472,19 +3474,19 @@ yyreduce:
     NotSupported(&(yylsp[-5]), yyscanner, "ALTER statements");
     YYERROR;
   }
-#line 3476 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3478 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 23:
-#line 723 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 723 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.create_table_statement_) = new quickstep::ParseStatementCreateTable((yylsp[-8]).first_line, (yylsp[-8]).first_column, (yyvsp[-6].string_value_), (yyvsp[-4].attribute_definition_list_), (yyvsp[-1].block_properties_), (yyvsp[0].partition_clause_));
   }
-#line 3484 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3486 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 24:
-#line 728 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 728 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[0].key_value_list_)) {
       (yyval.statement_) = new quickstep::ParseStatementCreateIndex((yylsp[-8]).first_line, (yylsp[-8]).first_column, (yyvsp[-6].string_value_), (yyvsp[-4].string_value_), (yyvsp[-3].attribute_list_), (yyvsp[-1].string_value_), (yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].key_value_list_));
@@ -3492,153 +3494,153 @@ yyreduce:
       (yyval.statement_) = new quickstep::ParseStatementCreateIndex((yylsp[-8]).first_line, (yylsp[-8]).first_column, (yyvsp[-6].string_value_), (yyvsp[-4].string_value_), (yyvsp[-3].attribute_list_), (yyvsp[-1].string_value_));
     }
   }
-#line 3496 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3498 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 25:
-#line 737 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 737 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.drop_table_statement_) = new quickstep::ParseStatementDropTable((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[0].string_value_));
   }
-#line 3504 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3506 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 26:
-#line 742 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 742 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_definition_) = new quickstep::ParseAttributeDefinition((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].string_value_), (yyvsp[-1].data_type_), (yyvsp[0].column_constraint_list_));
   }
-#line 3512 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3514 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 27:
-#line 747 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 747 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_definition_list_) = new quickstep::PtrList<quickstep::ParseAttributeDefinition>();
     (yyval.attribute_definition_list_)->push_back((yyvsp[0].attribute_definition_));
   }
-#line 3521 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3523 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 28:
-#line 751 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 751 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_definition_list_) = (yyvsp[-2].attribute_definition_list_);
     (yyval.attribute_definition_list_)->push_back((yyvsp[0].attribute_definition_));
   }
-#line 3530 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3532 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 29:
-#line 757 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 757 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = nullptr;
     NotSupported(&(yylsp[0]), yyscanner, "BIT data type");
     YYERROR;
   }
-#line 3540 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3542 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 30:
-#line 762 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 762 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDate));
   }
-#line 3548 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3550 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 31:
-#line 765 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 765 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetime));
   }
-#line 3556 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3558 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 32:
-#line 768 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 768 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = nullptr;
     NotSupported(&(yylsp[0]), yyscanner, "TIME data type");
     YYERROR;
   }
-#line 3566 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3568 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 33:
-#line 773 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 773 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetime));
   }
-#line 3574 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3576 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 34:
-#line 776 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 776 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDouble));
   }
-#line 3582 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3584 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 35:
-#line 779 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 779 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDouble));
   }
-#line 3590 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3592 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 36:
-#line 782 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 782 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDouble));
   }
-#line 3598 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3600 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 37:
-#line 785 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 785 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kFloat));
   }
-#line 3606 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3608 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 38:
-#line 788 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 788 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kInt));
   }
-#line 3614 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3616 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 39:
-#line 791 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 791 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kInt));
   }
-#line 3622 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3624 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 40:
-#line 794 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 794 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kLong));
   }
-#line 3630 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3632 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 41:
-#line 797 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 797 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kLong));
   }
-#line 3638 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3640 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 42:
-#line 800 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 800 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /**
      * NOTE(chasseur): This pattern exhibits a shift/reduce conflict with the
@@ -3651,27 +3653,27 @@ yyreduce:
         "or YEARMONTH INTERVAL");
     YYERROR;
   }
-#line 3655 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3657 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 43:
-#line 812 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 812 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetimeInterval));
   }
-#line 3663 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3665 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 44:
-#line 815 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 815 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kYearMonthInterval));
   }
-#line 3671 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3673 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 45:
-#line 818 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 818 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[-1].numeric_literal_value_)->float_like()) {
       delete (yyvsp[-1].numeric_literal_value_);
@@ -3690,11 +3692,11 @@ yyreduce:
       }
     }
   }
-#line 3694 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3696 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 46:
-#line 836 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 836 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[-1].numeric_literal_value_)->float_like()) {
       delete (yyvsp[-1].numeric_literal_value_);
@@ -3713,69 +3715,69 @@ yyreduce:
       }
     }
   }
-#line 3717 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3719 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 47:
-#line 856 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 856 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = new quickstep::ParseColumnConstraintNull((yylsp[0]).first_line, (yylsp[0]).first_column);
   }
-#line 3725 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3727 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 48:
-#line 859 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 859 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = new quickstep::ParseColumnConstraintNotNull((yylsp[-1]).first_line, (yylsp[-1]).first_column);
   }
-#line 3733 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3735 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 49:
-#line 862 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 862 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     NotSupported(&(yylsp[0]), yyscanner, "Column Constraints (UNIQUE)");
     YYERROR;
   }
-#line 3743 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3745 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 50:
-#line 867 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 867 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     NotSupported(&(yylsp[-1]), yyscanner, "Column Constraints (PRIMARY KEY)");
     YYERROR;
   }
-#line 3753 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3755 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 51:
-#line 872 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 872 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     delete (yyvsp[0].literal_value_);
     NotSupported(&(yylsp[-1]), yyscanner, "Column Constraints (DEFAULT)");
     YYERROR;
   }
-#line 3764 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3766 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 52:
-#line 878 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 878 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     delete (yyvsp[-1].predicate_);
     NotSupported(&(yylsp[-3]), yyscanner, "Column Constraints (CHECK)");
     YYERROR;
   }
-#line 3775 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3777 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 53:
-#line 884 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 884 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     delete (yyvsp[-3].string_value_);
@@ -3783,65 +3785,65 @@ yyreduce:
     NotSupported(&(yylsp[-4]), yyscanner, "Foreign Keys");
     YYERROR;
   }
-#line 3787 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3789 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 54:
-#line 893 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 893 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_list_) = (yyvsp[-1].column_constraint_list_);
     (yyval.column_constraint_list_)->push_back((yyvsp[0].column_constraint_));
   }
-#line 3796 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3798 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 55:
-#line 897 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 897 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_list_) = new quickstep::PtrList<quickstep::ParseColumnConstraint>();
     (yyval.column_constraint_list_)->push_back((yyvsp[0].column_constraint_));
   }
-#line 3805 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3807 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 56:
-#line 903 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 903 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_list_) = nullptr;
   }
-#line 3813 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3815 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 57:
-#line 906 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 906 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_list_) = (yyvsp[0].column_constraint_list_);
   }
-#line 3821 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3823 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 58:
-#line 911 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 911 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-1].string_list_);
     NotSupported(&(yylsp[-3]), yyscanner, "Table Constraints (UNIQUE)");
     YYERROR;
   }
-#line 3831 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3833 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 59:
-#line 916 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 916 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-1].string_list_);
     NotSupported(&(yylsp[-4]), yyscanner, "Table Constraints (PRIMARY KEY)");
     YYERROR;
   }
-#line 3841 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3843 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 60:
-#line 921 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 921 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-6].string_list_);
     delete (yyvsp[-3].string_value_);
@@ -3849,95 +3851,95 @@ yyreduce:
     NotSupported(&(yylsp[-9]), yyscanner, "Table Constraints (FOREIGN KEY)");
     YYERROR;
   }
-#line 3853 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3855 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 61:
-#line 928 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 928 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-1].predicate_);
     NotSupported(&(yylsp[-3]), yyscanner, "Table Constraints (CHECK)");
     YYERROR;
   }
-#line 3863 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3865 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 62:
-#line 935 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 935 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     NotSupported(&(yylsp[-2]), yyscanner, "Table Constraints");
     YYERROR;
   }
-#line 3872 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3874 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 63:
-#line 939 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 939 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     NotSupported(&(yylsp[0]), yyscanner, "Table Constraints");
     YYERROR;
   }
-#line 3881 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3883 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 64:
-#line 945 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 945 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /* $$ = nullptr; */
   }
-#line 3889 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3891 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 65:
-#line 948 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 948 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /* $$ = $1; */
   }
-#line 3897 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3899 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 66:
-#line 953 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 953 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_list_) = nullptr;
   }
-#line 3905 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3907 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 67:
-#line 956 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 956 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_list_) = (yyvsp[-1].attribute_list_);
   }
-#line 3913 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3915 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 68:
-#line 961 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 961 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.block_properties_) = nullptr;
   }
-#line 3921 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3923 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 69:
-#line 964 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 964 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.block_properties_) = new quickstep::ParseBlockProperties((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-1].key_value_list_));
   }
-#line 3929 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3931 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 70:
-#line 969 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 969 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.partition_clause_) = nullptr;
   }
-#line 3937 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3939 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 71:
-#line 972 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 972 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[0].numeric_literal_value_)->float_like()) {
       delete (yyvsp[0].numeric_literal_value_);
@@ -3955,97 +3957,95 @@ yyreduce:
       }
     }
   }
-#line 3959 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3961 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 72:
-#line 991 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 991 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column,
-           std::to_string(quickstep::PartitionSchemeHeader::PartitionType::kHash));
+    (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column, quickstep::kHashPartitionType);
   }
-#line 3968 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3969 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 73:
-#line 995 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 994 "../SqlParser.ypp" /* yacc.c:1661  */
     {
-    (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column,
-           std::to_string(quickstep::PartitionSchemeHeader::PartitionType::kRange));
+    (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column, quickstep::kRangePartitionType);
   }
-#line 3977 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3977 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 74:
-#line 1001 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 999 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_list_) = new quickstep::PtrList<quickstep::ParseKeyValue>();
     (yyval.key_value_list_)->push_back((yyvsp[0].key_value_));
   }
-#line 3986 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3986 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 75:
-#line 1005 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1003 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_list_) = (yyvsp[-2].key_value_list_);
     (yyval.key_value_list_)->push_back((yyvsp[0].key_value_));
   }
-#line 3995 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3995 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 76:
-#line 1011 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1009 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_) = (yyvsp[0].key_string_value_);
   }
-#line 4003 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4003 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 77:
-#line 1014 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1012 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_) = (yyvsp[0].key_string_list_);
   }
-#line 4011 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4011 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 78:
-#line 1017 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1015 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_) = (yyvsp[0].key_integer_value_);
   }
-#line 4019 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4019 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 79:
-#line 1022 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1020 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_string_value_) = new quickstep::ParseKeyStringValue((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].string_value_));
   }
-#line 4027 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4027 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 80:
-#line 1025 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1023 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     // This is a special case to handle the COMPRESS ALL option of the BLOCK PROPERTIES.
     (yyval.key_string_value_) = new quickstep::ParseKeyStringValue((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].string_value_),
         new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column, "ALL"));
   }
-#line 4037 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4037 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 81:
-#line 1032 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1030 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_string_list_) = new quickstep::ParseKeyStringList((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].string_list_));
   }
-#line 4045 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4045 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 82:
-#line 1037 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1035 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[0].numeric_literal_value_)->float_like()) {
       delete (yyvsp[0].numeric_literal_value_);
@@ -4055,64 +4055,64 @@ yyreduce:
     }
     (yyval.key_integer_value_) = new quickstep::ParseKeyIntegerValue((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].numeric_literal_value_));
   }
-#line 4059 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4059 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 83:
-#line 1048 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1046 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     // Defaults to BitWeavingV, but IndexProperties can change this to H.
     (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column,
            std::to_string(quickstep::IndexSubBlockType::kBitWeavingV));
   }
-#line 4069 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4069 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 84:
-#line 1053 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1051 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column,
            std::to_string(quickstep::IndexSubBlockType::kBloomFilter));
   }
-#line 4078 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4078 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 85:
-#line 1057 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1055 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column,
            std::to_string(quickstep::IndexSubBlockType::kCSBTree));
   }
-#line 4087 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4087 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 86:
-#line 1061 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1059 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column,
            std::to_string(quickstep::IndexSubBlockType::kSMA));
   }
-#line 4096 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4096 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 87:
-#line 1067 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1065 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_list_) = nullptr;
   }
-#line 4104 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4104 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 88:
-#line 1070 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1068 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_list_) = (yyvsp[-1].key_value_list_);
   }
-#line 4112 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4112 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 89:
-#line 1076 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1074 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-7].string_value_);
     delete (yyvsp[-5].string_list_);
@@ -4121,161 +4121,161 @@ yyreduce:
     NotSupported(&(yylsp[-6]), yyscanner, "list of column names in INSERT statement");
     YYERROR;
   }
-#line 4125 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4125 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 90:
-#line 1084 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1082 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.insert_statement_) = new quickstep::ParseStatementInsertTuple((yylsp[-6]).first_line, (yylsp[-6]).first_column, (yyvsp[-4].string_value_), (yyvsp[-1].literal_value_list_));
   }
-#line 4133 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4133 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 91:
-#line 1087 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1085 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.insert_statement_) = new quickstep::ParseStatementInsertSelection((yylsp[-3]).first_line, (yylsp[-2]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].select_query_), nullptr);
   }
-#line 4141 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4141 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 92:
-#line 1090 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1088 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.insert_statement_) = new quickstep::ParseStatementInsertSelection((yylsp[-4]).first_line, (yylsp[-3]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].select_query_), (yyvsp[-4].with_list_));
   }
-#line 4149 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4149 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 93:
-#line 1096 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1094 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.copy_from_statement_) = new quickstep::ParseStatementCopyFrom((yylsp[-4]).first_line, (yylsp[-4]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].string_value_), (yyvsp[0].copy_from_params_));
   }
-#line 4157 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4157 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 94:
-#line 1101 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1099 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.copy_from_params_) = nullptr;
   }
-#line 4165 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4165 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 95:
-#line 1104 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1102 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.copy_from_params_) = (yyvsp[-1].copy_from_params_);
   }
-#line 4173 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4173 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 96:
-#line 1109 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1107 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.copy_from_params_) = new quickstep::ParseCopyFromParams((yylsp[-1]).first_line, (yylsp[-1]).first_column);
     (yyval.copy_from_params_)->set_delimiter((yyvsp[0].string_value_));
   }
-#line 4182 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4182 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 97:
-#line 1113 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1111 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.copy_from_params_) = new quickstep::ParseCopyFromParams((yylsp[-1]).first_line, (yylsp[-1]).first_column);
     (yyval.copy_from_params_)->escape_strings = (yyvsp[0].boolean_value_);
   }
-#line 4191 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4191 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 98:
-#line 1117 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1115 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.copy_from_params_) = (yyvsp[-3].copy_from_params_);
     (yyval.copy_from_params_)->set_delimiter((yyvsp[0].string_value_));
   }
-#line 4200 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4200 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 99:
-#line 1121 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1119 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.copy_from_params_) = (yyvsp[-3].copy_from_params_);
     (yyval.copy_from_params_)->escape_strings = (yyvsp[0].boolean_value_);
   }
-#line 4209 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4209 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 100:
-#line 1127 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1125 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.update_statement_) = new quickstep::ParseStatementUpdate((yylsp[-4]).first_line, (yylsp[-4]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].assignment_list_), (yyvsp[0].predicate_));
   }
-#line 4217 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4217 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 101:
-#line 1132 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1130 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.delete_statement_) = new quickstep::ParseStatementDelete((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].predicate_));
   }
-#line 4225 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4225 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 102:
-#line 1137 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1135 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.assignment_list_) = (yyvsp[-2].assignment_list_);
     (yyval.assignment_list_)->push_back((yyvsp[0].assignment_));
   }
-#line 4234 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4234 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 103:
-#line 1141 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1139 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.assignment_list_) = new quickstep::PtrList<quickstep::ParseAssignment>();
     (yyval.assignment_list_)->push_back((yyvsp[0].assignment_));
   }
-#line 4243 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4243 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 104:
-#line 1147 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1145 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.assignment_) = new quickstep::ParseAssignment((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].string_value_), (yyvsp[0].expression_));
   }
-#line 4251 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4251 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 105:
-#line 1153 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1151 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.select_statement_) = new quickstep::ParseStatementSelect((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].select_query_), nullptr, (yyvsp[0].opt_priority_clause_));
   }
-#line 4259 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4259 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 106:
-#line 1156 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1154 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.select_statement_) = new quickstep::ParseStatementSelect((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-1].select_query_), (yyvsp[-2].with_list_), (yyvsp[0].opt_priority_clause_));
   }
-#line 4267 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4267 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 107:
-#line 1161 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1159 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_priority_clause_) = nullptr;
   }
-#line 4275 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4275 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 108:
-#line 1164 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1162 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[0].numeric_literal_value_)->float_like()) {
       delete (yyvsp[0].numeric_literal_value_);
@@ -4293,415 +4293,415 @@ yyreduce:
       }
     }
   }
-#line 4297 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4297 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 109:
-#line 1183 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1181 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.with_list_) = (yyvsp[0].with_list_);
   }
-#line 4305 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4305 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 110:
-#line 1188 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1186 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.with_list_) = new quickstep::PtrVector<quickstep::ParseSubqueryTableReference>();
     (yyval.with_list_)->push_back((yyvsp[0].with_list_element_));
   }
-#line 4314 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4314 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 111:
-#line 1192 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1190 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.with_list_) = (yyvsp[-2].with_list_);
     (yyval.with_list_)->push_back((yyvsp[0].with_list_element_));
   }
-#line 4323 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4323 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 112:
-#line 1198 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1196 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.with_list_element_) = new quickstep::ParseSubqueryTableReference((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[0].subquery_expression_));
     (yyval.with_list_element_)->set_table_reference_signature((yyvsp[-2].table_reference_signature_));
   }
-#line 4332 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4332 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 113:
-#line 1205 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1203 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.select_query_) = new quickstep::ParseSelect((yylsp[-9]).first_line, (yylsp[-9]).first_column, (yyvsp[-7].selection_), (yyvsp[-6].table_reference_list_), (yyvsp[-5].predicate_), (yyvsp[-4].opt_group_by_clause_), (yyvsp[-3].opt_having_clause_), (yyvsp[-2].opt_order_by_clause_), (yyvsp[-1].opt_limit_clause_), (yyvsp[0].opt_window_clause_));
   }
-#line 4340 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4340 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 114:
-#line 1210 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1208 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /* $$ = nullptr; */
   }
-#line 4348 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4348 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 115:
-#line 1213 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1211 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     NotSupported(&(yylsp[0]), yyscanner, "ALL in selection");
     YYERROR;
   }
-#line 4357 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4357 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 116:
-#line 1217 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1215 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     NotSupported(&(yylsp[0]), yyscanner, "DISTINCT in selection");
     YYERROR;
   }
-#line 4366 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4366 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 117:
-#line 1223 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1221 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.selection_) = new quickstep::ParseSelectionStar((yylsp[0]).first_line, (yylsp[0]).first_column);
   }
-#line 4374 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4374 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 118:
-#line 1226 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1224 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.selection_) = (yyvsp[0].selection_list_);
   }
-#line 4382 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4382 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 119:
-#line 1231 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1229 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.selection_list_) = new quickstep::ParseSelectionList((yylsp[0]).first_line, (yylsp[0]).first_column);
     (yyval.selection_list_)->add((yyvsp[0].selection_item_));
   }
-#line 4391 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4391 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 120:
-#line 1235 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1233 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.selection_list_) = (yyvsp[-2].selection_list_);
     (yyval.selection_list_)->add((yyvsp[0].selection_item_));
   }
-#line 4400 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4400 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 121:
-#line 1241 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1239 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.selection_item_) = new quickstep::ParseSelectionItem((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].expression_), (yyvsp[0].string_value_));
   }
-#line 4408 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4408 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 122:
-#line 1244 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1242 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.selection_item_) = new quickstep::ParseSelectionItem((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].expression_), (yyvsp[0].string_value_));
   }
-#line 4416 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4416 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 123:
-#line 1247 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1245 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.selection_item_) = new quickstep::ParseSelectionItem((yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].expression_));
   }
-#line 4424 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4424 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 124:
-#line 1252 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1250 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_list_) = (yyvsp[0].table_reference_list_);
   }
-#line 4432 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4432 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 125:
-#line 1257 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1255 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.subquery_expression_) = new quickstep::ParseSubqueryExpression((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-1].select_query_));
   }
-#line 4440 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4440 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 126:
-#line 1262 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1260 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_sample_clause_) = NULL;
   }
-#line 4448 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4448 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 127:
-#line 1265 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1263 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_sample_clause_) = new quickstep::ParseSample((yylsp[-2]).first_line, (yylsp[-2]).first_column, true, (yyvsp[-1].numeric_literal_value_));
   }
-#line 4456 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4456 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 128:
-#line 1268 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1266 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_sample_clause_) = new quickstep::ParseSample((yylsp[-2]).first_line, (yylsp[-2]).first_column, false, (yyvsp[-1].numeric_literal_value_));
   }
-#line 4464 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4464 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 129:
-#line 1273 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1271 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kInnerJoin;
   }
-#line 4472 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4472 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 130:
-#line 1276 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1274 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kInnerJoin;
   }
-#line 4480 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4480 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 131:
-#line 1279 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1277 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kLeftOuterJoin;
   }
-#line 4488 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4488 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 132:
-#line 1282 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1280 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kLeftOuterJoin;
   }
-#line 4496 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4496 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 133:
-#line 1285 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1283 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kRightOuterJoin;
   }
-#line 4504 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4504 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 134:
-#line 1288 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1286 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kRightOuterJoin;
   }
-#line 4512 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4512 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 135:
-#line 1291 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1289 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kFullOuterJoin;
   }
-#line 4520 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4520 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 136:
-#line 1294 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1292 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kFullOuterJoin;
   }
-#line 4528 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4528 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 137:
-#line 1299 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1297 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_) = new quickstep::ParseJoinedTableReference((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-4].join_type_), (yyvsp[-5].table_reference_), (yyvsp[-2].table_reference_), (yyvsp[0].predicate_));
   }
-#line 4536 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4536 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 138:
-#line 1302 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1300 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_) = (yyvsp[0].table_reference_);
   }
-#line 4544 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4544 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 139:
-#line 1307 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1305 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_) = new quickstep::ParseSubqueryTableReference((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].subquery_expression_));
     (yyval.table_reference_)->set_table_reference_signature((yyvsp[0].table_reference_signature_));
   }
-#line 4553 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4553 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 140:
-#line 1311 "../

<TRUNCATED>


[13/22] incubator-quickstep git commit: Refactored SelectorOperator regarding partitions.

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


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

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

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


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

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


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

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

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


[05/22] incubator-quickstep git commit: Refactored partition_type in parser.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/86411b9c/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index fe963be..f1876b8 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -1,8 +1,8 @@
-/* A Bison parser, made by GNU Bison 3.0.2.  */
+/* A Bison parser, made by GNU Bison 3.0.4.  */
 
 /* Bison interface for Yacc-like parsers in C
 
-   Copyright (C) 1984, 1989-1990, 2000-2013 Free Software Foundation, Inc.
+   Copyright (C) 1984, 1989-1990, 2000-2015 Free Software Foundation, Inc.
 
    This program is free software: you can redistribute it and/or modify
    it under the terms of the GNU General Public License as published by
@@ -181,10 +181,10 @@ extern int quickstep_yydebug;
 
 /* Value type.  */
 #if ! defined YYSTYPE && ! defined YYSTYPE_IS_DECLARED
-typedef union YYSTYPE YYSTYPE;
+
 union YYSTYPE
 {
-#line 120 "../SqlParser.ypp" /* yacc.c:1909  */
+#line 120 "../SqlParser.ypp" /* yacc.c:1915  */
 
   quickstep::ParseString *string_value_;
 
@@ -284,8 +284,10 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 288 "SqlParser_gen.hpp" /* yacc.c:1909  */
+#line 288 "SqlParser_gen.hpp" /* yacc.c:1915  */
 };
+
+typedef union YYSTYPE YYSTYPE;
 # define YYSTYPE_IS_TRIVIAL 1
 # define YYSTYPE_IS_DECLARED 1
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/86411b9c/parser/tests/Create.test
----------------------------------------------------------------------
diff --git a/parser/tests/Create.test b/parser/tests/Create.test
index 49e7ccd..eee44a6 100644
--- a/parser/tests/Create.test
+++ b/parser/tests/Create.test
@@ -147,7 +147,7 @@ CreateTableStatement[relation_name=test]
 +-attribute_list=
 | +-AttributeDefinition[name=attr,type=Int]
 +-partition_clause=
-  +-PartitionClause[partition_type=0]
+  +-PartitionClause[partition_type=hash]
     +-Number of Partitions=NumericLiteral[numeric_string=4,float_like=false]
     +-attribute_name_list=
       +-String[value=attr]


[07/22] incubator-quickstep git commit: Refactored partition_type in parser.

Posted by ji...@apache.org.
Refactored partition_type in parser.


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

Branch: refs/heads/exact-filter
Commit: 86411b9c1a289cf1cbaaa0f65278e69322f07abd
Parents: 5fee821
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue Jan 10 19:02:34 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue Jan 10 19:02:34 2017 -0800

----------------------------------------------------------------------
 parser/ParsePartitionClause.hpp       |    3 +
 parser/SqlParser.ypp                  |    6 +-
 parser/preprocessed/SqlLexer_gen.cpp  |  386 ++++----
 parser/preprocessed/SqlLexer_gen.hpp  |  190 ++--
 parser/preprocessed/SqlParser_gen.cpp | 1422 ++++++++++++++--------------
 parser/preprocessed/SqlParser_gen.hpp |   12 +-
 parser/tests/Create.test              |    2 +-
 7 files changed, 1065 insertions(+), 956 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/86411b9c/parser/ParsePartitionClause.hpp
----------------------------------------------------------------------
diff --git a/parser/ParsePartitionClause.hpp b/parser/ParsePartitionClause.hpp
index d2912e7..bcc4b52 100644
--- a/parser/ParsePartitionClause.hpp
+++ b/parser/ParsePartitionClause.hpp
@@ -36,6 +36,9 @@ namespace quickstep {
  *  @{
  */
 
+static const char kHashPartitionType[] = "hash";
+static const char kRangePartitionType[] = "range";
+
 /**
  * @brief A parsed representation of partition clause.
  */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/86411b9c/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index 547bb40..29b69d7 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -989,12 +989,10 @@ opt_partition_clause:
 
 partition_type:
   TOKEN_HASH {
-    $$ = new quickstep::ParseString(@1.first_line, @1.first_column,
-           std::to_string(quickstep::PartitionSchemeHeader::PartitionType::kHash));
+    $$ = new quickstep::ParseString(@1.first_line, @1.first_column, quickstep::kHashPartitionType);
   }
   | TOKEN_RANGE{
-    $$ = new quickstep::ParseString(@1.first_line, @1.first_column,
-           std::to_string(quickstep::PartitionSchemeHeader::PartitionType::kRange));
+    $$ = new quickstep::ParseString(@1.first_line, @1.first_column, quickstep::kRangePartitionType);
   };
 
 key_value_list:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/86411b9c/parser/preprocessed/SqlLexer_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.cpp b/parser/preprocessed/SqlLexer_gen.cpp
index c17e1a1..1cb0ac8 100644
--- a/parser/preprocessed/SqlLexer_gen.cpp
+++ b/parser/preprocessed/SqlLexer_gen.cpp
@@ -1,6 +1,6 @@
-#line 2 "SqlLexer_gen.cpp"
+#line 1 "SqlLexer_gen.cpp"
 
-#line 4 "SqlLexer_gen.cpp"
+#line 3 "SqlLexer_gen.cpp"
 
 #define  YY_INT_ALIGNED short int
 
@@ -9,11 +9,89 @@
 #define FLEX_SCANNER
 #define YY_FLEX_MAJOR_VERSION 2
 #define YY_FLEX_MINOR_VERSION 6
-#define YY_FLEX_SUBMINOR_VERSION 0
+#define YY_FLEX_SUBMINOR_VERSION 3
 #if YY_FLEX_SUBMINOR_VERSION > 0
 #define FLEX_BETA
 #endif
 
+    #define yy_create_buffer quickstep_yy_create_buffer
+
+    #define yy_delete_buffer quickstep_yy_delete_buffer
+
+    #define yy_scan_buffer quickstep_yy_scan_buffer
+
+    #define yy_scan_string quickstep_yy_scan_string
+
+    #define yy_scan_bytes quickstep_yy_scan_bytes
+
+    #define yy_init_buffer quickstep_yy_init_buffer
+
+    #define yy_flush_buffer quickstep_yy_flush_buffer
+
+    #define yy_load_buffer_state quickstep_yy_load_buffer_state
+
+    #define yy_switch_to_buffer quickstep_yy_switch_to_buffer
+
+    #define yypush_buffer_state quickstep_yypush_buffer_state
+
+    #define yypop_buffer_state quickstep_yypop_buffer_state
+
+    #define yyensure_buffer_stack quickstep_yyensure_buffer_stack
+
+    #define yylex quickstep_yylex
+
+    #define yyrestart quickstep_yyrestart
+
+    #define yylex_init quickstep_yylex_init
+
+    #define yylex_init_extra quickstep_yylex_init_extra
+
+    #define yylex_destroy quickstep_yylex_destroy
+
+    #define yyget_debug quickstep_yyget_debug
+
+    #define yyset_debug quickstep_yyset_debug
+
+    #define yyget_extra quickstep_yyget_extra
+
+    #define yyset_extra quickstep_yyset_extra
+
+    #define yyget_in quickstep_yyget_in
+
+    #define yyset_in quickstep_yyset_in
+
+    #define yyget_out quickstep_yyget_out
+
+    #define yyset_out quickstep_yyset_out
+
+    #define yyget_leng quickstep_yyget_leng
+
+    #define yyget_text quickstep_yyget_text
+
+    #define yyget_lineno quickstep_yyget_lineno
+
+    #define yyset_lineno quickstep_yyset_lineno
+
+        #define yyget_column quickstep_yyget_column
+
+        #define yyset_column quickstep_yyset_column
+
+    #define yywrap quickstep_yywrap
+
+    #define yyget_lval quickstep_yyget_lval
+
+    #define yyset_lval quickstep_yyset_lval
+
+    #define yyget_lloc quickstep_yyget_lloc
+
+    #define yyset_lloc quickstep_yyset_lloc
+
+    #define yyalloc quickstep_yyalloc
+
+    #define yyrealloc quickstep_yyrealloc
+
+    #define yyfree quickstep_yyfree
+
 /* First, we deal with  platform-specific or compiler-specific issues. */
 
 /* begin standard C headers. */
@@ -88,36 +166,22 @@ typedef unsigned int flex_uint32_t;
 
 #endif /* ! FLEXINT_H */
 
-#ifdef __cplusplus
-
-/* The "const" storage-class-modifier is valid. */
-#define YY_USE_CONST
-
-#else	/* ! __cplusplus */
-
-/* C99 requires __STDC__ to be defined as 1. */
-#if defined (__STDC__)
-
-#define YY_USE_CONST
-
-#endif	/* defined (__STDC__) */
-#endif	/* ! __cplusplus */
-
-#ifdef YY_USE_CONST
+/* TODO: this is always defined, so inline it */
 #define yyconst const
+
+#if defined(__GNUC__) && __GNUC__ >= 3
+#define yynoreturn __attribute__((__noreturn__))
 #else
-#define yyconst
+#define yynoreturn
 #endif
 
 /* Returned upon end-of-file. */
 #define YY_NULL 0
 
-/* Promotes a possibly negative, possibly signed char to an unsigned
- * integer for use as an array index.  If the signed char is negative,
- * we want to instead treat it as an 8-bit unsigned char, hence the
- * double cast.
+/* Promotes a possibly negative, possibly signed char to an
+ *   integer in range [0..255] for use as an array index.
  */
-#define YY_SC_TO_UI(c) ((unsigned int) (unsigned char) c)
+#define YY_SC_TO_UI(c) ((YY_CHAR) (c))
 
 /* An opaque pointer. */
 #ifndef YY_TYPEDEF_YY_SCANNER_T
@@ -141,20 +205,16 @@ typedef void* yyscan_t;
  * definition of BEGIN.
  */
 #define BEGIN yyg->yy_start = 1 + 2 *
-
 /* Translate the current start state into a value that can be later handed
  * to BEGIN to return to the state.  The YYSTATE alias is for lex
  * compatibility.
  */
 #define YY_START ((yyg->yy_start - 1) / 2)
 #define YYSTATE YY_START
-
 /* Action number for EOF rule of a given start state. */
 #define YY_STATE_EOF(state) (YY_END_OF_BUFFER + state + 1)
-
 /* Special action meaning "start processing a new file". */
 #define YY_NEW_FILE quickstep_yyrestart(yyin ,yyscanner )
-
 #define YY_END_OF_BUFFER_CHAR 0
 
 /* Size of default input buffer. */
@@ -187,10 +247,10 @@ typedef size_t yy_size_t;
 #define EOB_ACT_CONTINUE_SCAN 0
 #define EOB_ACT_END_OF_FILE 1
 #define EOB_ACT_LAST_MATCH 2
-
+    
     /* Note: We specifically omit the test for yy_rule_can_match_eol because it requires
      *       access to the local variable yy_act. Since yyless() is a macro, it would break
-     *       existing scanners that call yyless() from OUTSIDE quickstep_yylex. 
+     *       existing scanners that call yyless() from OUTSIDE quickstep_yylex.
      *       One obvious solution it to make yy_act a global. I tried that, and saw
      *       a 5% performance hit in a non-yylineno scanner, because yy_act is
      *       normally declared as a register variable-- so it is not worth it.
@@ -223,7 +283,6 @@ typedef size_t yy_size_t;
 		YY_DO_BEFORE_ACTION; /* set up yytext again */ \
 		} \
 	while ( 0 )
-
 #define unput(c) yyunput( c, yyg->yytext_ptr , yyscanner )
 
 #ifndef YY_STRUCT_YY_BUFFER_STATE
@@ -238,12 +297,12 @@ struct yy_buffer_state
 	/* Size of input buffer in bytes, not including room for EOB
 	 * characters.
 	 */
-	yy_size_t yy_buf_size;
+	int yy_buf_size;
 
 	/* Number of characters read into yy_ch_buf, not including EOB
 	 * characters.
 	 */
-	yy_size_t yy_n_chars;
+	int yy_n_chars;
 
 	/* Whether we "own" the buffer - i.e., we know we created it,
 	 * and can realloc() it to grow it, and should free() it to
@@ -266,7 +325,7 @@ struct yy_buffer_state
 
     int yy_bs_lineno; /**< The line count. */
     int yy_bs_column; /**< The column count. */
-    
+
 	/* Whether to try to fill the input buffer when we reach the
 	 * end of it.
 	 */
@@ -300,36 +359,33 @@ struct yy_buffer_state
 #define YY_CURRENT_BUFFER ( yyg->yy_buffer_stack \
                           ? yyg->yy_buffer_stack[yyg->yy_buffer_stack_top] \
                           : NULL)
-
 /* Same as previous macro, but useful when we know that the buffer stack is not
  * NULL or when we need an lvalue. For internal use only.
  */
 #define YY_CURRENT_BUFFER_LVALUE yyg->yy_buffer_stack[yyg->yy_buffer_stack_top]
 
-void quickstep_yyrestart (FILE *input_file ,yyscan_t yyscanner );
-void quickstep_yy_switch_to_buffer (YY_BUFFER_STATE new_buffer ,yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_create_buffer (FILE *file,int size ,yyscan_t yyscanner );
-void quickstep_yy_delete_buffer (YY_BUFFER_STATE b ,yyscan_t yyscanner );
-void quickstep_yy_flush_buffer (YY_BUFFER_STATE b ,yyscan_t yyscanner );
-void quickstep_yypush_buffer_state (YY_BUFFER_STATE new_buffer ,yyscan_t yyscanner );
-void quickstep_yypop_buffer_state (yyscan_t yyscanner );
-
-static void quickstep_yyensure_buffer_stack (yyscan_t yyscanner );
-static void quickstep_yy_load_buffer_state (yyscan_t yyscanner );
-static void quickstep_yy_init_buffer (YY_BUFFER_STATE b,FILE *file ,yyscan_t yyscanner );
-
+void quickstep_yyrestart ( FILE *input_file , yyscan_t yyscanner );
+void quickstep_yy_switch_to_buffer ( YY_BUFFER_STATE new_buffer , yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_create_buffer ( FILE *file, int size , yyscan_t yyscanner );
+void quickstep_yy_delete_buffer ( YY_BUFFER_STATE b , yyscan_t yyscanner );
+void quickstep_yy_flush_buffer ( YY_BUFFER_STATE b , yyscan_t yyscanner );
+void quickstep_yypush_buffer_state ( YY_BUFFER_STATE new_buffer , yyscan_t yyscanner );
+void quickstep_yypop_buffer_state ( yyscan_t yyscanner );
+
+static void quickstep_yyensure_buffer_stack ( yyscan_t yyscanner );
+static void quickstep_yy_load_buffer_state ( yyscan_t yyscanner );
+static void quickstep_yy_init_buffer ( YY_BUFFER_STATE b, FILE *file , yyscan_t yyscanner );
 #define YY_FLUSH_BUFFER quickstep_yy_flush_buffer(YY_CURRENT_BUFFER ,yyscanner)
 
-YY_BUFFER_STATE quickstep_yy_scan_buffer (char *base,yy_size_t size ,yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_scan_string (yyconst char *yy_str ,yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_scan_bytes (yyconst char *bytes,yy_size_t len ,yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_buffer ( char *base, yy_size_t size , yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_string ( const char *yy_str , yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_bytes ( const char *bytes, int len , yyscan_t yyscanner );
 
-void *quickstep_yyalloc (yy_size_t ,yyscan_t yyscanner );
-void *quickstep_yyrealloc (void *,yy_size_t ,yyscan_t yyscanner );
-void quickstep_yyfree (void * ,yyscan_t yyscanner );
+void *quickstep_yyalloc ( yy_size_t , yyscan_t yyscanner );
+void *quickstep_yyrealloc ( void *, yy_size_t , yyscan_t yyscanner );
+void quickstep_yyfree ( void * , yyscan_t yyscanner );
 
 #define yy_new_buffer quickstep_yy_create_buffer
-
 #define yy_set_interactive(is_interactive) \
 	{ \
 	if ( ! YY_CURRENT_BUFFER ){ \
@@ -339,7 +395,6 @@ void quickstep_yyfree (void * ,yyscan_t yyscanner );
 	} \
 	YY_CURRENT_BUFFER_LVALUE->yy_is_interactive = is_interactive; \
 	}
-
 #define yy_set_bol(at_bol) \
 	{ \
 	if ( ! YY_CURRENT_BUFFER ){\
@@ -349,38 +404,32 @@ void quickstep_yyfree (void * ,yyscan_t yyscanner );
 	} \
 	YY_CURRENT_BUFFER_LVALUE->yy_at_bol = at_bol; \
 	}
-
 #define YY_AT_BOL() (YY_CURRENT_BUFFER_LVALUE->yy_at_bol)
 
 /* Begin user sect3 */
 
 #define quickstep_yywrap(yyscanner) (/*CONSTCOND*/1)
 #define YY_SKIP_YYWRAP
-
-typedef unsigned char YY_CHAR;
+typedef flex_uint8_t YY_CHAR;
 
 typedef int yy_state_type;
 
 #define yytext_ptr yytext_r
 
-static yy_state_type yy_get_previous_state (yyscan_t yyscanner );
-static yy_state_type yy_try_NUL_trans (yy_state_type current_state  ,yyscan_t yyscanner);
-static int yy_get_next_buffer (yyscan_t yyscanner );
-#if defined(__GNUC__) && __GNUC__ >= 3
-__attribute__((__noreturn__))
-#endif
-static void yy_fatal_error (yyconst char msg[] ,yyscan_t yyscanner );
+static yy_state_type yy_get_previous_state ( yyscan_t yyscanner );
+static yy_state_type yy_try_NUL_trans ( yy_state_type current_state  , yyscan_t yyscanner);
+static int yy_get_next_buffer ( yyscan_t yyscanner );
+static void yynoreturn yy_fatal_error ( const char* msg , yyscan_t yyscanner );
 
 /* Done after the current pattern has been matched and before the
  * corresponding action - sets up yytext.
  */
 #define YY_DO_BEFORE_ACTION \
 	yyg->yytext_ptr = yy_bp; \
-	yyleng = (size_t) (yy_cp - yy_bp); \
+	yyleng = (int) (yy_cp - yy_bp); \
 	yyg->yy_hold_char = *yy_cp; \
 	*yy_cp = '\0'; \
 	yyg->yy_c_buf_p = yy_cp;
-
 #define YY_NUM_RULES 161
 #define YY_END_OF_BUFFER 162
 /* This struct is not used in this scanner,
@@ -390,7 +439,7 @@ struct yy_trans_info
 	flex_int32_t yy_verify;
 	flex_int32_t yy_nxt;
 	};
-static yyconst flex_int16_t yy_accept[593] =
+static const flex_int16_t yy_accept[593] =
     {   0,
         0,    0,    0,    0,    0,    0,    0,    0,    0,    0,
         0,    0,  162,    2,    2,  160,  160,  159,  158,  160,
@@ -460,7 +509,7 @@ static yyconst flex_int16_t yy_accept[593] =
 
     } ;
 
-static yyconst YY_CHAR yy_ec[256] =
+static const YY_CHAR yy_ec[256] =
     {   0,
         1,    1,    1,    1,    1,    1,    1,    1,    2,    3,
         1,    1,    4,    1,    1,    1,    1,    1,    1,    1,
@@ -492,7 +541,7 @@ static yyconst YY_CHAR yy_ec[256] =
         1,    1,    1,    1,    1
     } ;
 
-static yyconst YY_CHAR yy_meta[72] =
+static const YY_CHAR yy_meta[72] =
     {   0,
         1,    1,    2,    1,    1,    3,    1,    4,    1,    5,
         5,    6,    6,    5,    1,    1,    1,    7,    7,    7,
@@ -504,7 +553,7 @@ static yyconst YY_CHAR yy_meta[72] =
         8
     } ;
 
-static yyconst flex_uint16_t yy_base[608] =
+static const flex_int16_t yy_base[608] =
     {   0,
         0,    1,   46,    0,  117,  163,    2,    3,  128,  132,
         6,   10,  211, 1312, 1312,    0, 1312,   13, 1312,  194,
@@ -575,7 +624,7 @@ static yyconst flex_uint16_t yy_base[608] =
      1257, 1267, 1277, 1287, 1297, 1302, 1304
     } ;
 
-static yyconst flex_int16_t yy_def[608] =
+static const flex_int16_t yy_def[608] =
     {   0,
       593,  593,  592,    3,  594,  594,  595,  595,  596,  596,
       597,  597,  592,  592,  592,  598,  592,  592,  592,  592,
@@ -646,7 +695,7 @@ static yyconst flex_int16_t yy_def[608] =
       592,  592,  592,  592,  592,  592,  592
     } ;
 
-static yyconst flex_uint16_t yy_nxt[1384] =
+static const flex_int16_t yy_nxt[1384] =
     {   0,
       592,  592,   15,   15,   61,   61,  155,  155,   67,   62,
        62,   68,   67,  592,   70,   68,   70,   73,   73,   77,
@@ -802,7 +851,7 @@ static yyconst flex_uint16_t yy_nxt[1384] =
       592,  592,  592
     } ;
 
-static yyconst flex_int16_t yy_chk[1384] =
+static const flex_int16_t yy_chk[1384] =
     {   0,
         0,    0,    1,    2,    7,    8,   57,   57,   11,    7,
         8,   11,   12,    0,   18,   12,   18,   25,   25,   27,
@@ -959,7 +1008,7 @@ static yyconst flex_int16_t yy_chk[1384] =
     } ;
 
 /* Table of booleans, true if rule could match eol. */
-static yyconst flex_int32_t yy_rule_can_match_eol[162] =
+static const flex_int32_t yy_rule_can_match_eol[162] =
     {   0,
 0, 1, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
@@ -1079,17 +1128,14 @@ class UnaryOperation;
     yycolumn += yyleng;                                   \
   }
 
+#line 1131 "SqlLexer_gen.cpp"
 /* FIXME(chasseur, qzeng): Add support for hexadecimal literals. */
 /**
  * These patterns are based on the SQL-2011 standard for syntax of numeric
  * literals (Part 2, Section 5.3 of the standard).
  **/
 
-
-
-
-
-#line 1093 "SqlLexer_gen.cpp"
+#line 1138 "SqlLexer_gen.cpp"
 
 #define INITIAL 0
 #define CONDITION_SQL 1
@@ -1123,8 +1169,8 @@ struct yyguts_t
     size_t yy_buffer_stack_max; /**< capacity of stack. */
     YY_BUFFER_STATE * yy_buffer_stack; /**< Stack as an array. */
     char yy_hold_char;
-    yy_size_t yy_n_chars;
-    yy_size_t yyleng_r;
+    int yy_n_chars;
+    int yyleng_r;
     char *yy_c_buf_p;
     int yy_init;
     int yy_start;
@@ -1148,7 +1194,7 @@ struct yyguts_t
 
     }; /* end struct yyguts_t */
 
-static int yy_init_globals (yyscan_t yyscanner );
+static int yy_init_globals ( yyscan_t yyscanner );
 
     /* This must go here because YYSTYPE and YYLTYPE are included
      * from bison output in section 1.*/
@@ -1158,48 +1204,48 @@ static int yy_init_globals (yyscan_t yyscanner );
     
 int quickstep_yylex_init (yyscan_t* scanner);
 
-int quickstep_yylex_init_extra (YY_EXTRA_TYPE user_defined,yyscan_t* scanner);
+int quickstep_yylex_init_extra ( YY_EXTRA_TYPE user_defined, yyscan_t* scanner);
 
 /* Accessor methods to globals.
    These are made visible to non-reentrant scanners for convenience. */
 
-int quickstep_yylex_destroy (yyscan_t yyscanner );
+int quickstep_yylex_destroy ( yyscan_t yyscanner );
 
-int quickstep_yyget_debug (yyscan_t yyscanner );
+int quickstep_yyget_debug ( yyscan_t yyscanner );
 
-void quickstep_yyset_debug (int debug_flag ,yyscan_t yyscanner );
+void quickstep_yyset_debug ( int debug_flag , yyscan_t yyscanner );
 
-YY_EXTRA_TYPE quickstep_yyget_extra (yyscan_t yyscanner );
+YY_EXTRA_TYPE quickstep_yyget_extra ( yyscan_t yyscanner );
 
-void quickstep_yyset_extra (YY_EXTRA_TYPE user_defined ,yyscan_t yyscanner );
+void quickstep_yyset_extra ( YY_EXTRA_TYPE user_defined , yyscan_t yyscanner );
 
-FILE *quickstep_yyget_in (yyscan_t yyscanner );
+FILE *quickstep_yyget_in ( yyscan_t yyscanner );
 
-void quickstep_yyset_in  (FILE * _in_str ,yyscan_t yyscanner );
+void quickstep_yyset_in  ( FILE * _in_str , yyscan_t yyscanner );
 
-FILE *quickstep_yyget_out (yyscan_t yyscanner );
+FILE *quickstep_yyget_out ( yyscan_t yyscanner );
 
-void quickstep_yyset_out  (FILE * _out_str ,yyscan_t yyscanner );
+void quickstep_yyset_out  ( FILE * _out_str , yyscan_t yyscanner );
 
-yy_size_t quickstep_yyget_leng (yyscan_t yyscanner );
+			int quickstep_yyget_leng ( yyscan_t yyscanner );
 
-char *quickstep_yyget_text (yyscan_t yyscanner );
+char *quickstep_yyget_text ( yyscan_t yyscanner );
 
-int quickstep_yyget_lineno (yyscan_t yyscanner );
+int quickstep_yyget_lineno ( yyscan_t yyscanner );
 
-void quickstep_yyset_lineno (int _line_number ,yyscan_t yyscanner );
+void quickstep_yyset_lineno ( int _line_number , yyscan_t yyscanner );
 
-int quickstep_yyget_column  (yyscan_t yyscanner );
+int quickstep_yyget_column  ( yyscan_t yyscanner );
 
-void quickstep_yyset_column (int _column_no ,yyscan_t yyscanner );
+void quickstep_yyset_column ( int _column_no , yyscan_t yyscanner );
 
-YYSTYPE * quickstep_yyget_lval (yyscan_t yyscanner );
+YYSTYPE * quickstep_yyget_lval ( yyscan_t yyscanner );
 
-void quickstep_yyset_lval (YYSTYPE * yylval_param ,yyscan_t yyscanner );
+void quickstep_yyset_lval ( YYSTYPE * yylval_param , yyscan_t yyscanner );
 
-       YYLTYPE *quickstep_yyget_lloc (yyscan_t yyscanner );
+       YYLTYPE *quickstep_yyget_lloc ( yyscan_t yyscanner );
     
-        void quickstep_yyset_lloc (YYLTYPE * yylloc_param ,yyscan_t yyscanner );
+        void quickstep_yyset_lloc ( YYLTYPE * yylloc_param , yyscan_t yyscanner );
     
 /* Macros after this point can all be overridden by user definitions in
  * section 1.
@@ -1207,9 +1253,9 @@ void quickstep_yyset_lval (YYSTYPE * yylval_param ,yyscan_t yyscanner );
 
 #ifndef YY_SKIP_YYWRAP
 #ifdef __cplusplus
-extern "C" int quickstep_yywrap (yyscan_t yyscanner );
+extern "C" int quickstep_yywrap ( yyscan_t yyscanner );
 #else
-extern int quickstep_yywrap (yyscan_t yyscanner );
+extern int quickstep_yywrap ( yyscan_t yyscanner );
 #endif
 #endif
 
@@ -1218,19 +1264,18 @@ extern int quickstep_yywrap (yyscan_t yyscanner );
 #endif
 
 #ifndef yytext_ptr
-static void yy_flex_strncpy (char *,yyconst char *,int ,yyscan_t yyscanner);
+static void yy_flex_strncpy ( char *, const char *, int , yyscan_t yyscanner);
 #endif
 
 #ifdef YY_NEED_STRLEN
-static int yy_flex_strlen (yyconst char * ,yyscan_t yyscanner);
+static int yy_flex_strlen ( const char * , yyscan_t yyscanner);
 #endif
 
 #ifndef YY_NO_INPUT
-
 #ifdef __cplusplus
-static int yyinput (yyscan_t yyscanner );
+static int yyinput ( yyscan_t yyscanner );
 #else
-static int input (yyscan_t yyscanner );
+static int input ( yyscan_t yyscanner );
 #endif
 
 #endif
@@ -1250,7 +1295,7 @@ static int input (yyscan_t yyscanner );
 /* This used to be an fputs(), but since the string might contain NUL's,
  * we now use fwrite().
  */
-#define ECHO do { if (fwrite( yytext, yyleng, 1, yyout )) {} } while (0)
+#define ECHO do { if (fwrite( yytext, (size_t) yyleng, 1, yyout )) {} } while (0)
 #endif
 
 /* Gets input and stuffs it into "buf".  number of characters read, or YY_NULL,
@@ -1261,7 +1306,7 @@ static int input (yyscan_t yyscanner );
 	if ( YY_CURRENT_BUFFER_LVALUE->yy_is_interactive ) \
 		{ \
 		int c = '*'; \
-		size_t n; \
+		int n; \
 		for ( n = 0; n < max_size && \
 			     (c = getc( yyin )) != EOF && c != '\n'; ++n ) \
 			buf[n] = (char) c; \
@@ -1274,7 +1319,7 @@ static int input (yyscan_t yyscanner );
 	else \
 		{ \
 		errno=0; \
-		while ( (result = fread(buf, 1, max_size, yyin))==0 && ferror(yyin)) \
+		while ( (result = (int) fread(buf, 1, (yy_size_t) max_size, yyin)) == 0 && ferror(yyin)) \
 			{ \
 			if( errno != EINTR) \
 				{ \
@@ -1316,7 +1361,7 @@ static int input (yyscan_t yyscanner );
 #define YY_DECL_IS_OURS 1
 
 extern int quickstep_yylex \
-               (YYSTYPE * yylval_param,YYLTYPE * yylloc_param ,yyscan_t yyscanner);
+               (YYSTYPE * yylval_param, YYLTYPE * yylloc_param , yyscan_t yyscanner);
 
 #define YY_DECL int quickstep_yylex \
                (YYSTYPE * yylval_param, YYLTYPE * yylloc_param , yyscan_t yyscanner)
@@ -1380,7 +1425,7 @@ YY_DECL
 #line 131 "../SqlLexer.lpp"
 
 
-#line 1384 "SqlLexer_gen.cpp"
+#line 1428 "SqlLexer_gen.cpp"
 
 	while ( /*CONSTCOND*/1 )		/* loops until end-of-file is reached */
 		{
@@ -1408,9 +1453,9 @@ yy_match:
 				{
 				yy_current_state = (int) yy_def[yy_current_state];
 				if ( yy_current_state >= 593 )
-					yy_c = yy_meta[(unsigned int) yy_c];
+					yy_c = yy_meta[yy_c];
 				}
-			yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
+			yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c];
 			++yy_cp;
 			}
 		while ( yy_current_state != 592 );
@@ -1424,10 +1469,10 @@ yy_find_action:
 
 		if ( yy_act != YY_END_OF_BUFFER && yy_rule_can_match_eol[yy_act] )
 			{
-			yy_size_t yyl;
+			int yyl;
 			for ( yyl = 0; yyl < yyleng; ++yyl )
 				if ( yytext[yyl] == '\n' )
-					   
+					
     do{ yylineno++;
         yycolumn=0;
     }while(0)
@@ -2412,7 +2457,7 @@ YY_RULE_SETUP
 #line 466 "../SqlLexer.lpp"
 YY_FATAL_ERROR( "flex scanner jammed" );
 	YY_BREAK
-#line 2416 "SqlLexer_gen.cpp"
+#line 2460 "SqlLexer_gen.cpp"
 
 	case YY_END_OF_BUFFER:
 		{
@@ -2557,7 +2602,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
 	char *dest = YY_CURRENT_BUFFER_LVALUE->yy_ch_buf;
 	char *source = yyg->yytext_ptr;
-	yy_size_t number_to_move, i;
+	int number_to_move, i;
 	int ret_val;
 
 	if ( yyg->yy_c_buf_p > &YY_CURRENT_BUFFER_LVALUE->yy_ch_buf[yyg->yy_n_chars + 1] )
@@ -2586,7 +2631,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 	/* Try to read more data. */
 
 	/* First move last chars to start of buffer. */
-	number_to_move = (yy_size_t) (yyg->yy_c_buf_p - yyg->yytext_ptr) - 1;
+	number_to_move = (int) (yyg->yy_c_buf_p - yyg->yytext_ptr - 1);
 
 	for ( i = 0; i < number_to_move; ++i )
 		*(dest++) = *(source++);
@@ -2599,7 +2644,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 
 	else
 		{
-			yy_size_t num_to_read =
+			int num_to_read =
 			YY_CURRENT_BUFFER_LVALUE->yy_buf_size - number_to_move - 1;
 
 		while ( num_to_read <= 0 )
@@ -2613,7 +2658,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 
 			if ( b->yy_is_our_buffer )
 				{
-				yy_size_t new_size = b->yy_buf_size * 2;
+				int new_size = b->yy_buf_size * 2;
 
 				if ( new_size <= 0 )
 					b->yy_buf_size += b->yy_buf_size / 8;
@@ -2622,11 +2667,11 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 
 				b->yy_ch_buf = (char *)
 					/* Include room in for 2 EOB chars. */
-					quickstep_yyrealloc((void *) b->yy_ch_buf,b->yy_buf_size + 2 ,yyscanner );
+					quickstep_yyrealloc((void *) b->yy_ch_buf,(yy_size_t) (b->yy_buf_size + 2) ,yyscanner );
 				}
 			else
 				/* Can't grow it, we don't own it. */
-				b->yy_ch_buf = 0;
+				b->yy_ch_buf = NULL;
 
 			if ( ! b->yy_ch_buf )
 				YY_FATAL_ERROR(
@@ -2668,10 +2713,10 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 	else
 		ret_val = EOB_ACT_CONTINUE_SCAN;
 
-	if ((yy_size_t) (yyg->yy_n_chars + number_to_move) > YY_CURRENT_BUFFER_LVALUE->yy_buf_size) {
+	if ((yyg->yy_n_chars + number_to_move) > YY_CURRENT_BUFFER_LVALUE->yy_buf_size) {
 		/* Extend the array by 50%, plus the number we really need. */
-		yy_size_t new_size = yyg->yy_n_chars + number_to_move + (yyg->yy_n_chars >> 1);
-		YY_CURRENT_BUFFER_LVALUE->yy_ch_buf = (char *) quickstep_yyrealloc((void *) YY_CURRENT_BUFFER_LVALUE->yy_ch_buf,new_size ,yyscanner );
+		int new_size = yyg->yy_n_chars + number_to_move + (yyg->yy_n_chars >> 1);
+		YY_CURRENT_BUFFER_LVALUE->yy_ch_buf = (char *) quickstep_yyrealloc((void *) YY_CURRENT_BUFFER_LVALUE->yy_ch_buf,(yy_size_t) new_size ,yyscanner );
 		if ( ! YY_CURRENT_BUFFER_LVALUE->yy_ch_buf )
 			YY_FATAL_ERROR( "out of dynamic memory in yy_get_next_buffer()" );
 	}
@@ -2707,9 +2752,9 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 			{
 			yy_current_state = (int) yy_def[yy_current_state];
 			if ( yy_current_state >= 593 )
-				yy_c = yy_meta[(unsigned int) yy_c];
+				yy_c = yy_meta[yy_c];
 			}
-		yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
+		yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c];
 		}
 
 	return yy_current_state;
@@ -2736,9 +2781,9 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 		{
 		yy_current_state = (int) yy_def[yy_current_state];
 		if ( yy_current_state >= 593 )
-			yy_c = yy_meta[(unsigned int) yy_c];
+			yy_c = yy_meta[yy_c];
 		}
-	yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
+	yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c];
 	yy_is_jam = (yy_current_state == 592);
 
 	(void)yyg;
@@ -2774,7 +2819,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 
 		else
 			{ /* need more input */
-			yy_size_t offset = yyg->yy_c_buf_p - yyg->yytext_ptr;
+			int offset = (int) (yyg->yy_c_buf_p - yyg->yytext_ptr);
 			++yyg->yy_c_buf_p;
 
 			switch ( yy_get_next_buffer( yyscanner ) )
@@ -2798,7 +2843,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 				case EOB_ACT_END_OF_FILE:
 					{
 					if ( quickstep_yywrap(yyscanner ) )
-						return EOF;
+						return 0;
 
 					if ( ! yyg->yy_did_buffer_switch_on_eof )
 						YY_NEW_FILE;
@@ -2821,7 +2866,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 	yyg->yy_hold_char = *++yyg->yy_c_buf_p;
 
 	if ( c == '\n' )
-		   
+		
     do{ yylineno++;
         yycolumn=0;
     }while(0)
@@ -2909,12 +2954,12 @@ static void quickstep_yy_load_buffer_state  (yyscan_t yyscanner)
 	if ( ! b )
 		YY_FATAL_ERROR( "out of dynamic memory in quickstep_yy_create_buffer()" );
 
-	b->yy_buf_size = (yy_size_t)size;
+	b->yy_buf_size = size;
 
 	/* yy_ch_buf has to be 2 characters longer than the size given because
 	 * we need to put in 2 end-of-buffer characters.
 	 */
-	b->yy_ch_buf = (char *) quickstep_yyalloc(b->yy_buf_size + 2 ,yyscanner );
+	b->yy_ch_buf = (char *) quickstep_yyalloc((yy_size_t) (b->yy_buf_size + 2) ,yyscanner );
 	if ( ! b->yy_ch_buf )
 		YY_FATAL_ERROR( "out of dynamic memory in quickstep_yy_create_buffer()" );
 
@@ -3070,15 +3115,15 @@ static void quickstep_yyensure_buffer_stack (yyscan_t yyscanner)
 		 * scanner will even need a stack. We use 2 instead of 1 to avoid an
 		 * immediate realloc on the next call.
          */
-		num_to_alloc = 1; // After all that talk, this was set to 1 anyways...
+      num_to_alloc = 1; /* After all that talk, this was set to 1 anyways... */
 		yyg->yy_buffer_stack = (struct yy_buffer_state**)quickstep_yyalloc
 								(num_to_alloc * sizeof(struct yy_buffer_state*)
 								, yyscanner);
 		if ( ! yyg->yy_buffer_stack )
 			YY_FATAL_ERROR( "out of dynamic memory in quickstep_yyensure_buffer_stack()" );
-								  
+
 		memset(yyg->yy_buffer_stack, 0, num_to_alloc * sizeof(struct yy_buffer_state*));
-				
+
 		yyg->yy_buffer_stack_max = num_to_alloc;
 		yyg->yy_buffer_stack_top = 0;
 		return;
@@ -3107,7 +3152,7 @@ static void quickstep_yyensure_buffer_stack (yyscan_t yyscanner)
  * @param base the character buffer
  * @param size the size in bytes of the character buffer
  * @param yyscanner The scanner object.
- * @return the newly allocated buffer state object. 
+ * @return the newly allocated buffer state object.
  */
 YY_BUFFER_STATE quickstep_yy_scan_buffer  (char * base, yy_size_t  size , yyscan_t yyscanner)
 {
@@ -3117,16 +3162,16 @@ YY_BUFFER_STATE quickstep_yy_scan_buffer  (char * base, yy_size_t  size , yyscan
 	     base[size-2] != YY_END_OF_BUFFER_CHAR ||
 	     base[size-1] != YY_END_OF_BUFFER_CHAR )
 		/* They forgot to leave room for the EOB's. */
-		return 0;
+		return NULL;
 
 	b = (YY_BUFFER_STATE) quickstep_yyalloc(sizeof( struct yy_buffer_state ) ,yyscanner );
 	if ( ! b )
 		YY_FATAL_ERROR( "out of dynamic memory in quickstep_yy_scan_buffer()" );
 
-	b->yy_buf_size = size - 2;	/* "- 2" to take care of EOB's */
+	b->yy_buf_size = (int) (size - 2);	/* "- 2" to take care of EOB's */
 	b->yy_buf_pos = b->yy_ch_buf = base;
 	b->yy_is_our_buffer = 0;
-	b->yy_input_file = 0;
+	b->yy_input_file = NULL;
 	b->yy_n_chars = b->yy_buf_size;
 	b->yy_is_interactive = 0;
 	b->yy_at_bol = 1;
@@ -3146,10 +3191,10 @@ YY_BUFFER_STATE quickstep_yy_scan_buffer  (char * base, yy_size_t  size , yyscan
  * @note If you want to scan bytes that may contain NUL values, then use
  *       quickstep_yy_scan_bytes() instead.
  */
-YY_BUFFER_STATE quickstep_yy_scan_string (yyconst char * yystr , yyscan_t yyscanner)
+YY_BUFFER_STATE quickstep_yy_scan_string (const char * yystr , yyscan_t yyscanner)
 {
     
-	return quickstep_yy_scan_bytes(yystr,strlen(yystr) ,yyscanner);
+	return quickstep_yy_scan_bytes(yystr,(int) strlen(yystr) ,yyscanner);
 }
 
 /** Setup the input buffer state to scan the given bytes. The next call to quickstep_yylex() will
@@ -3159,15 +3204,15 @@ YY_BUFFER_STATE quickstep_yy_scan_string (yyconst char * yystr , yyscan_t yyscan
  * @param yyscanner The scanner object.
  * @return the newly allocated buffer state object.
  */
-YY_BUFFER_STATE quickstep_yy_scan_bytes  (yyconst char * yybytes, yy_size_t  _yybytes_len , yyscan_t yyscanner)
+YY_BUFFER_STATE quickstep_yy_scan_bytes  (const char * yybytes, int  _yybytes_len , yyscan_t yyscanner)
 {
 	YY_BUFFER_STATE b;
 	char *buf;
 	yy_size_t n;
-	yy_size_t i;
+	int i;
     
 	/* Get memory for full buffer, including space for trailing EOB's. */
-	n = _yybytes_len + 2;
+	n = (yy_size_t) (_yybytes_len + 2);
 	buf = (char *) quickstep_yyalloc(n ,yyscanner );
 	if ( ! buf )
 		YY_FATAL_ERROR( "out of dynamic memory in quickstep_yy_scan_bytes()" );
@@ -3193,7 +3238,7 @@ YY_BUFFER_STATE quickstep_yy_scan_bytes  (yyconst char * yybytes, yy_size_t  _yy
 #define YY_EXIT_FAILURE 2
 #endif
 
-static void yy_fatal_error (yyconst char* msg , yyscan_t yyscanner)
+static void yynoreturn yy_fatal_error (const char* msg , yyscan_t yyscanner)
 {
 	struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
 	(void)yyg;
@@ -3235,7 +3280,7 @@ YY_EXTRA_TYPE quickstep_yyget_extra  (yyscan_t yyscanner)
 int quickstep_yyget_lineno  (yyscan_t yyscanner)
 {
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
-    
+
         if (! YY_CURRENT_BUFFER)
             return 0;
     
@@ -3248,7 +3293,7 @@ int quickstep_yyget_lineno  (yyscan_t yyscanner)
 int quickstep_yyget_column  (yyscan_t yyscanner)
 {
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
-    
+
         if (! YY_CURRENT_BUFFER)
             return 0;
     
@@ -3276,7 +3321,7 @@ FILE *quickstep_yyget_out  (yyscan_t yyscanner)
 /** Get the length of the current token.
  * @param yyscanner The scanner object.
  */
-yy_size_t quickstep_yyget_leng  (yyscan_t yyscanner)
+int quickstep_yyget_leng  (yyscan_t yyscanner)
 {
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
     return yyleng;
@@ -3394,9 +3439,7 @@ void quickstep_yyset_lloc (YYLTYPE *  yylloc_param , yyscan_t yyscanner)
  * the ONLY reentrant function that doesn't take the scanner as the last argument.
  * That's why we explicitly handle the declaration, instead of using our macros.
  */
-
 int quickstep_yylex_init(yyscan_t* ptr_yy_globals)
-
 {
     if (ptr_yy_globals == NULL){
         errno = EINVAL;
@@ -3423,9 +3466,7 @@ int quickstep_yylex_init(yyscan_t* ptr_yy_globals)
  * The user defined value in the first argument will be available to quickstep_yyalloc in
  * the yyextra field.
  */
-
 int quickstep_yylex_init_extra(YY_EXTRA_TYPE yy_user_defined,yyscan_t* ptr_yy_globals )
-
 {
     struct yyguts_t dummy_yyguts;
 
@@ -3435,20 +3476,20 @@ int quickstep_yylex_init_extra(YY_EXTRA_TYPE yy_user_defined,yyscan_t* ptr_yy_gl
         errno = EINVAL;
         return 1;
     }
-	
+
     *ptr_yy_globals = (yyscan_t) quickstep_yyalloc ( sizeof( struct yyguts_t ), &dummy_yyguts );
-	
+
     if (*ptr_yy_globals == NULL){
         errno = ENOMEM;
         return 1;
     }
-    
+
     /* By setting to 0xAA, we expose bugs in
     yy_init_globals. Leave at 0x00 for releases. */
     memset(*ptr_yy_globals,0x00,sizeof(struct yyguts_t));
-    
+
     quickstep_yyset_extra (yy_user_defined, *ptr_yy_globals);
-    
+
     return yy_init_globals ( *ptr_yy_globals );
 }
 
@@ -3459,10 +3500,10 @@ static int yy_init_globals (yyscan_t yyscanner)
      * This function is called from quickstep_yylex_destroy(), so don't allocate here.
      */
 
-    yyg->yy_buffer_stack = 0;
+    yyg->yy_buffer_stack = NULL;
     yyg->yy_buffer_stack_top = 0;
     yyg->yy_buffer_stack_max = 0;
-    yyg->yy_c_buf_p = (char *) 0;
+    yyg->yy_c_buf_p = NULL;
     yyg->yy_init = 0;
     yyg->yy_start = 0;
 
@@ -3475,8 +3516,8 @@ static int yy_init_globals (yyscan_t yyscanner)
     yyin = stdin;
     yyout = stdout;
 #else
-    yyin = (FILE *) 0;
-    yyout = (FILE *) 0;
+    yyin = NULL;
+    yyout = NULL;
 #endif
 
     /* For future reference: Set errno on error, since we are called by
@@ -3520,7 +3561,7 @@ int quickstep_yylex_destroy  (yyscan_t yyscanner)
  */
 
 #ifndef yytext_ptr
-static void yy_flex_strncpy (char* s1, yyconst char * s2, int n , yyscan_t yyscanner)
+static void yy_flex_strncpy (char* s1, const char * s2, int n , yyscan_t yyscanner)
 {
 	struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
 	(void)yyg;
@@ -3532,7 +3573,7 @@ static void yy_flex_strncpy (char* s1, yyconst char * s2, int n , yyscan_t yysca
 #endif
 
 #ifdef YY_NEED_STRLEN
-static int yy_flex_strlen (yyconst char * s , yyscan_t yyscanner)
+static int yy_flex_strlen (const char * s , yyscan_t yyscanner)
 {
 	int n;
 	for ( n = 0; s[n]; ++n )
@@ -3546,7 +3587,7 @@ void *quickstep_yyalloc (yy_size_t  size , yyscan_t yyscanner)
 {
 	struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
 	(void)yyg;
-	return (void *) malloc( size );
+	return malloc(size);
 }
 
 void *quickstep_yyrealloc  (void * ptr, yy_size_t  size , yyscan_t yyscanner)
@@ -3561,7 +3602,7 @@ void *quickstep_yyrealloc  (void * ptr, yy_size_t  size , yyscan_t yyscanner)
 	 * any pointer type to void*, and deal with argument conversions
 	 * as though doing an assignment.
 	 */
-	return (void *) realloc( (char *) ptr, size );
+	return realloc(ptr, size);
 }
 
 void quickstep_yyfree (void * ptr , yyscan_t yyscanner)
@@ -3576,4 +3617,3 @@ void quickstep_yyfree (void * ptr , yyscan_t yyscanner)
 #line 466 "../SqlLexer.lpp"
 
 
-

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/86411b9c/parser/preprocessed/SqlLexer_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.hpp b/parser/preprocessed/SqlLexer_gen.hpp
index 0dd70e4..1997e75 100644
--- a/parser/preprocessed/SqlLexer_gen.hpp
+++ b/parser/preprocessed/SqlLexer_gen.hpp
@@ -2,9 +2,9 @@
 #define quickstep_yyHEADER_H 1
 #define quickstep_yyIN_HEADER 1
 
-#line 6 "SqlLexer_gen.hpp"
+#line 5 "SqlLexer_gen.hpp"
 
-#line 8 "SqlLexer_gen.hpp"
+#line 7 "SqlLexer_gen.hpp"
 
 #define  YY_INT_ALIGNED short int
 
@@ -13,11 +13,89 @@
 #define FLEX_SCANNER
 #define YY_FLEX_MAJOR_VERSION 2
 #define YY_FLEX_MINOR_VERSION 6
-#define YY_FLEX_SUBMINOR_VERSION 0
+#define YY_FLEX_SUBMINOR_VERSION 3
 #if YY_FLEX_SUBMINOR_VERSION > 0
 #define FLEX_BETA
 #endif
 
+    #define yy_create_buffer quickstep_yy_create_buffer
+
+    #define yy_delete_buffer quickstep_yy_delete_buffer
+
+    #define yy_scan_buffer quickstep_yy_scan_buffer
+
+    #define yy_scan_string quickstep_yy_scan_string
+
+    #define yy_scan_bytes quickstep_yy_scan_bytes
+
+    #define yy_init_buffer quickstep_yy_init_buffer
+
+    #define yy_flush_buffer quickstep_yy_flush_buffer
+
+    #define yy_load_buffer_state quickstep_yy_load_buffer_state
+
+    #define yy_switch_to_buffer quickstep_yy_switch_to_buffer
+
+    #define yypush_buffer_state quickstep_yypush_buffer_state
+
+    #define yypop_buffer_state quickstep_yypop_buffer_state
+
+    #define yyensure_buffer_stack quickstep_yyensure_buffer_stack
+
+    #define yylex quickstep_yylex
+
+    #define yyrestart quickstep_yyrestart
+
+    #define yylex_init quickstep_yylex_init
+
+    #define yylex_init_extra quickstep_yylex_init_extra
+
+    #define yylex_destroy quickstep_yylex_destroy
+
+    #define yyget_debug quickstep_yyget_debug
+
+    #define yyset_debug quickstep_yyset_debug
+
+    #define yyget_extra quickstep_yyget_extra
+
+    #define yyset_extra quickstep_yyset_extra
+
+    #define yyget_in quickstep_yyget_in
+
+    #define yyset_in quickstep_yyset_in
+
+    #define yyget_out quickstep_yyget_out
+
+    #define yyset_out quickstep_yyset_out
+
+    #define yyget_leng quickstep_yyget_leng
+
+    #define yyget_text quickstep_yyget_text
+
+    #define yyget_lineno quickstep_yyget_lineno
+
+    #define yyset_lineno quickstep_yyset_lineno
+
+        #define yyget_column quickstep_yyget_column
+
+        #define yyset_column quickstep_yyset_column
+
+    #define yywrap quickstep_yywrap
+
+    #define yyget_lval quickstep_yyget_lval
+
+    #define yyset_lval quickstep_yyset_lval
+
+    #define yyget_lloc quickstep_yyget_lloc
+
+    #define yyset_lloc quickstep_yyset_lloc
+
+    #define yyalloc quickstep_yyalloc
+
+    #define yyrealloc quickstep_yyrealloc
+
+    #define yyfree quickstep_yyfree
+
 /* First, we deal with  platform-specific or compiler-specific issues. */
 
 /* begin standard C headers. */
@@ -92,25 +170,13 @@ typedef unsigned int flex_uint32_t;
 
 #endif /* ! FLEXINT_H */
 
-#ifdef __cplusplus
-
-/* The "const" storage-class-modifier is valid. */
-#define YY_USE_CONST
-
-#else	/* ! __cplusplus */
-
-/* C99 requires __STDC__ to be defined as 1. */
-#if defined (__STDC__)
-
-#define YY_USE_CONST
-
-#endif	/* defined (__STDC__) */
-#endif	/* ! __cplusplus */
-
-#ifdef YY_USE_CONST
+/* TODO: this is always defined, so inline it */
 #define yyconst const
+
+#if defined(__GNUC__) && __GNUC__ >= 3
+#define yynoreturn __attribute__((__noreturn__))
 #else
-#define yyconst
+#define yynoreturn
 #endif
 
 /* An opaque pointer. */
@@ -165,12 +231,12 @@ struct yy_buffer_state
 	/* Size of input buffer in bytes, not including room for EOB
 	 * characters.
 	 */
-	yy_size_t yy_buf_size;
+	int yy_buf_size;
 
 	/* Number of characters read into yy_ch_buf, not including EOB
 	 * characters.
 	 */
-	yy_size_t yy_n_chars;
+	int yy_n_chars;
 
 	/* Whether we "own" the buffer - i.e., we know we created it,
 	 * and can realloc() it to grow it, and should free() it to
@@ -193,7 +259,7 @@ struct yy_buffer_state
 
     int yy_bs_lineno; /**< The line count. */
     int yy_bs_column; /**< The column count. */
-    
+
 	/* Whether to try to fill the input buffer when we reach the
 	 * end of it.
 	 */
@@ -204,21 +270,21 @@ struct yy_buffer_state
 	};
 #endif /* !YY_STRUCT_YY_BUFFER_STATE */
 
-void quickstep_yyrestart (FILE *input_file ,yyscan_t yyscanner );
-void quickstep_yy_switch_to_buffer (YY_BUFFER_STATE new_buffer ,yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_create_buffer (FILE *file,int size ,yyscan_t yyscanner );
-void quickstep_yy_delete_buffer (YY_BUFFER_STATE b ,yyscan_t yyscanner );
-void quickstep_yy_flush_buffer (YY_BUFFER_STATE b ,yyscan_t yyscanner );
-void quickstep_yypush_buffer_state (YY_BUFFER_STATE new_buffer ,yyscan_t yyscanner );
-void quickstep_yypop_buffer_state (yyscan_t yyscanner );
+void quickstep_yyrestart ( FILE *input_file , yyscan_t yyscanner );
+void quickstep_yy_switch_to_buffer ( YY_BUFFER_STATE new_buffer , yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_create_buffer ( FILE *file, int size , yyscan_t yyscanner );
+void quickstep_yy_delete_buffer ( YY_BUFFER_STATE b , yyscan_t yyscanner );
+void quickstep_yy_flush_buffer ( YY_BUFFER_STATE b , yyscan_t yyscanner );
+void quickstep_yypush_buffer_state ( YY_BUFFER_STATE new_buffer , yyscan_t yyscanner );
+void quickstep_yypop_buffer_state ( yyscan_t yyscanner );
 
-YY_BUFFER_STATE quickstep_yy_scan_buffer (char *base,yy_size_t size ,yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_scan_string (yyconst char *yy_str ,yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_scan_bytes (yyconst char *bytes,yy_size_t len ,yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_buffer ( char *base, yy_size_t size , yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_string ( const char *yy_str , yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_bytes ( const char *bytes, int len , yyscan_t yyscanner );
 
-void *quickstep_yyalloc (yy_size_t ,yyscan_t yyscanner );
-void *quickstep_yyrealloc (void *,yy_size_t ,yyscan_t yyscanner );
-void quickstep_yyfree (void * ,yyscan_t yyscanner );
+void *quickstep_yyalloc ( yy_size_t , yyscan_t yyscanner );
+void *quickstep_yyrealloc ( void *, yy_size_t , yyscan_t yyscanner );
+void quickstep_yyfree ( void * , yyscan_t yyscanner );
 
 /* Begin user sect3 */
 
@@ -251,48 +317,48 @@ void quickstep_yyfree (void * ,yyscan_t yyscanner );
 
 int quickstep_yylex_init (yyscan_t* scanner);
 
-int quickstep_yylex_init_extra (YY_EXTRA_TYPE user_defined,yyscan_t* scanner);
+int quickstep_yylex_init_extra ( YY_EXTRA_TYPE user_defined, yyscan_t* scanner);
 
 /* Accessor methods to globals.
    These are made visible to non-reentrant scanners for convenience. */
 
-int quickstep_yylex_destroy (yyscan_t yyscanner );
+int quickstep_yylex_destroy ( yyscan_t yyscanner );
 
-int quickstep_yyget_debug (yyscan_t yyscanner );
+int quickstep_yyget_debug ( yyscan_t yyscanner );
 
-void quickstep_yyset_debug (int debug_flag ,yyscan_t yyscanner );
+void quickstep_yyset_debug ( int debug_flag , yyscan_t yyscanner );
 
-YY_EXTRA_TYPE quickstep_yyget_extra (yyscan_t yyscanner );
+YY_EXTRA_TYPE quickstep_yyget_extra ( yyscan_t yyscanner );
 
-void quickstep_yyset_extra (YY_EXTRA_TYPE user_defined ,yyscan_t yyscanner );
+void quickstep_yyset_extra ( YY_EXTRA_TYPE user_defined , yyscan_t yyscanner );
 
-FILE *quickstep_yyget_in (yyscan_t yyscanner );
+FILE *quickstep_yyget_in ( yyscan_t yyscanner );
 
-void quickstep_yyset_in  (FILE * _in_str ,yyscan_t yyscanner );
+void quickstep_yyset_in  ( FILE * _in_str , yyscan_t yyscanner );
 
-FILE *quickstep_yyget_out (yyscan_t yyscanner );
+FILE *quickstep_yyget_out ( yyscan_t yyscanner );
 
-void quickstep_yyset_out  (FILE * _out_str ,yyscan_t yyscanner );
+void quickstep_yyset_out  ( FILE * _out_str , yyscan_t yyscanner );
 
-yy_size_t quickstep_yyget_leng (yyscan_t yyscanner );
+			int quickstep_yyget_leng ( yyscan_t yyscanner );
 
-char *quickstep_yyget_text (yyscan_t yyscanner );
+char *quickstep_yyget_text ( yyscan_t yyscanner );
 
-int quickstep_yyget_lineno (yyscan_t yyscanner );
+int quickstep_yyget_lineno ( yyscan_t yyscanner );
 
-void quickstep_yyset_lineno (int _line_number ,yyscan_t yyscanner );
+void quickstep_yyset_lineno ( int _line_number , yyscan_t yyscanner );
 
-int quickstep_yyget_column  (yyscan_t yyscanner );
+int quickstep_yyget_column  ( yyscan_t yyscanner );
 
-void quickstep_yyset_column (int _column_no ,yyscan_t yyscanner );
+void quickstep_yyset_column ( int _column_no , yyscan_t yyscanner );
 
-YYSTYPE * quickstep_yyget_lval (yyscan_t yyscanner );
+YYSTYPE * quickstep_yyget_lval ( yyscan_t yyscanner );
 
-void quickstep_yyset_lval (YYSTYPE * yylval_param ,yyscan_t yyscanner );
+void quickstep_yyset_lval ( YYSTYPE * yylval_param , yyscan_t yyscanner );
 
-       YYLTYPE *quickstep_yyget_lloc (yyscan_t yyscanner );
+       YYLTYPE *quickstep_yyget_lloc ( yyscan_t yyscanner );
     
-        void quickstep_yyset_lloc (YYLTYPE * yylloc_param ,yyscan_t yyscanner );
+        void quickstep_yyset_lloc ( YYLTYPE * yylloc_param , yyscan_t yyscanner );
     
 /* Macros after this point can all be overridden by user definitions in
  * section 1.
@@ -300,18 +366,18 @@ void quickstep_yyset_lval (YYSTYPE * yylval_param ,yyscan_t yyscanner );
 
 #ifndef YY_SKIP_YYWRAP
 #ifdef __cplusplus
-extern "C" int quickstep_yywrap (yyscan_t yyscanner );
+extern "C" int quickstep_yywrap ( yyscan_t yyscanner );
 #else
-extern int quickstep_yywrap (yyscan_t yyscanner );
+extern int quickstep_yywrap ( yyscan_t yyscanner );
 #endif
 #endif
 
 #ifndef yytext_ptr
-static void yy_flex_strncpy (char *,yyconst char *,int ,yyscan_t yyscanner);
+static void yy_flex_strncpy ( char *, const char *, int , yyscan_t yyscanner);
 #endif
 
 #ifdef YY_NEED_STRLEN
-static int yy_flex_strlen (yyconst char * ,yyscan_t yyscanner);
+static int yy_flex_strlen ( const char * , yyscan_t yyscanner);
 #endif
 
 #ifndef YY_NO_INPUT
@@ -340,7 +406,7 @@ static int yy_flex_strlen (yyconst char * ,yyscan_t yyscanner);
 #define YY_DECL_IS_OURS 1
 
 extern int quickstep_yylex \
-               (YYSTYPE * yylval_param,YYLTYPE * yylloc_param ,yyscan_t yyscanner);
+               (YYSTYPE * yylval_param, YYLTYPE * yylloc_param , yyscan_t yyscanner);
 
 #define YY_DECL int quickstep_yylex \
                (YYSTYPE * yylval_param, YYLTYPE * yylloc_param , yyscan_t yyscanner)
@@ -363,6 +429,6 @@ extern int quickstep_yylex \
 #line 466 "../SqlLexer.lpp"
 
 
-#line 367 "SqlLexer_gen.hpp"
+#line 432 "SqlLexer_gen.hpp"
 #undef quickstep_yyIN_HEADER
 #endif /* quickstep_yyHEADER_H */



[02/22] incubator-quickstep git commit: Fixed the bug in resetting exactness for InsertSelect.

Posted by ji...@apache.org.
Fixed the bug in resetting exactness for InsertSelect.


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

Branch: refs/heads/exact-filter
Commit: c0d510eb0469c095cc216846d2da5df65961507b
Parents: 9fcb0ac
Author: Zuyu Zhang <zu...@apache.org>
Authored: Fri Jan 6 13:45:59 2017 -0800
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Fri Jan 6 13:45:59 2017 -0800

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp      | 2 +-
 relational_operators/SaveBlocksOperator.hpp | 6 +++---
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0d510eb/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index d24f498..29e67f7 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -1243,7 +1243,7 @@ void ExecutionGenerator::convertInsertSelection(
   insert_destination_proto->set_relational_op_index(insert_selection_index);
 
   CatalogRelation *mutable_relation =
-      catalog_database_->getRelationByIdMutable(selection_relation->getID());
+      catalog_database_->getRelationByIdMutable(destination_relation.getID());
   const QueryPlan::DAGNodeIndex save_blocks_index =
       execution_plan_->addRelationalOperator(
           new SaveBlocksOperator(query_handle_->query_id(), mutable_relation));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c0d510eb/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index a8d5327..573d81e 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -61,9 +61,9 @@ class SaveBlocksOperator : public RelationalOperator {
    * @param force If true, force writing of all blocks to disk, otherwise only
    *        write dirty blocks.
    **/
-  explicit SaveBlocksOperator(const std::size_t query_id,
-                              CatalogRelation *relation,
-                              const bool force = false)
+  SaveBlocksOperator(const std::size_t query_id,
+                     CatalogRelation *relation,
+                     const bool force = false)
       : RelationalOperator(query_id),
         force_(force),
         relation_(relation),


[11/22] incubator-quickstep git commit: Removed the unused feedInputBlocks.

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


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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


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

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


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

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

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


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

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

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

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


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

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


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

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

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


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


[16/22] incubator-quickstep git commit: Added partition_id in feedInputBlock.

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


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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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