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/05/04 03:22:45 UTC

[01/32] incubator-quickstep git commit: Topological sort functionality in DAG [Forced Update!]

Repository: incubator-quickstep
Updated Branches:
  refs/heads/new-op da9baf7e6 -> 583724ea8 (forced update)


Topological sort functionality in DAG

- Implemented a very simple Kahn's algorithm for topological sorting of nodes
  in the DAG class.


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

Branch: refs/heads/new-op
Commit: 6e3499a80c559cb3ab11b9800cf5813b0f233f77
Parents: 563abe0
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Tue Apr 18 15:34:16 2017 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Wed Apr 19 11:06:38 2017 -0500

----------------------------------------------------------------------
 utility/DAG.hpp                | 51 +++++++++++++++++++++++
 utility/tests/DAG_unittest.cpp | 82 +++++++++++++++++++++++++++++++++++++
 2 files changed, 133 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6e3499a8/utility/DAG.hpp
----------------------------------------------------------------------
diff --git a/utility/DAG.hpp b/utility/DAG.hpp
index a1f2619..c286880 100644
--- a/utility/DAG.hpp
+++ b/utility/DAG.hpp
@@ -22,6 +22,7 @@
 
 #include <cstddef>
 #include <memory>
+#include <queue>
 #include <unordered_map>
 #include <unordered_set>
 #include <utility>
@@ -246,6 +247,11 @@ class DAG {
     return nodes_[node_index].getDependents().end();
   }
 
+  /**
+   * @brief Get a topologically sorted list of node IDs.
+   **/
+  std::vector<size_type_nodes> getTopologicalSorting() const;
+
  private:
   /**
    * @brief A node in the DAG which contains a payload. DAGNode owns its
@@ -489,6 +495,51 @@ bool DAG<T, LinkMetadataT>::hasCycleHelper(const typename DAG<T, LinkMetadataT>:
   return false;
 }
 
+template <class T, class LinkMetadataT>
+std::vector<typename DAG<T, LinkMetadataT>::size_type_nodes>
+DAG<T, LinkMetadataT>::getTopologicalSorting() const {
+  // As a clarification, if A->B then A is the dependency for B and B is dependent on A.
+  // We implement "Kahn's algorithm" for the sorting.
+  DCHECK(!hasCycle());
+  // This list is going to be the topologically sorted output.
+  std::unique_ptr<std::vector<typename DAG<T, LinkMetadataT>::size_type_nodes>>
+      sorted_list(new std::vector<size_type_nodes>());
+  sorted_list->reserve(this->size());
+  // Key = node ID, value = # incoming edges for this node.
+  // NOTE(harshad) - We modify the "values" in this map as we go along.
+  std::unordered_map<typename DAG<T, LinkMetadataT>::size_type_nodes,
+                     std::size_t> num_dependencies;
+  std::queue<typename DAG<T, LinkMetadataT>::size_type_nodes> nodes_with_no_dependencies;
+  // First store the nodes without any dependencies in a list.
+  // Also remember the number of dependencies for each node in a map.
+  for (auto node_id = 0u; node_id < this->size(); ++node_id) {
+    if (nodes_[node_id].getDependencies().empty()) {
+      nodes_with_no_dependencies.emplace(node_id);
+    }
+    num_dependencies[node_id] = nodes_[node_id].getDependencies().size();
+  }
+  // The algorithm begins now.
+  while (!nodes_with_no_dependencies.empty()) {
+    // For a node with no dependencies ...
+    auto curr_node = nodes_with_no_dependencies.front();
+    nodes_with_no_dependencies.pop();
+    // Add the node to the sorted list.
+    sorted_list->emplace_back(curr_node);
+    auto dependents_of_curr_node = getDependents(curr_node);
+    for (auto dependent_iterator : dependents_of_curr_node) {
+      // For each dependent of the current node ...
+      auto dependent_node_id = dependent_iterator.first;
+      // Remove the incoming edge from curr_node.
+      DCHECK_GE(num_dependencies[dependent_node_id], 1u);
+      if (--num_dependencies[dependent_node_id] == 0) {
+        // Now this node has no children.
+        nodes_with_no_dependencies.emplace(dependent_node_id);
+      }
+    }
+  }
+  return *(sorted_list.release());
+}
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/6e3499a8/utility/tests/DAG_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/DAG_unittest.cpp b/utility/tests/DAG_unittest.cpp
index 3fe2990..3e8d167 100644
--- a/utility/tests/DAG_unittest.cpp
+++ b/utility/tests/DAG_unittest.cpp
@@ -490,6 +490,88 @@ TEST(DAGTest, LinkMetadataBoolTest) {
   EXPECT_FALSE(dag_.getLinkMetadata(1, 0));
 }
 
+TEST(DAGTest, TopoSortTest) {
+  const int kNodeSize = 5;
+  DAG<DummyPayload, int> dag_;
+
+  for (int node_index = 0; node_index < kNodeSize; ++node_index) {
+    ASSERT_EQ(static_cast<std::size_t>(node_index),
+              dag_.createNode(new DummyPayload(node_index)));
+  }
+
+  /*
+   *    0
+   *   / \
+   *  v   v
+   *  1   2
+   *   \ /
+   *    v
+   *    3
+   *    |
+   *    v
+   *    4
+   *
+   */
+
+  dag_.createLink(0, 1, 2);
+  dag_.createLink(0, 2, 1);
+  dag_.createLink(1, 3, 1);
+  dag_.createLink(2, 3, 1);
+  dag_.createLink(3, 4, 1);
+
+  vector<DAG<DummyPayload, int>::size_type_nodes> sorted_list =
+      dag_.getTopologicalSorting();
+  std::unordered_map<DAG<DummyPayload, int>::size_type_nodes, bool> node_exists;
+  // First check if the ordering is a legal sequence of nodes, i.e. every node
+  // appears exactly once.
+  for (auto node_id = 0u; node_id < dag_.size(); ++node_id) {
+    node_exists[node_id] = false;
+  }
+  for (auto i : sorted_list) {
+    node_exists[i] = true;
+  }
+  for (auto node_id = 0u; node_id < dag_.size(); ++node_id) {
+    ASSERT_TRUE(node_exists[node_id]);
+  }
+  // We apply the following condition for verifying if we have obtained a valid
+  // topological sorting.
+  // If there is an edge i->j between two nodes i and j, then i comes before j
+  // in the topologically sorted order.
+  // We use a map to store the position of a given node in the sorted order.
+  // Key = node ID, value = position of the node in the sorted order.
+  std::unordered_map<std::size_t, std::size_t> position_in_sorted_order;
+  for (std::size_t i = 0; i < sorted_list.size(); ++i) {
+    position_in_sorted_order[sorted_list[i]] = i;
+  }
+  std::vector<std::tuple<std::size_t, std::size_t>> edges;
+  // Populate the list of edges.
+  edges.emplace_back(0, 1);
+  edges.emplace_back(0, 2);
+  edges.emplace_back(1, 3);
+  edges.emplace_back(2, 3);
+  edges.emplace_back(3, 4);
+  for (auto curr_edge : edges) {
+    // (i, j) : i is "from node", j is "to node".
+    std::size_t from_node_position = position_in_sorted_order[std::get<0>(curr_edge)];
+    std::size_t to_node_position = position_in_sorted_order[std::get<1>(curr_edge)];
+    ASSERT_LT(from_node_position, to_node_position);
+  }
+  // Now extend the same logic that we applied for edges for paths in the DAG.
+  // We have already verified paths with length = 1 (edges), so we will only
+  // consider paths with length more than one.
+  std::vector<std::tuple<std::size_t, std::size_t>> paths;
+  paths.emplace_back(0, 3);
+  paths.emplace_back(0, 4);
+  paths.emplace_back(1, 4);
+  paths.emplace_back(2, 4);
+  for (auto curr_path : paths) {
+    // (i, j) : i is "from node", j is "to node".
+    std::size_t from_node_position = position_in_sorted_order[std::get<0>(curr_path)];
+    std::size_t to_node_position = position_in_sorted_order[std::get<1>(curr_path)];
+    ASSERT_LT(from_node_position, to_node_position);
+  }
+}
+
 #ifdef QUICKSTEP_DEBUG
 TEST(DAGDeathTest, SixNodeStagesCycleTest) {
   const int kNodeSize = 6;


[10/32] incubator-quickstep git commit: Fix the issues with the common subexpression feature

Posted by ji...@apache.org.
Fix the issues with the common subexpression feature


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

Branch: refs/heads/new-op
Commit: 30021acf8bdf6d33f6e940f3343760384d971830
Parents: d6a01e7
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Apr 24 02:33:39 2017 -0500
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Apr 24 17:55:22 2017 -0700

----------------------------------------------------------------------
 expressions/predicate/Predicate.hpp             |  6 +--
 expressions/scalar/Scalar.cpp                   |  2 +-
 expressions/scalar/Scalar.hpp                   |  5 +-
 expressions/scalar/ScalarSharedExpression.cpp   | 51 +++++++++-----------
 expressions/scalar/ScalarSharedExpression.hpp   | 13 +++--
 .../tests/ScalarCaseExpression_unittest.cpp     | 26 +++++++---
 .../expressions/CommonSubexpression.hpp         |  4 +-
 query_optimizer/expressions/SimpleCase.cpp      |  6 +--
 query_optimizer/rules/CollapseSelection.cpp     |  2 +-
 query_optimizer/rules/CollapseSelection.hpp     |  2 +
 .../rules/ExtractCommonSubexpression.cpp        |  2 +-
 .../rules/ExtractCommonSubexpression.hpp        |  2 +-
 .../rules/ReuseAggregateExpressions.cpp         |  7 +--
 .../rules/ReuseAggregateExpressions.hpp         |  2 +
 14 files changed, 68 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/expressions/predicate/Predicate.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/Predicate.hpp b/expressions/predicate/Predicate.hpp
index 6a2ba6d..df04644 100644
--- a/expressions/predicate/Predicate.hpp
+++ b/expressions/predicate/Predicate.hpp
@@ -65,11 +65,7 @@ class Predicate : public Expression {
    **/
   static const char *kPredicateTypeNames[kNumPredicateTypes];
 
-  /**
-   * @brief Virtual destructor.
-   *
-   **/
-  virtual ~Predicate() {
+  ~Predicate() override {
   }
 
   std::string getName() const override {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/expressions/scalar/Scalar.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/Scalar.cpp b/expressions/scalar/Scalar.cpp
index c552d8b..da0fc1b 100644
--- a/expressions/scalar/Scalar.cpp
+++ b/expressions/scalar/Scalar.cpp
@@ -31,8 +31,8 @@ const char *Scalar::kScalarDataSourceNames[] = {
   "Attribute",
   "UnaryExpression",
   "BinaryExpression",
-  "SharedExpression",
   "SimpleCase"
+  "SharedExpression",
 };
 
 const TypedValue& Scalar::getStaticValue() const {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/expressions/scalar/Scalar.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/Scalar.hpp b/expressions/scalar/Scalar.hpp
index 472b71c..6e482c2 100644
--- a/expressions/scalar/Scalar.hpp
+++ b/expressions/scalar/Scalar.hpp
@@ -69,10 +69,7 @@ class Scalar : public Expression {
    **/
   static const char *kScalarDataSourceNames[kNumScalarDataSources];
 
-  /**
-   * @brief Virtual destructor.
-   **/
-  virtual ~Scalar() {
+  ~Scalar() override {
   }
 
   std::string getName() const override {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/expressions/scalar/ScalarSharedExpression.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarSharedExpression.cpp b/expressions/scalar/ScalarSharedExpression.cpp
index f97c60b..e301116 100644
--- a/expressions/scalar/ScalarSharedExpression.cpp
+++ b/expressions/scalar/ScalarSharedExpression.cpp
@@ -34,19 +34,12 @@ namespace quickstep {
 
 struct SubBlocksReference;
 
-ScalarSharedExpression::ScalarSharedExpression(const int share_id,
-                                               Scalar *operand)
-    : Scalar(operand->getType()),
-      share_id_(share_id),
-      operand_(operand) {
-}
-
 serialization::Scalar ScalarSharedExpression::getProto() const {
   serialization::Scalar proto;
   proto.set_data_source(serialization::Scalar::SHARED_EXPRESSION);
   proto.SetExtension(serialization::ScalarSharedExpression::share_id, share_id_);
   proto.MutableExtension(serialization::ScalarSharedExpression::operand)
-      ->CopyFrom(operand_->getProto());
+      ->MergeFrom(operand_->getProto());
 
   return proto;
 }
@@ -81,16 +74,16 @@ ColumnVectorPtr ScalarSharedExpression::getAllValues(
     ColumnVectorCache *cv_cache) const {
   if (cv_cache == nullptr) {
     return operand_->getAllValues(accessor, sub_blocks_ref, cv_cache);
+  }
+
+  ColumnVectorPtr result;
+  if (cv_cache->contains(share_id_)) {
+    result = cv_cache->get(share_id_);
   } else {
-    ColumnVectorPtr result;
-    if (cv_cache->contains(share_id_)) {
-      result = cv_cache->get(share_id_);
-    } else {
-      result = operand_->getAllValues(accessor, sub_blocks_ref, cv_cache);
-      cv_cache->set(share_id_, result);
-    }
-    return result;
+    result = operand_->getAllValues(accessor, sub_blocks_ref, cv_cache);
+    cv_cache->set(share_id_, result);
   }
+  return result;
 }
 
 ColumnVectorPtr ScalarSharedExpression::getAllValuesForJoin(
@@ -107,21 +100,21 @@ ColumnVectorPtr ScalarSharedExpression::getAllValuesForJoin(
                                          right_accessor,
                                          joined_tuple_ids,
                                          cv_cache);
+  }
+
+  ColumnVectorPtr result;
+  if (cv_cache->contains(share_id_)) {
+    result = cv_cache->get(share_id_);
   } else {
-    ColumnVectorPtr result;
-    if (cv_cache->contains(share_id_)) {
-      result = cv_cache->get(share_id_);
-    } else {
-      result = operand_->getAllValuesForJoin(left_relation_id,
-                                             left_accessor,
-                                             right_relation_id,
-                                             right_accessor,
-                                             joined_tuple_ids,
-                                             cv_cache);
-      cv_cache->set(share_id_, result);
-    }
-    return result;
+    result = operand_->getAllValuesForJoin(left_relation_id,
+                                           left_accessor,
+                                           right_relation_id,
+                                           right_accessor,
+                                           joined_tuple_ids,
+                                           cv_cache);
+    cv_cache->set(share_id_, result);
   }
+  return result;
 }
 
 void ScalarSharedExpression::getFieldStringItems(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/expressions/scalar/ScalarSharedExpression.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarSharedExpression.hpp b/expressions/scalar/ScalarSharedExpression.hpp
index d5dddbc..f39c45b 100644
--- a/expressions/scalar/ScalarSharedExpression.hpp
+++ b/expressions/scalar/ScalarSharedExpression.hpp
@@ -53,11 +53,16 @@ class ScalarSharedExpression : public Scalar {
   /**
    * @brief Constructor.
    *
-   * @param share_id The unique integer identifier for each equivalence class of
-   *        common subexpressions.
-   * @param operand The underlying scalar subexpression.
+   * @param share_id The unique integer identifier for each equivalence class
+   *        of common subexpressions.
+   * @param operand The underlying scalar subexpression, which this
+   *        ScalarSharedExpression takes ownership of.
    **/
-  ScalarSharedExpression(const int share_id, Scalar *operand);
+  ScalarSharedExpression(const int share_id, Scalar *operand)
+      : Scalar(operand->getType()),
+        share_id_(share_id),
+        operand_(operand) {
+  }
 
   /**
    * @brief Destructor.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp b/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp
index 2de9e84..7182642 100644
--- a/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp
+++ b/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp
@@ -309,7 +309,9 @@ TEST_F(ScalarCaseExpressionTest, BasicComparisonAndLiteralWithFilteredInputTest)
                         varchar_type));
 
   ColumnVectorPtr result_cv(
-      case_expr.getAllValues(filtered_accessor.get(), nullptr, nullptr));
+      case_expr.getAllValues(filtered_accessor.get(),
+                             nullptr /* sub_blocks_ref */,
+                             nullptr /* cv_cache */));
   ASSERT_FALSE(result_cv->isNative());
   const IndirectColumnVector &indirect_result_cv
       = static_cast<const IndirectColumnVector&>(*result_cv);
@@ -381,7 +383,9 @@ TEST_F(ScalarCaseExpressionTest, WhenClauseOrderTest) {
                         varchar_type));
 
   ColumnVectorPtr result_cv(
-      case_expr.getAllValues(&sample_data_value_accessor_, nullptr, nullptr));
+      case_expr.getAllValues(&sample_data_value_accessor_,
+                             nullptr /* sub_blocks_ref */,
+                             nullptr /* cv_cache */));
   ASSERT_FALSE(result_cv->isNative());
   const IndirectColumnVector &indirect_result_cv
       = static_cast<const IndirectColumnVector&>(*result_cv);
@@ -475,7 +479,9 @@ TEST_F(ScalarCaseExpressionTest, ComplexConjunctionAndCalculatedExpressionTest)
           new ScalarAttribute(*sample_relation_->getAttributeById(0))));
 
   ColumnVectorPtr result_cv(
-      case_expr.getAllValues(&sample_data_value_accessor_, nullptr, nullptr));
+      case_expr.getAllValues(&sample_data_value_accessor_,
+                             nullptr /* sub_blocks_ref */,
+                             nullptr /* cv_cache */));
   ASSERT_TRUE(result_cv->isNative());
   const NativeColumnVector &native_result_cv
       = static_cast<const NativeColumnVector&>(*result_cv);
@@ -598,7 +604,9 @@ TEST_F(ScalarCaseExpressionTest,
           new ScalarAttribute(*sample_relation_->getAttributeById(0))));
 
   ColumnVectorPtr result_cv(
-      case_expr.getAllValues(filtered_accessor.get(), nullptr, nullptr));
+      case_expr.getAllValues(filtered_accessor.get(),
+                             nullptr /* sub_blocks_ref */,
+                             nullptr /* cv_cache */));
   ASSERT_TRUE(result_cv->isNative());
   const NativeColumnVector &native_result_cv
       = static_cast<const NativeColumnVector&>(*result_cv);
@@ -708,7 +716,9 @@ TEST_F(ScalarCaseExpressionTest, ComplexDisjunctionAndCalculatedExpressionTest)
           new ScalarAttribute(*sample_relation_->getAttributeById(0))));
 
   ColumnVectorPtr result_cv(
-      case_expr.getAllValues(&sample_data_value_accessor_, nullptr, nullptr));
+      case_expr.getAllValues(&sample_data_value_accessor_,
+                             nullptr /* sub_blocks_ref */,
+                             nullptr /* cv_cache */));
   ASSERT_TRUE(result_cv->isNative());
   const NativeColumnVector &native_result_cv
       = static_cast<const NativeColumnVector&>(*result_cv);
@@ -828,7 +838,9 @@ TEST_F(ScalarCaseExpressionTest,
           new ScalarAttribute(*sample_relation_->getAttributeById(0))));
 
   ColumnVectorPtr result_cv(
-      case_expr.getAllValues(filtered_accessor.get(), nullptr, nullptr));
+      case_expr.getAllValues(filtered_accessor.get(),
+                             nullptr /* sub_blocks_ref */,
+                             nullptr /* cv_cache */));
   ASSERT_TRUE(result_cv->isNative());
   const NativeColumnVector &native_result_cv
       = static_cast<const NativeColumnVector&>(*result_cv);
@@ -935,7 +947,7 @@ TEST_F(ScalarCaseExpressionTest, JoinTest) {
       1,
       &other_accessor,
       joined_tuple_ids,
-      nullptr));
+      nullptr /* cv_cache */));
   ASSERT_TRUE(result_cv->isNative());
   const NativeColumnVector &native_result_cv
       = static_cast<const NativeColumnVector&>(*result_cv);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/query_optimizer/expressions/CommonSubexpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/CommonSubexpression.hpp b/query_optimizer/expressions/CommonSubexpression.hpp
index ce7589d..2c2d86c 100644
--- a/query_optimizer/expressions/CommonSubexpression.hpp
+++ b/query_optimizer/expressions/CommonSubexpression.hpp
@@ -126,8 +126,8 @@ class CommonSubexpression : public Scalar {
     addChild(operand);
   }
 
-  ExprId common_subexpression_id_;
-  ScalarPtr operand_;
+  const ExprId common_subexpression_id_;
+  const ScalarPtr operand_;
 
   DISALLOW_COPY_AND_ASSIGN(CommonSubexpression);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/query_optimizer/expressions/SimpleCase.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/SimpleCase.cpp b/query_optimizer/expressions/SimpleCase.cpp
index ccdd8e5..b127d51 100644
--- a/query_optimizer/expressions/SimpleCase.cpp
+++ b/query_optimizer/expressions/SimpleCase.cpp
@@ -195,10 +195,8 @@ bool SimpleCase::equals(const ScalarPtr &other) const {
       return false;
     }
   }
-  if ((else_result_expression_ == nullptr
-       || expr->else_result_expression_ == nullptr)
-      && else_result_expression_ != expr->else_result_expression_) {
-    return false;
+  if (else_result_expression_ == nullptr || expr->else_result_expression_ == nullptr) {
+    return else_result_expression_ == expr->else_result_expression_;
   }
   if (!else_result_expression_->equals(expr->else_result_expression_)) {
     return false;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/query_optimizer/rules/CollapseSelection.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CollapseSelection.cpp b/query_optimizer/rules/CollapseSelection.cpp
index e5427b4..f92e1b2 100644
--- a/query_optimizer/rules/CollapseSelection.cpp
+++ b/query_optimizer/rules/CollapseSelection.cpp
@@ -46,7 +46,7 @@ P::PhysicalPtr CollapseSelection::applyToNode(const P::PhysicalPtr &input) {
         selection->project_expressions();
     PullUpProjectExpressions(child_selection->project_expressions(),
                              {} /* non_project_expression_lists */,
-                             {&project_expressions} /* project_expression_lists */);
+                             { &project_expressions } /* project_expression_lists */);
     return P::Selection::Create(child_selection->input(),
                                 project_expressions,
                                 selection->filter_predicate());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/query_optimizer/rules/CollapseSelection.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CollapseSelection.hpp b/query_optimizer/rules/CollapseSelection.hpp
index bc5e4a3..25c3492 100644
--- a/query_optimizer/rules/CollapseSelection.hpp
+++ b/query_optimizer/rules/CollapseSelection.hpp
@@ -43,6 +43,8 @@ class CollapseSelection : public BottomUpRule<physical::Physical> {
    */
   CollapseSelection() {}
 
+  ~CollapseSelection() override {}
+
   std::string getName() const override {
     return "CollapseSelection";
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/query_optimizer/rules/ExtractCommonSubexpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ExtractCommonSubexpression.cpp b/query_optimizer/rules/ExtractCommonSubexpression.cpp
index e3f996c..63b6b17 100644
--- a/query_optimizer/rules/ExtractCommonSubexpression.cpp
+++ b/query_optimizer/rules/ExtractCommonSubexpression.cpp
@@ -258,7 +258,7 @@ E::ExpressionPtr ExtractCommonSubexpression::transformExpression(
 bool ExtractCommonSubexpression::visitAndCount(
     const E::ExpressionPtr &expression,
     ScalarCounter *counter,
-    ScalarHashable *hashable) {
+    ScalarHashable *hashable) const {
   // This bool flag is for avoiding some unnecessary hash() computation.
   bool children_hashable = true;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/query_optimizer/rules/ExtractCommonSubexpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ExtractCommonSubexpression.hpp b/query_optimizer/rules/ExtractCommonSubexpression.hpp
index 3cdd70e..26b09cc 100644
--- a/query_optimizer/rules/ExtractCommonSubexpression.hpp
+++ b/query_optimizer/rules/ExtractCommonSubexpression.hpp
@@ -111,7 +111,7 @@ class ExtractCommonSubexpression : public Rule<physical::Physical> {
   bool visitAndCount(
       const expressions::ExpressionPtr &expression,
       ScalarCounter *counter,
-      ScalarHashable *hashable);
+      ScalarHashable *hashable) const;
 
   // Traverse the expression tree and transform subexpressions (to common
   // subexpressions) if applicable.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/query_optimizer/rules/ReuseAggregateExpressions.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReuseAggregateExpressions.cpp b/query_optimizer/rules/ReuseAggregateExpressions.cpp
index 79dede6..a7c62c6 100644
--- a/query_optimizer/rules/ReuseAggregateExpressions.cpp
+++ b/query_optimizer/rules/ReuseAggregateExpressions.cpp
@@ -157,7 +157,7 @@ P::PhysicalPtr ReuseAggregateExpressions::applyToNode(
     count_star_ref = *it;
 
     for (++it; it != count_star_info.end(); ++it) {
-      agg_refs[*it].reset(new AggregateReference(count_star_ref));
+      agg_refs[*it] = std::make_unique<AggregateReference>(count_star_ref);
     }
   }
 
@@ -194,7 +194,7 @@ P::PhysicalPtr ReuseAggregateExpressions::applyToNode(
             sum_it == ref_map.end() ? kInvalidRef : sum_it->second.front();
 
         for (const std::size_t idx : avg_it->second) {
-          agg_refs[idx].reset(new AggregateReference(sum_ref, count_ref));
+          agg_refs[idx] = std::make_unique<AggregateReference>(sum_ref, count_ref);
         }
         is_avg_processed = true;
       }
@@ -209,7 +209,7 @@ P::PhysicalPtr ReuseAggregateExpressions::applyToNode(
       DCHECK(!indices.empty());
       const std::size_t ref = indices.front();
       for (std::size_t i = 1; i < indices.size(); ++i) {
-        agg_refs[indices[i]].reset(new AggregateReference(ref));
+        agg_refs[indices[i]] = std::make_unique<AggregateReference>(ref);
       }
     }
   }
@@ -329,6 +329,7 @@ P::PhysicalPtr ReuseAggregateExpressions::applyToNode(
                                agg_expr->attribute_name(),
                                agg_expr->attribute_alias(),
                                agg_expr->relation_name()));
+          break;
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30021acf/query_optimizer/rules/ReuseAggregateExpressions.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReuseAggregateExpressions.hpp b/query_optimizer/rules/ReuseAggregateExpressions.hpp
index 182e9d9..12f81d6 100644
--- a/query_optimizer/rules/ReuseAggregateExpressions.hpp
+++ b/query_optimizer/rules/ReuseAggregateExpressions.hpp
@@ -98,6 +98,8 @@ class ReuseAggregateExpressions : public BottomUpRule<physical::Physical> {
   explicit ReuseAggregateExpressions(OptimizerContext *optimizer_context)
       : optimizer_context_(optimizer_context) {}
 
+  ~ReuseAggregateExpressions() override {}
+
   std::string getName() const override {
     return "ReuseAggregateExpressions";
   }


[15/32] incubator-quickstep git commit: Fixed a bug regarding the table name containing '_' in Commands.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/678e2c39/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index 871053f..5cd7426 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -3302,148 +3302,148 @@ yyreduce:
   switch (yyn)
     {
         case 2:
-#line 639 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 639 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     *parsedStatement = (yyvsp[-1].statement_);
     YYACCEPT;
   }
-#line 3311 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3311 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 3:
-#line 643 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 643 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     *parsedStatement = (yyvsp[-1].statement_);
     YYACCEPT;
   }
-#line 3320 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3320 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 4:
-#line 647 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 647 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     *parsedStatement = (yyvsp[-1].command_);
     YYACCEPT;
   }
-#line 3329 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3329 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 5:
-#line 651 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 651 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     *parsedStatement = (yyvsp[-1].command_);
     YYACCEPT;
   }
-#line 3338 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3338 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 6:
-#line 655 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 655 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     YYABORT;
   }
-#line 3346 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3346 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 7:
-#line 658 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 658 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     // Regular yyparse() return codes are non-negative, so use a negative one here.
     return -1;
   }
-#line 3355 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3355 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 8:
-#line 665 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 665 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].statement_);
   }
-#line 3363 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3363 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 9:
-#line 668 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 668 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].copy_from_statement_);
   }
-#line 3371 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3371 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 10:
-#line 671 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 671 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].create_table_statement_);
   }
-#line 3379 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3379 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 11:
-#line 674 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 674 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].statement_);
   }
-#line 3387 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3387 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 12:
-#line 677 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 677 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].delete_statement_);
   }
-#line 3395 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3395 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 13:
-#line 680 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 680 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].drop_table_statement_);
   }
-#line 3403 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3403 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 14:
-#line 683 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 683 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].insert_statement_);
   }
-#line 3411 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3411 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 15:
-#line 686 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 686 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].quit_statement_);
   }
-#line 3419 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3419 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 16:
-#line 689 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 689 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].set_operation_statement_);
   }
-#line 3427 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3427 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 17:
-#line 692 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 692 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.statement_) = (yyvsp[0].update_statement_);
   }
-#line 3435 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3435 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 18:
-#line 698 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 698 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.quit_statement_) = new quickstep::ParseStatementQuit((yylsp[0]).first_line, (yylsp[0]).first_column);
   }
-#line 3443 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3443 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 19:
-#line 704 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 704 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].string_value_);
     delete (yyvsp[0].attribute_definition_);
@@ -3451,22 +3451,22 @@ yyreduce:
     NotSupported(&(yylsp[-5]), yyscanner, "ALTER statements");
     YYERROR;
   }
-#line 3455 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3455 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 20:
-#line 711 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 711 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].string_value_);
     (yyval.statement_) = nullptr;
     NotSupported(&(yylsp[-5]), yyscanner, "ALTER statements");
     YYERROR;
   }
-#line 3466 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3466 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 21:
-#line 717 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 717 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].string_value_);
     delete (yyvsp[0].string_value_);
@@ -3474,11 +3474,11 @@ yyreduce:
     NotSupported(&(yylsp[-5]), yyscanner, "ALTER statements");
     YYERROR;
   }
-#line 3478 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3478 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 22:
-#line 724 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 724 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].string_value_);
     delete (yyvsp[0].string_value_);
@@ -3486,19 +3486,19 @@ yyreduce:
     NotSupported(&(yylsp[-5]), yyscanner, "ALTER statements");
     YYERROR;
   }
-#line 3490 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3490 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 23:
-#line 733 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 733 "../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 3498 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3498 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 24:
-#line 738 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 738 "../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_));
@@ -3506,153 +3506,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 3510 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3510 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 25:
-#line 747 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 747 "../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 3518 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3518 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 26:
-#line 752 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 752 "../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 3526 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3526 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 27:
-#line 757 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 757 "../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 3535 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3535 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 28:
-#line 761 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 761 "../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 3544 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3544 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 29:
-#line 767 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 767 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = nullptr;
     NotSupported(&(yylsp[0]), yyscanner, "BIT data type");
     YYERROR;
   }
-#line 3554 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3554 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 30:
-#line 772 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 772 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDate));
   }
-#line 3562 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3562 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 31:
-#line 775 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 775 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetime));
   }
-#line 3570 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3570 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 32:
-#line 778 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 778 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = nullptr;
     NotSupported(&(yylsp[0]), yyscanner, "TIME data type");
     YYERROR;
   }
-#line 3580 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3580 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 33:
-#line 783 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 783 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetime));
   }
-#line 3588 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3588 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 34:
-#line 786 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 786 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDouble));
   }
-#line 3596 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3596 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 35:
-#line 789 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 789 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDouble));
   }
-#line 3604 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3604 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 36:
-#line 792 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 792 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDouble));
   }
-#line 3612 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3612 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 37:
-#line 795 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 795 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kFloat));
   }
-#line 3620 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3620 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 38:
-#line 798 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 798 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kInt));
   }
-#line 3628 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3628 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 39:
-#line 801 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 801 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kInt));
   }
-#line 3636 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3636 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 40:
-#line 804 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 804 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kLong));
   }
-#line 3644 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3644 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 41:
-#line 807 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 807 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kLong));
   }
-#line 3652 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3652 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 42:
-#line 810 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 810 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /**
      * NOTE(chasseur): This pattern exhibits a shift/reduce conflict with the
@@ -3665,27 +3665,27 @@ yyreduce:
         "or YEARMONTH INTERVAL");
     YYERROR;
   }
-#line 3669 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3669 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 43:
-#line 822 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 822 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetimeInterval));
   }
-#line 3677 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3677 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 44:
-#line 825 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 825 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.data_type_) = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kYearMonthInterval));
   }
-#line 3685 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3685 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 45:
-#line 828 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 828 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[-1].numeric_literal_value_)->float_like()) {
       delete (yyvsp[-1].numeric_literal_value_);
@@ -3704,11 +3704,11 @@ yyreduce:
       }
     }
   }
-#line 3708 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3708 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 46:
-#line 846 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 846 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[-1].numeric_literal_value_)->float_like()) {
       delete (yyvsp[-1].numeric_literal_value_);
@@ -3727,69 +3727,69 @@ yyreduce:
       }
     }
   }
-#line 3731 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3731 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 47:
-#line 866 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 866 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = new quickstep::ParseColumnConstraintNull((yylsp[0]).first_line, (yylsp[0]).first_column);
   }
-#line 3739 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3739 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 48:
-#line 869 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 869 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = new quickstep::ParseColumnConstraintNotNull((yylsp[-1]).first_line, (yylsp[-1]).first_column);
   }
-#line 3747 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3747 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 49:
-#line 872 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 872 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     NotSupported(&(yylsp[0]), yyscanner, "Column Constraints (UNIQUE)");
     YYERROR;
   }
-#line 3757 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3757 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 50:
-#line 877 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 877 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     NotSupported(&(yylsp[-1]), yyscanner, "Column Constraints (PRIMARY KEY)");
     YYERROR;
   }
-#line 3767 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3767 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 51:
-#line 882 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 882 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     delete (yyvsp[0].literal_value_);
     NotSupported(&(yylsp[-1]), yyscanner, "Column Constraints (DEFAULT)");
     YYERROR;
   }
-#line 3778 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3778 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 52:
-#line 888 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 888 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     delete (yyvsp[-1].predicate_);
     NotSupported(&(yylsp[-3]), yyscanner, "Column Constraints (CHECK)");
     YYERROR;
   }
-#line 3789 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3789 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 53:
-#line 894 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 894 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_) = nullptr;
     delete (yyvsp[-3].string_value_);
@@ -3797,65 +3797,65 @@ yyreduce:
     NotSupported(&(yylsp[-4]), yyscanner, "Foreign Keys");
     YYERROR;
   }
-#line 3801 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3801 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 54:
-#line 903 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 903 "../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 3810 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3810 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 55:
-#line 907 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 907 "../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 3819 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3819 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 56:
-#line 913 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 913 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_list_) = nullptr;
   }
-#line 3827 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3827 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 57:
-#line 916 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 916 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.column_constraint_list_) = (yyvsp[0].column_constraint_list_);
   }
-#line 3835 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3835 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 58:
-#line 921 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 921 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-1].string_list_);
     NotSupported(&(yylsp[-3]), yyscanner, "Table Constraints (UNIQUE)");
     YYERROR;
   }
-#line 3845 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3845 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 59:
-#line 926 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 926 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-1].string_list_);
     NotSupported(&(yylsp[-4]), yyscanner, "Table Constraints (PRIMARY KEY)");
     YYERROR;
   }
-#line 3855 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3855 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 60:
-#line 931 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 931 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-6].string_list_);
     delete (yyvsp[-3].string_value_);
@@ -3863,95 +3863,95 @@ yyreduce:
     NotSupported(&(yylsp[-9]), yyscanner, "Table Constraints (FOREIGN KEY)");
     YYERROR;
   }
-#line 3867 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3867 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 61:
-#line 938 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 938 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-1].predicate_);
     NotSupported(&(yylsp[-3]), yyscanner, "Table Constraints (CHECK)");
     YYERROR;
   }
-#line 3877 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3877 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 62:
-#line 945 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 945 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     NotSupported(&(yylsp[-2]), yyscanner, "Table Constraints");
     YYERROR;
   }
-#line 3886 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3886 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 63:
-#line 949 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 949 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     NotSupported(&(yylsp[0]), yyscanner, "Table Constraints");
     YYERROR;
   }
-#line 3895 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3895 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 64:
-#line 955 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 955 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /* $$ = nullptr; */
   }
-#line 3903 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3903 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 65:
-#line 958 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 958 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     /* $$ = $1; */
   }
-#line 3911 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3911 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 66:
-#line 963 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 963 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_list_) = nullptr;
   }
-#line 3919 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3919 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 67:
-#line 966 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 966 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.attribute_list_) = (yyvsp[-1].attribute_list_);
   }
-#line 3927 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3927 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 68:
-#line 971 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 971 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.block_properties_) = nullptr;
   }
-#line 3935 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3935 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 69:
-#line 974 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 974 "../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 3943 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3943 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 70:
-#line 979 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 979 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.partition_clause_) = nullptr;
   }
-#line 3951 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3951 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 71:
-#line 982 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 982 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[0].numeric_literal_value_)->float_like()) {
       delete (yyvsp[0].numeric_literal_value_);
@@ -3969,95 +3969,95 @@ yyreduce:
       }
     }
   }
-#line 3973 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3973 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 72:
-#line 1001 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1001 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column, quickstep::kHashPartitionType);
   }
-#line 3981 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3981 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 73:
-#line 1004 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1004 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.string_value_) = new quickstep::ParseString((yylsp[0]).first_line, (yylsp[0]).first_column, quickstep::kRangePartitionType);
   }
-#line 3989 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3989 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 74:
-#line 1009 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1009 "../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 3998 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 3998 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 75:
-#line 1013 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1013 "../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 4007 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4007 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 76:
-#line 1019 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1019 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_) = (yyvsp[0].key_string_value_);
   }
-#line 4015 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4015 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 77:
-#line 1022 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1022 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_) = (yyvsp[0].key_string_list_);
   }
-#line 4023 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4023 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 78:
-#line 1025 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1025 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_) = (yyvsp[0].key_integer_value_);
   }
-#line 4031 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4031 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 79:
-#line 1030 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1030 "../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 4039 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4039 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 80:
-#line 1033 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1033 "../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 4049 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4049 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 81:
-#line 1040 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1040 "../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 4057 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4057 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 82:
-#line 1045 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1045 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[0].numeric_literal_value_)->float_like()) {
       delete (yyvsp[0].numeric_literal_value_);
@@ -4067,64 +4067,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 4071 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4071 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 83:
-#line 1056 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1056 "../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 4081 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4081 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 84:
-#line 1061 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1061 "../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 4090 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4090 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 85:
-#line 1065 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1065 "../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 4099 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4099 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 86:
-#line 1069 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1069 "../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 4108 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4108 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 87:
-#line 1075 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1075 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_list_) = nullptr;
   }
-#line 4116 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4116 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 88:
-#line 1078 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1078 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.key_value_list_) = (yyvsp[-1].key_value_list_);
   }
-#line 4124 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4124 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 89:
-#line 1084 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1084 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-7].string_value_);
     delete (yyvsp[-5].string_list_);
@@ -4133,161 +4133,161 @@ yyreduce:
     NotSupported(&(yylsp[-6]), yyscanner, "list of column names in INSERT statement");
     YYERROR;
   }
-#line 4137 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4137 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 90:
-#line 1092 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1092 "../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 4145 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4145 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 91:
-#line 1095 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1095 "../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 4153 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4153 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 92:
-#line 1098 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1098 "../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 4161 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4161 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 93:
-#line 1104 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1104 "../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 4169 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4169 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 94:
-#line 1109 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1109 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.copy_from_params_) = nullptr;
   }
-#line 4177 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4177 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 95:
-#line 1112 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1112 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.copy_from_params_) = (yyvsp[-1].copy_from_params_);
   }
-#line 4185 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4185 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 96:
-#line 1117 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1117 "../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 4194 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4194 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 97:
-#line 1121 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1121 "../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 4203 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4203 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 98:
-#line 1125 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1125 "../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 4212 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4212 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 99:
-#line 1129 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1129 "../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 4221 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4221 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 100:
-#line 1135 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1135 "../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 4229 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4229 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 101:
-#line 1140 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1140 "../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 4237 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4237 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 102:
-#line 1145 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1145 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.assignment_list_) = (yyvsp[-2].assignment_list_);
     (yyval.assignment_list_)->push_back((yyvsp[0].assignment_));
   }
-#line 4246 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4246 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 103:
-#line 1149 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1149 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.assignment_list_) = new quickstep::PtrList<quickstep::ParseAssignment>();
     (yyval.assignment_list_)->push_back((yyvsp[0].assignment_));
   }
-#line 4255 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4255 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 104:
-#line 1155 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1155 "../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 4263 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4263 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 105:
-#line 1164 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1164 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.set_operation_statement_) = new quickstep::ParseStatementSetOperation((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].set_operation_), nullptr, (yyvsp[0].opt_priority_clause_));
   }
-#line 4271 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4271 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 106:
-#line 1167 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1167 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.set_operation_statement_) = new quickstep::ParseStatementSetOperation((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-1].set_operation_), (yyvsp[-2].with_list_), (yyvsp[0].opt_priority_clause_));
   }
-#line 4279 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4279 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 107:
-#line 1172 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1172 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_priority_clause_) = nullptr;
   }
-#line 4287 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4287 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 108:
-#line 1175 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1175 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[0].numeric_literal_value_)->float_like()) {
       delete (yyvsp[0].numeric_literal_value_);
@@ -4305,46 +4305,46 @@ yyreduce:
       }
     }
   }
-#line 4309 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4309 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 109:
-#line 1194 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1194 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.with_list_) = (yyvsp[0].with_list_);
   }
-#line 4317 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4317 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 110:
-#line 1199 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1199 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.with_list_) = new quickstep::PtrVector<quickstep::ParseSubqueryTableReference>();
     (yyval.with_list_)->push_back((yyvsp[0].with_list_element_));
   }
-#line 4326 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4326 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 111:
-#line 1203 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1203 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.with_list_) = (yyvsp[-2].with_list_);
     (yyval.with_list_)->push_back((yyvsp[0].with_list_element_));
   }
-#line 4335 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4335 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 112:
-#line 1209 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1209 "../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 4344 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4344 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 113:
-#line 1215 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1215 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[-1].boolean_value_)) {
       (yyval.set_operation_) = new quickstep::ParseSetOperation((yylsp[-3]).first_line, (yylsp[-3]).first_column, quickstep::ParseSetOperation::kUnion);
@@ -4354,19 +4354,19 @@ yyreduce:
     (yyval.set_operation_)->addOperand((yyvsp[-3].set_operation_));
     (yyval.set_operation_)->addOperand((yyvsp[0].set_operation_));
   }
-#line 4358 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4358 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 114:
-#line 1224 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1224 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.set_operation_) = (yyvsp[0].set_operation_);
   }
-#line 4366 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4366 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 115:
-#line 1229 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1229 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.set_operation_) = new quickstep::ParseSetOperation((yylsp[-2]).first_line, (yylsp[-2]).first_column, quickstep::ParseSetOperation::kIntersect);
     quickstep::ParseSetOperation *op = new quickstep::ParseSetOperation(
@@ -4375,387 +4375,387 @@ yyreduce:
     (yyval.set_operation_)->addOperand((yyvsp[-2].set_operation_));
     (yyval.set_operation_)->addOperand(op);
   }
-#line 4379 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4379 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 116:
-#line 1237 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1237 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.set_operation_) = new quickstep::ParseSetOperation((yylsp[0]).first_line, (yylsp[0]).first_column, quickstep::ParseSetOperation::kSelect);
     (yyval.set_operation_)->addOperand((yyvsp[0].select_query_));
   }
-#line 4388 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4388 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 117:
-#line 1244 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1244 "../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 4396 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4396 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 118:
-#line 1249 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1249 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.boolean_value_) = true;  // Distinct
   }
-#line 4404 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4404 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 119:
-#line 1252 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1252 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.boolean_value_) = false;  // All
   }
-#line 4412 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4412 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 120:
-#line 1255 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1255 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.boolean_value_) = true;  // Distinct
   }
-#line 4420 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4420 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 121:
-#line 1260 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1260 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.selection_) = new quickstep::ParseSelectionStar((yylsp[0]).first_line, (yylsp[0]).first_column);
   }
-#line 4428 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4428 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 122:
-#line 1263 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1263 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.selection_) = (yyvsp[0].selection_list_);
   }
-#line 4436 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4436 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 123:
-#line 1268 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1268 "../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 4445 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4445 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 124:
-#line 1272 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1272 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.selection_list_) = (yyvsp[-2].selection_list_);
     (yyval.selection_list_)->add((yyvsp[0].selection_item_));
   }
-#line 4454 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4454 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 125:
-#line 1278 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1278 "../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 4462 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4462 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 126:
-#line 1281 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1281 "../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 4470 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4470 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 127:
-#line 1284 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1284 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.selection_item_) = new quickstep::ParseSelectionItem((yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].expression_));
   }
-#line 4478 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4478 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 128:
-#line 1289 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1289 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_list_) = (yyvsp[0].table_reference_list_);
   }
-#line 4486 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4486 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 129:
-#line 1294 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1294 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.subquery_expression_) = new quickstep::ParseSubqueryExpression((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-1].set_operation_));
   }
-#line 4494 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4494 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 130:
-#line 1299 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1299 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_sample_clause_) = NULL;
   }
-#line 4502 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4502 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 131:
-#line 1302 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1302 "../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 4510 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4510 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 132:
-#line 1305 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1305 "../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 4518 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4518 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 133:
-#line 1310 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1310 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kInnerJoin;
   }
-#line 4526 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4526 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 134:
-#line 1313 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1313 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kInnerJoin;
   }
-#line 4534 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4534 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 135:
-#line 1316 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1316 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kLeftOuterJoin;
   }
-#line 4542 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4542 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 136:
-#line 1319 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1319 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kLeftOuterJoin;
   }
-#line 4550 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4550 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 137:
-#line 1322 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1322 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kRightOuterJoin;
   }
-#line 4558 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4558 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 138:
-#line 1325 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1325 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kRightOuterJoin;
   }
-#line 4566 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4566 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 139:
-#line 1328 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1328 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kFullOuterJoin;
   }
-#line 4574 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4574 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 140:
-#line 1331 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1331 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.join_type_) = quickstep::ParseJoinedTableReference::JoinType::kFullOuterJoin;
   }
-#line 4582 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4582 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 141:
-#line 1336 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1336 "../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 4590 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4590 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 142:
-#line 1339 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1339 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_) = (yyvsp[0].table_reference_);
   }
-#line 4598 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4598 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 143:
-#line 1344 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1344 "../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 4607 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4607 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 144:
-#line 1348 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1348 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_) = new quickstep::ParseSimpleTableReference((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].string_value_), (yyvsp[-1].opt_sample_clause_));
     (yyval.table_reference_)->set_table_reference_signature((yyvsp[0].table_reference_signature_));
   }
-#line 4616 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4616 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 145:
-#line 1352 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1352 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_) = new quickstep::ParseSimpleTableReference((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].string_value_), (yyvsp[0].opt_sample_clause_));
   }
-#line 4624 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4624 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 146:
-#line 1355 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1355 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_) = new quickstep::ParseGeneratorTableReference((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-1].function_call_));
     (yyval.table_reference_)->set_table_reference_signature((yyvsp[0].table_reference_signature_));
   }
-#line 4633 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4633 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 147:
-#line 1359 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1359 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_) = new quickstep::ParseGeneratorTableReference((yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].function_call_));
   }
-#line 4641 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4641 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 148:
-#line 1362 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1362 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_) = (yyvsp[-1].table_reference_);
   }
-#line 4649 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4649 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 149:
-#line 1367 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1367 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_signature_) = (yyvsp[0].table_reference_signature_);
   }
-#line 4657 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4657 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 150:
-#line 1370 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1370 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_signature_) = (yyvsp[0].table_reference_signature_);
   }
-#line 4665 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4665 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 151:
-#line 1375 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1375 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_signature_) = new ::quickstep::ParseTableReferenceSignature((yylsp[0]).first_line, (yylsp[0]).first_column, (yyvsp[0].string_value_));
   }
-#line 4673 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4673 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 152:
-#line 1378 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1378 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_signature_) = new ::quickstep::ParseTableReferenceSignature((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].string_list_));
   }
-#line 4681 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4681 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 153:
-#line 1383 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1383 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_list_) = new quickstep::PtrList<quickstep::ParseTableReference>();
     (yyval.table_reference_list_)->push_back((yyvsp[0].table_reference_));
   }
-#line 4690 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4690 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 154:
-#line 1387 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1387 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.table_reference_list_) = (yyvsp[-2].table_reference_list_);
     (yyval.table_reference_list_)->push_back((yyvsp[0].table_reference_));
   }
-#line 4699 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4699 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 155:
-#line 1393 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1393 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_group_by_clause_) = nullptr;
   }
-#line 4707 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4707 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 156:
-#line 1396 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1396 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_group_by_clause_) = new quickstep::ParseGroupBy((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[0].expression_list_));
   }
-#line 4715 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4715 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 157:
-#line 1401 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1401 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_having_clause_) = nullptr;
   }
-#line 4723 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4723 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 158:
-#line 1404 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1404 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_having_clause_) = new quickstep::ParseHaving((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[0].predicate_));
   }
-#line 4731 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4731 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 159:
-#line 1409 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1409 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_order_by_clause_) = nullptr;
   }
-#line 4739 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4739 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 160:
-#line 1412 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1412 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_order_by_clause_) = new quickstep::ParseOrderBy((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[0].order_commalist_));
   }
-#line 4747 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4747 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 161:
-#line 1417 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1417 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_limit_clause_) = nullptr;
   }
-#line 4755 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4755 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 162:
-#line 1420 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1420 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[0].numeric_literal_value_)->float_like()) {
       delete (yyvsp[0].numeric_literal_value_);
@@ -4773,258 +4773,258 @@ yyreduce:
       }
     }
   }
-#line 4777 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4777 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 163:
-#line 1439 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1439 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_window_clause_) = nullptr;
   }
-#line 4785 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4785 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 164:
-#line 1442 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1442 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_window_clause_) = (yyvsp[0].opt_window_clause_);
   }
-#line 4793 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4793 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 165:
-#line 1447 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1447 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_window_clause_) = new quickstep::PtrList<quickstep::ParseWindow>();
     (yyval.opt_window_clause_)->push_back((yyvsp[0].window_definition_));
   }
-#line 4802 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4802 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 166:
-#line 1451 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1451 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.opt_window_clause_) = (yyvsp[-1].opt_window_clause_);
     (yyval.opt_window_clause_)->push_back((yyvsp[0].window_definition_));
   }
-#line 4811 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4811 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 167:
-#line 1457 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1457 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.window_definition_) = (yyvsp[-1].window_definition_);
     (yyval.window_definition_)->setName((yyvsp[-4].string_value_));
   }
-#line 4820 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4820 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 168:
-#line 1463 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1463 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.window_definition_) = new quickstep::ParseWindow((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].window_partition_by_list_), (yyvsp[-1].window_order_by_list_), (yyvsp[0].window_frame_info_));
   }
-#line 4828 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4828 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 169:
-#line 1468 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1468 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.window_partition_by_list_) = nullptr;
   }
-#line 4836 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4836 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 170:
-#line 1471 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1471 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.window_partition_by_list_) = (yyvsp[0].expression_list_);
   }
-#line 4844 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4844 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 171:
-#line 1476 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1476 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.window_order_by_list_) = nullptr;
   }
-#line 4852 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4852 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 172:
-#line 1479 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1479 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.window_order_by_list_) = (yyvsp[0].order_commalist_);
   }
-#line 4860 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4860 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 173:
-#line 1484 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1484 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.window_frame_info_) = nullptr;
   }
-#line 4868 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4868 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 174:
-#line 1487 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1487 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.window_frame_info_) = new quickstep::ParseFrameInfo((yylsp[-4]).first_line, (yylsp[-4]).first_column, (yyvsp[-4].boolean_value_), (yyvsp[-2].numeric_literal_value_)->long_value(), (yyvsp[0].numeric_literal_value_)->long_value());
   }
-#line 4876 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4876 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 175:
-#line 1492 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1492 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.boolean_value_) = true;
   }
-#line 4884 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4884 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 176:
-#line 1495 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1495 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.boolean_value_) = false;
   }
-#line 4892 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4892 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 178:
-#line 1501 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1501 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.numeric_literal_value_) = new quickstep::NumericParseLiteralValue((yylsp[-1]).first_line, (yylsp[-1]).first_column, "-1");
   }
-#line 4900 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4900 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 179:
-#line 1504 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1504 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.numeric_literal_value_) = new quickstep::NumericParseLiteralValue((yylsp[-1]).first_line, (yylsp[-1]).first_column, "0");
   }
-#line 4908 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4908 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 181:
-#line 1510 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1510 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.numeric_literal_value_) = new quickstep::NumericParseLiteralValue((yylsp[-1]).first_line, (yylsp[-1]).first_column, "-1");
   }
-#line 4916 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4916 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 182:
-#line 1513 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1513 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.numeric_literal_value_) = new quickstep::NumericParseLiteralValue((yylsp[-1]).first_line, (yylsp[-1]).first_column, "0");
   }
-#line 4924 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4924 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 183:
-#line 1518 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1518 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.order_commalist_) = new quickstep::PtrList<quickstep::ParseOrderByItem>();
     (yyval.order_commalist_)->push_back((yyvsp[0].order_item_));
   }
-#line 4933 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4933 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 184:
-#line 1522 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1522 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.order_commalist_) = (yyvsp[-2].order_commalist_);
     (yyval.order_commalist_)->push_back((yyvsp[0].order_item_));
   }
-#line 4942 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4942 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 185:
-#line 1528 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1528 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.order_item_) = new quickstep::ParseOrderByItem((yylsp[-2]).first_line, (yylsp[-2]).first_column, (yyvsp[-2].expression_), (yyvsp[-1].order_direction_), (yyvsp[0].order_direction_));
     delete (yyvsp[-1].order_direction_);
     delete (yyvsp[0].order_direction_);
   }
-#line 4952 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4952 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 186:
-#line 1535 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1535 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.order_direction_) = nullptr;
   }
-#line 4960 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4960 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 187:
-#line 1538 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1538 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.order_direction_) = new bool(true);
   }
-#line 4968 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4968 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 188:
-#line 1541 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1541 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.order_direction_) = new bool(false);
   }
-#line 4976 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4976 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 189:
-#line 1546 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1546 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.order_direction_) = nullptr;
   }
-#line 4984 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4984 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 190:
-#line 1549 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1549 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.order_direction_) = new bool(true);
   }
-#line 4992 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 4992 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 191:
-#line 1552 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1552 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.order_direction_) = new bool(false);
   }
-#line 5000 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5000 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 192:
-#line 1558 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1558 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = nullptr;
   }
-#line 5008 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5008 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 193:
-#line 1561 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1561 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = (yyvsp[0].predicate_);
   }
-#line 5016 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5016 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 194:
-#line 1566 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1566 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = (yyvsp[0].predicate_);
   }
-#line 5024 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5024 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 195:
-#line 1571 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1571 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[-2].predicate_)->getParsePredicateType() == quickstep::ParsePredicate::kDisjunction) {
       (yyval.predicate_) = (yyvsp[-2].predicate_);
@@ -5034,19 +5034,19 @@ yyreduce:
     }
     static_cast<quickstep::ParsePredicateDisjunction *>((yyval.predicate_))->addPredicate((yyvsp[0].predicate_));
   }
-#line 5038 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5038 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 196:
-#line 1580 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1580 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = (yyvsp[0].predicate_);
   }
-#line 5046 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5046 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 197:
-#line 1585 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1585 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     if ((yyvsp[-2].predicate_)->getParsePredicateType() == quickstep::ParsePredicate::kConjunction) {
       (yyval.predicate_) = (yyvsp[-2].predicate_);
@@ -5056,453 +5056,453 @@ yyreduce:
     }
     static_cast<quickstep::ParsePredicateConjunction *>((yyval.predicate_))->addPredicate((yyvsp[0].predicate_));
   }
-#line 5060 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5060 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 198:
-#line 1594 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1594 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = (yyvsp[0].predicate_);
   }
-#line 5068 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5068 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 199:
-#line 1599 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1599 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = new quickstep::ParsePredicateNegation((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[0].predicate_));
   }
-#line 5076 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5076 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 200:
-#line 1602 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1602 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = (yyvsp[0].predicate_);
   }
-#line 5084 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5084 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 201:
-#line 1607 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1607 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = new quickstep::ParsePredicateBetween((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-4].expression_), (yyvsp[-2].expression_), (yyvsp[0].expression_));
   }
-#line 5092 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5092 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 202:
-#line 1610 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1610 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = new quickstep::ParsePredicateNegation(
         (yylsp[-4]).first_line, (yylsp[-4]).first_column,
         new quickstep::ParsePredicateBetween((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-5].expression_), (yyvsp[-2].expression_), (yyvsp[0].expression_)));
   }
-#line 5102 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5102 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 203:
-#line 1615 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1615 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-3].attribute_);
     (yyval.predicate_) = nullptr;
     NotSupported(&(yylsp[-2]), yyscanner, "NULL comparison predicates");
     YYERROR;
   }
-#line 5113 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5113 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 204:
-#line 1621 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1621 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     delete (yyvsp[-2].attribute_);
     (yyval.predicate_) = nullptr;
     NotSupported(&(yylsp[-1]), yyscanner, "NULL comparison predicates");
     YYERROR;
   }
-#line 5124 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5124 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 205:
-#line 1627 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1627 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = new quickstep::ParsePredicateComparison((yylsp[-1]).first_line, (yylsp[-1]).first_column, *(yyvsp[-1].comparison_), (yyvsp[-2].expression_), (yyvsp[0].expression_));
   }
-#line 5132 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5132 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 206:
-#line 1630 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1630 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = (yyvsp[-1].predicate_);
   }
-#line 5140 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5140 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 207:
-#line 1633 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1633 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = new quickstep::ParsePredicateExists((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[0].subquery_expression_));
   }
-#line 5148 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5148 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 208:
-#line 1636 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1636 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = new quickstep::ParsePredicateInTableQuery((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-2].expression_), (yyvsp[0].subquery_expression_));
   }
-#line 5156 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5156 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 209:
-#line 1639 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1639 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = new quickstep::ParsePredicateInValueList((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-4].expression_), (yyvsp[-1].expression_list_));
   }
-#line 5164 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5164 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 210:
-#line 1642 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1642 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = new quickstep::ParsePredicateNegation(
         (yylsp[-2]).first_line,
         (yylsp[-2]).first_column,
         new quickstep::ParsePredicateInTableQuery((yylsp[-1]).first_line, (yylsp[-1]).first_column, (yyvsp[-3].expression_), (yyvsp[0].subquery_expression_)));
   }
-#line 5175 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5175 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 211:
-#line 1648 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1648 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.predicate_) = new quickstep::ParsePredicateNegation(
         (yylsp[-4]).first_line,
         (yylsp[-4]).first_column,
         new quickstep::ParsePredicateInValueList((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-5].expression_), (yyvsp[-1].expression_list_)));
   }
-#line 5186 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5186 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 212:
-#line 1657 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1657 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = new quickstep::ParseBinaryExpression((yylsp[-1]).first_line, (yylsp[-1]).first_column, *(yyvsp[-1].binary_operation_), (yyvsp[-2].expression_), (yyvsp[0].expression_));
   }
-#line 5194 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5194 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 213:
-#line 1660 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1660 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = (yyvsp[0].expression_);
   }
-#line 5202 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5202 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 214:
-#line 1665 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1665 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = new quickstep::ParseBinaryExpression((yylsp[-1]).first_line, (yylsp[-1]).first_column, *(yyvsp[-1].binary_operation_), (yyvsp[-2].expression_), (yyvsp[0].expression_));
   }
-#line 5210 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5210 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 215:
-#line 1668 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1668 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = (yyvsp[0].expression_);
   }
-#line 5218 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5218 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 216:
-#line 1673 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1673 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = new quickstep::ParseUnaryExpression((yylsp[-1]).first_line, (yylsp[-1]).first_column, *(yyvsp[-1].unary_operation_), (yyvsp[0].expression_));
   }
-#line 5226 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5226 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 217:
-#line 1676 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1676 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = (yyvsp[0].expression_);
   }
-#line 5234 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5234 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 218:
-#line 1681 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1681 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = (yyvsp[0].attribute_);
   }
-#line 5242 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5242 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 219:
-#line 1684 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1684 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = new quickstep::ParseScalarLiteral((yyvsp[0].literal_value_));
   }
-#line 5250 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5250 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 220:
-#line 1687 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1687 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = (yyvsp[0].function_call_);
   }
-#line 5258 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5258 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 221:
-#line 1690 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1690 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyvsp[-2].function_call_)->setWindowName((yyvsp[0].string_value_));
     (yyval.expression_) = (yyvsp[-2].function_call_);
   }
-#line 5267 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5267 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 222:
-#line 1694 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1694 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyvsp[-4].function_call_)->setWindow((yyvsp[-1].window_definition_));
     (yyval.expression_) = (yyvsp[-4].function_call_);
   }
-#line 5276 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5276 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 223:
-#line 1698 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1698 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = (yyvsp[0].expression_);
   }
-#line 5284 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5284 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 224:
-#line 1701 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1701 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = (yyvsp[0].expression_);
   }
-#line 5292 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5292 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 225:
-#line 1704 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1704 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = (yyvsp[0].expression_);
   }
-#line 5300 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5300 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 226:
-#line 1707 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1707 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = (yyvsp[-1].expression_);
   }
-#line 5308 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5308 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 227:
-#line 1710 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1710 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = (yyvsp[0].subquery_expression_);
   }
-#line 5316 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5316 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 228:
-#line 1715 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1715 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.function_call_) = new quickstep::ParseFunctionCall(
         (yylsp[-2]).first_line, (yylsp[-2]).first_column, false, (yyvsp[-2].string_value_), new quickstep::PtrList<quickstep::ParseExpression>());
   }
-#line 5325 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5325 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 229:
-#line 1719 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1719 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.function_call_) = new quickstep::ParseFunctionCall(
         (yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-3].string_value_), new quickstep::ParseStar((yylsp[-1]).first_line, (yylsp[-1]).first_column));
   }
-#line 5334 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5334 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 230:
-#line 1723 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1723 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.function_call_) = new quickstep::ParseFunctionCall((yylsp[-3]).first_line, (yylsp[-3]).first_column, false, (yyvsp[-3].string_value_), (yyvsp[-1].expression_list_));
   }
-#line 5342 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5342 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 231:
-#line 1726 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1726 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.function_call_) = new quickstep::ParseFunctionCall((yylsp[-4]).first_line, (yylsp[-4]).first_column, true, (yyvsp[-4].string_value_), (yyvsp[-1].expression_list_));
   }
-#line 5350 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5350 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 232:
-#line 1731 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1731 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = new quickstep::ParseExtractFunction((yylsp[-5]).first_line, (yylsp[-5]).first_column, (yyvsp[-3].string_value_), (yyvsp[-1].expression_));
   }
-#line 5358 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5358 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 233:
-#line 1736 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1736 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = new quickstep::ParseSubstringFunction(
         (yylsp[-5]).first_line, (yylsp[-5]).first_column, (yyvsp[-3].expression_), (yyvsp[-1].numeric_literal_value_)->long_value());
   }
-#line 5367 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5367 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 234:
-#line 1740 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1740 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = new quickstep::ParseSubstringFunction(
         (yylsp[-7]).first_line, (yylsp[-7]).first_column, (yyvsp[-5].expression_), (yyvsp[-3].numeric_literal_value_)->long_value(), (yyvsp[-1].numeric_literal_value_)->long_value());
   }
-#line 5376 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5376 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 235:
-#line 1746 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1746 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = new quickstep::ParseSimpleCaseExpression((yylsp[-4]).first_line, (yylsp[-4]).first_column, (yyvsp[-3].expression_), (yyvsp[-2].simple_when_clause_list_), (yyvsp[-1].expression_));
   }
-#line 5384 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5384 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 236:
-#line 1749 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1749 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.expression_) = new quickstep::ParseSearchedCaseExpression((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-2].searched_when_clause_list_), (yyvsp[-1].expression_));
   }
-#line 5392 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5392 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 237:
-#line 1754 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1754 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.simple_when_clause_list_) = new quickstep::PtrVector<quickstep::ParseSimpleWhenClause>;
     (yyval.simple_when_clause_list_)->push_back((yyvsp[0].simple_when_clause_));
   }
-#line 5401 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5401 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 238:
-#line 1758 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1758 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.simple_when_clause_list_) = (yyvsp[-1].simple_when_clause_list_);
     (yyval.simple_when_clause_list_)->push_back((yyvsp[0].simple_when_clause_));
   }
-#line 5410 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5410 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 239:
-#line 1764 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1764 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.simple_when_clause_) = new quickstep::ParseSimpleWhenClause((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-2].expression_), (yyvsp[0].expression_));
   }
-#line 5418 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5418 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 240:
-#line 1769 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1769 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.searched_when_clause_list_) = new quickstep::PtrVector<quickstep::ParseSearchedWhenClause>;
     (yyval.searched_when_clause_list_)->push_back((yyvsp[0].searched_when_clause_));
   }
-#line 5427 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5427 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 241:
-#line 1773 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1773 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.searched_when_clause_list_) = (yyvsp[-1].searched_when_clause_list_);
     (yyval.searched_when_clause_list_)->push_back((yyvsp[0].searched_when_clause_));
   }
-#line 5436 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5436 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;
 
   case 242:
-#line 1779 "../SqlParser.ypp" /* yacc.c:1646  */
+#line 1779 "../SqlParser.ypp" /* yacc.c:1661  */
     {
     (yyval.searched_when_clause_) = new quickstep::ParseSearchedWhenClause((yylsp[-3]).first_line, (yylsp[-3]).first_column, (yyvsp[-2].predicate_), (yyvsp[0].expression_));
   }
-#line 5444 "SqlParser_gen.cpp" /* yacc.c:1646  */
+#line 5444 "SqlParser_gen.cpp" /* yacc.c:1661  */
     break;


<TRUNCATED>


[23/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryOperators.hpp b/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
deleted file mode 100644
index 7224a0c..0000000
--- a/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
+++ /dev/null
@@ -1,848 +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_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATORS_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATORS_HPP_
-
-#include <cmath>
-#include <cstddef>
-#include <cstdint>
-#include <type_traits>
-#include <utility>
-#include <vector>
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/StorageBlockInfo.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
-#include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-// We use these functors instead of the standard-library ones, because the
-// standard-library functors in <functional> have to be instantiated for the
-// most specific argument type, which would unnecessisarily introduce
-// multiple copies of distinct template instantiations of operators.
-template <typename LeftArgument, typename RightArgument> struct AddFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &right) const -> decltype(left + right) {
-    return left + right;
-  }
-};
-
-// NOTE(zuyu): The C++ compiler in general converts all integers to floats
-//             when doing the following operations,
-//             but we could like to return double instead.
-template <>
-struct AddFunctor<std::int64_t, float> {
-  inline double operator() (const std::int64_t &left, const float &right) const {
-    return static_cast<double>(left) + static_cast<double>(right);
-  }
-};
-
-template <>
-struct AddFunctor<float, std::int64_t> {
-  inline double operator() (const float &left, const std::int64_t &right) const {
-    return static_cast<double>(left) + static_cast<double>(right);
-  }
-};
-
-template <typename LeftArgument, typename RightArgument> struct SubtractFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &right) const -> decltype(left - right) {
-    return left - right;
-  }
-};
-
-// NOTE(zuyu): The C++ compiler in general converts all integers to floats
-//             when doing the following operations,
-//             but we could like to return double instead.
-template <>
-struct SubtractFunctor<std::int64_t, float> {
-  inline double operator() (const std::int64_t &left, const float &right) const {
-    return static_cast<double>(left) - static_cast<double>(right);
-  }
-};
-
-template <>
-struct SubtractFunctor<float, std::int64_t> {
-  inline double operator() (const float &left, const std::int64_t &right) const {
-    return static_cast<double>(left) - static_cast<double>(right);
-  }
-};
-
-template <typename LeftArgument, typename RightArgument> struct MultiplyFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &right) const -> decltype(left * right) {
-    return left * right;
-  }
-};
-
-// NOTE(zuyu): The C++ compiler in general converts all integers to floats
-//             when doing the following operations,
-//             but we could like to return double instead.
-template <>
-struct MultiplyFunctor<std::int64_t, float> {
-  inline double operator() (const std::int64_t &left, const float &right) const {
-    return static_cast<double>(left) * static_cast<double>(right);
-  }
-};
-
-template <>
-struct MultiplyFunctor<float, std::int64_t> {
-  inline double operator() (const float &left, const std::int64_t &right) const {
-    return static_cast<double>(left) * static_cast<double>(right);
-  }
-};
-
-template <typename LeftArgument, typename RightArgument> struct DivideFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &right) const -> decltype(left / right) {
-    return left / right;
-  }
-};
-
-// NOTE(zuyu): The C++ compiler in general converts all integers to floats
-//             when doing the following operations,
-//             but we could like to return double instead.
-template <>
-struct DivideFunctor<std::int64_t, float> {
-  inline double operator() (const std::int64_t &left, const float &right) const {
-    return static_cast<double>(left) / static_cast<double>(right);
-  }
-};
-
-template <>
-struct DivideFunctor<float, std::int64_t> {
-  inline double operator() (const float &left, const std::int64_t &right) const {
-    return static_cast<double>(left) / static_cast<double>(right);
-  }
-};
-
-template <typename LeftArgument, typename RightArgument> struct IntegerModuloFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &right) const -> decltype(left % right) {
-    return left % right;
-  }
-};
-
-// NOTE(jianqiao): The C++11 standard specifies the following type signatures for fmod:
-// (1) (double, double) -> double
-// (2) (float, float) -> float
-// (3) (long double, long double) -> long double
-// (3) (Arithmetic, Arithmetic) -> double
-template <typename LeftArgument, typename RightArgument> struct FloatModuloFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &right) const
-      -> decltype(std::fmod(left, right)) {
-    return std::fmod(left, right);
-  }
-};
-
-template <template <typename LeftCppType, typename RightCppType> class OpFunctor,
-          typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-class ArithmeticUncheckedBinaryOperator : public UncheckedBinaryOperator {
- public:
-  ArithmeticUncheckedBinaryOperator() = default;
-  ArithmeticUncheckedBinaryOperator(const ArithmeticUncheckedBinaryOperator &orig) = default;
-  ~ArithmeticUncheckedBinaryOperator() = default;
-
-  inline TypedValue applyToTypedValues(const TypedValue &left,
-                                       const TypedValue &right) const override {
-    return applyToTypedValuesInl(left, right);
-  }
-
-  // NOTE(chasseur): This inline version does NOT override a virtual in the
-  // base class. g++ (and probably other compilers) will not inline a method
-  // that overrides a virtual, so we use this instead when we cast to a
-  // specific subclass so that we can actually inline the call.
-  inline TypedValue applyToTypedValuesInl(const TypedValue &left,
-                                          const TypedValue &right) const {
-    if ((left_nullable && left.isNull()) || (right_nullable && right.isNull())) {
-      return TypedValue(ResultType::kStaticTypeID);
-    }
-    return TypedValue(op_functor_(left.getLiteral<LeftCppType>(),
-                                  right.getLiteral<RightCppType>()));
-  }
-
-  inline TypedValue applyToDataPtrs(const void *left, const void *right) const override {
-    return applyToDataPtrsInl(left, right);
-  }
-
-  // See above note about inlines.
-  inline TypedValue applyToDataPtrsInl(const void *left, const void *right) const {
-    if ((left_nullable && (left == nullptr)) || (right_nullable && (right == nullptr))) {
-      return TypedValue(ResultType::kStaticTypeID);
-    }
-
-    return TypedValue(op_functor_(*static_cast<const LeftCppType*>(left),
-                                  *static_cast<const RightCppType*>(right)));
-  }
-
-  ColumnVector* applyToColumnVectors(
-      const ColumnVector &left,
-      const ColumnVector &right) const override {
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-    // All arithmetic types (numbers, datetime, and intervals) are usable with
-    // NativeColumnVector, so 'left' and 'right' should always be native.
-    DCHECK(left.isNative());
-    DCHECK(right.isNative());
-
-    const NativeColumnVector &left_native = static_cast<const NativeColumnVector&>(left);
-    const NativeColumnVector &right_native = static_cast<const NativeColumnVector&>(right);
-
-    DCHECK_EQ(left_native.size(), right_native.size());
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(left_nullable || right_nullable),
-        left_native.size());
-    for (std::size_t pos = 0;
-         pos < left_native.size();
-         ++pos) {
-      const LeftCppType *left_value
-          = static_cast<const LeftCppType*>(
-              left_native.getUntypedValue<left_nullable>(pos));
-      if (left_nullable && (left_value == nullptr)) {
-        result->appendNullValue();
-        continue;
-      }
-      const RightCppType *right_value
-          = static_cast<const RightCppType*>(
-              right_native.getUntypedValue<right_nullable>(pos));
-      if (right_nullable && (right_value == nullptr)) {
-        result->appendNullValue();
-        continue;
-      }
-      *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-          = op_functor_(*left_value, *right_value);
-    }
-    return result;
-  }
-
-  ColumnVector* applyToColumnVectorAndStaticValue(
-      const ColumnVector &left,
-      const TypedValue &right) const override {
-    return applyToColumnVectorAndStaticValueHelper<true>(left, right);
-  }
-
-  ColumnVector* applyToStaticValueAndColumnVector(
-      const TypedValue &left,
-      const ColumnVector &right) const override {
-    return applyToColumnVectorAndStaticValueHelper<false>(right, left);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  ColumnVector* applyToSingleValueAccessor(
-      ValueAccessor *accessor,
-      const attribute_id left_id,
-      const attribute_id right_id) const override {
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        accessor,
-        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-      NativeColumnVector *result = new NativeColumnVector(
-          ResultType::Instance(left_nullable || right_nullable),
-          accessor->getNumTuples());
-      accessor->beginIteration();
-      while (accessor->next()) {
-        const LeftCppType *left_value = static_cast<const LeftCppType*>(
-            accessor->template getUntypedValue<left_nullable>(left_id));
-        if (left_nullable && (left_value == nullptr)) {
-          result->appendNullValue();
-          continue;
-        }
-        const RightCppType *right_value = static_cast<const RightCppType*>(
-            accessor->template getUntypedValue<right_nullable>(right_id));
-        if (right_nullable && (right_value == nullptr)) {
-          result->appendNullValue();
-          continue;
-        }
-        *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-            = op_functor_(*left_value, *right_value);
-      }
-      return result;
-    });
-  }
-
-  ColumnVector* applyToValueAccessorAndStaticValue(
-      ValueAccessor *left_accessor,
-      const attribute_id left_id,
-      const TypedValue &right) const override {
-    return applyToValueAccessorAndStaticValueHelper<true>(left_accessor, left_id, right);
-  }
-
-  ColumnVector* applyToStaticValueAndValueAccessor(
-      const TypedValue &left,
-      ValueAccessor *right_accessor,
-      const attribute_id right_id) const override {
-    return applyToValueAccessorAndStaticValueHelper<false>(right_accessor, right_id, left);
-  }
-
-  ColumnVector* applyToColumnVectorAndValueAccessor(
-      const ColumnVector &left,
-      ValueAccessor *right_accessor,
-      const attribute_id right_id) const override {
-    return applyToColumnVectorAndValueAccessorHelper<true>(left, right_accessor, right_id);
-  }
-
-  ColumnVector* applyToValueAccessorAndColumnVector(
-      ValueAccessor *left_accessor,
-      const attribute_id left_id,
-      const ColumnVector &right) const override {
-    return applyToColumnVectorAndValueAccessorHelper<false>(right, left_accessor, left_id);
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  ColumnVector* applyToValueAccessorAndStaticValueForJoin(
-      ValueAccessor *left_accessor,
-      const bool left_accessor_is_left_relation,
-      const attribute_id left_id,
-      const TypedValue &right,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    return applyToValueAccessorAndStaticValueForJoinHelper<true>(left_accessor,
-                                                                 left_accessor_is_left_relation,
-                                                                 left_id,
-                                                                 right,
-                                                                 joined_tuple_ids);
-  }
-
-  ColumnVector* applyToStaticValueAndValueAccessorForJoin(
-      const TypedValue &left,
-      ValueAccessor *right_accessor,
-      const bool right_accessor_is_left_relation,
-      const attribute_id right_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    return applyToValueAccessorAndStaticValueForJoinHelper<false>(right_accessor,
-                                                                  right_accessor_is_left_relation,
-                                                                  right_id,
-                                                                  left,
-                                                                  joined_tuple_ids);
-  }
-
-  ColumnVector* applyToColumnVectorAndValueAccessorForJoin(
-      const ColumnVector &left,
-      ValueAccessor *right_accessor,
-      const bool right_accessor_is_left_relation,
-      const attribute_id right_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    return applyToColumnVectorAndValueAccessorForJoinHelper<true>(left,
-                                                                  right_accessor,
-                                                                  right_accessor_is_left_relation,
-                                                                  right_id,
-                                                                  joined_tuple_ids);
-  }
-
-  ColumnVector* applyToValueAccessorAndColumnVectorForJoin(
-      ValueAccessor *left_accessor,
-      const bool left_accessor_is_left_relation,
-      const attribute_id left_id,
-      const ColumnVector &right,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    return applyToColumnVectorAndValueAccessorForJoinHelper<false>(right,
-                                                                   left_accessor,
-                                                                   left_accessor_is_left_relation,
-                                                                   left_id,
-                                                                   joined_tuple_ids);
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN_WITH_BINARY_EXPRESSIONS
-  ColumnVector* applyToValueAccessorsForJoin(
-      ValueAccessor *left_accessor,
-      const bool left_accessor_is_left_relation,
-      const attribute_id left_id,
-      ValueAccessor *right_accessor,
-      const bool right_accessor_is_left_relation,
-      const attribute_id right_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(left_nullable || right_nullable),
-        joined_tuple_ids.size());
-    InvokeOnValueAccessorNotAdapter(
-        left_accessor,
-        [&](auto *left_accessor) -> void {  // NOLINT(build/c++11)
-      InvokeOnValueAccessorNotAdapter(
-          right_accessor,
-          [&](auto *right_accessor) -> void {  // NOLINT(build/c++11)
-        for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-          const LeftCppType *left_value = static_cast<const LeftCppType*>(
-              left_accessor->template getUntypedValueAtAbsolutePosition<left_nullable>(
-                  left_id,
-                  left_accessor_is_left_relation ? joined_pair.first
-                                                 : joined_pair.second));
-          if (left_nullable && (left_value == nullptr)) {
-            result->appendNullValue();
-            continue;
-          }
-          const RightCppType *right_value = static_cast<const RightCppType*>(
-              right_accessor->template getUntypedValueAtAbsolutePosition<right_nullable>(
-                  right_id,
-                  right_accessor_is_left_relation ? joined_pair.first
-                                                  : joined_pair.second));
-          if (right_nullable && (right_value == nullptr)) {
-            result->appendNullValue();
-            continue;
-          }
-          *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-              = op_functor_(*left_value, *right_value);
-        }
-      });
-    });
-    return result;
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN_WITH_BINARY_EXPRESSIONS
-
-  TypedValue accumulateColumnVector(
-      const TypedValue &current,
-      const ColumnVector &column_vector,
-      std::size_t *num_tuples_applied) const override {
-    return accumulateColumnVectorHelper(
-        current,
-        column_vector,
-        num_tuples_applied,
-        std::is_same<LeftCppType, typename ResultType::cpptype>());
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  TypedValue accumulateValueAccessor(
-      const TypedValue &current,
-      ValueAccessor *accessor,
-      const attribute_id value_accessor_id,
-      std::size_t *num_tuples_applied) const override {
-    return accumulateValueAccessorHelper(
-        current,
-        accessor,
-        value_accessor_id,
-        num_tuples_applied,
-        std::is_same<LeftCppType, typename ResultType::cpptype>());
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
- private:
-  template <bool column_vector_on_left>
-  ColumnVector* applyToColumnVectorAndStaticValueHelper(
-      const ColumnVector &column_vector,
-      const TypedValue &static_value) const {
-    typedef typename std::conditional<column_vector_on_left,
-                                      RightCppType,
-                                      LeftCppType>::type
-        StaticValueCppType;
-
-    constexpr bool cv_nullable = column_vector_on_left ? left_nullable : right_nullable;
-    constexpr bool static_value_nullable = column_vector_on_left ? right_nullable : left_nullable;
-
-    DCHECK(column_vector.isNative());
-    const NativeColumnVector &native_column_vector = static_cast<const NativeColumnVector&>(column_vector);
-
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(left_nullable || right_nullable),
-        native_column_vector.size());
-    if (static_value_nullable && static_value.isNull()) {
-      result->fillWithNulls();
-      return result;
-    }
-    const StaticValueCppType literal = static_value.getLiteral<StaticValueCppType>();
-    for (std::size_t pos = 0;
-         pos < native_column_vector.size();
-         ++pos) {
-      const void* cv_value = native_column_vector.getUntypedValue<cv_nullable>(pos);
-      if (cv_nullable && (cv_value == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-            = castAndApply<column_vector_on_left>(cv_value, &literal);
-      }
-    }
-    return result;
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <bool value_accessor_on_left>
-  ColumnVector* applyToValueAccessorAndStaticValueHelper(
-      ValueAccessor *value_accessor,
-      const attribute_id value_accessor_attr_id,
-      const TypedValue &static_value) const {
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-
-    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        value_accessor,
-        [&](auto *value_accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-      typedef typename std::conditional<value_accessor_on_left,
-                                        RightCppType,
-                                        LeftCppType>::type
-          StaticValueCppType;
-
-      constexpr bool va_nullable = value_accessor_on_left ? left_nullable : right_nullable;
-      constexpr bool static_value_nullable = value_accessor_on_left ? right_nullable : left_nullable;
-
-      NativeColumnVector *result = new NativeColumnVector(
-          ResultType::Instance(left_nullable || right_nullable),
-          value_accessor->getNumTuples());
-      if (static_value_nullable && static_value.isNull()) {
-        result->fillWithNulls();
-        return result;
-      }
-      const StaticValueCppType literal = static_value.getLiteral<StaticValueCppType>();
-      value_accessor->beginIteration();
-      while (value_accessor->next()) {
-        const void* va_value
-            = value_accessor->template getUntypedValue<va_nullable>(value_accessor_attr_id);
-        if (va_nullable && (va_value == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-              = this->castAndApply<value_accessor_on_left>(va_value, &literal);
-        }
-      }
-      return result;
-    });
-  }
-
-  template <bool column_vector_on_left>
-  ColumnVector* applyToColumnVectorAndValueAccessorHelper(
-      const ColumnVector &column_vector,
-      ValueAccessor *value_accessor,
-      const attribute_id value_accessor_attr_id) const {
-    DCHECK(column_vector.isNative());
-    const NativeColumnVector &native_column_vector = static_cast<const NativeColumnVector&>(column_vector);
-
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        value_accessor,
-        [&](auto *value_accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-      constexpr bool cv_nullable = column_vector_on_left ? left_nullable : right_nullable;
-      constexpr bool va_nullable = column_vector_on_left ? right_nullable : left_nullable;
-
-      DCHECK_EQ(native_column_vector.size(),
-                static_cast<std::size_t>(value_accessor->getNumTuples()));
-      NativeColumnVector *result = new NativeColumnVector(
-          ResultType::Instance(left_nullable || right_nullable),
-          native_column_vector.size());
-      std::size_t cv_pos = 0;
-      value_accessor->beginIteration();
-      while (value_accessor->next()) {
-        const void *cv_value = native_column_vector.getUntypedValue<cv_nullable>(cv_pos);
-        if (cv_nullable && (cv_value == nullptr)) {
-          result->appendNullValue();
-          ++cv_pos;
-          continue;
-        }
-        const void *va_value
-            = value_accessor->template getUntypedValue<va_nullable>(value_accessor_attr_id);
-        if (va_nullable && (va_value == nullptr)) {
-          result->appendNullValue();
-          ++cv_pos;
-          continue;
-        }
-        *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-            = this->castAndApply<column_vector_on_left>(cv_value, va_value);
-        ++cv_pos;
-      }
-      return result;
-    });
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  template <bool value_accessor_on_left>
-  ColumnVector* applyToValueAccessorAndStaticValueForJoinHelper(
-      ValueAccessor *value_accessor,
-      const bool accessor_is_left_relation,
-      const attribute_id value_accessor_attr_id,
-      const TypedValue &static_value,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
-    typedef typename std::conditional<value_accessor_on_left,
-                                      RightCppType,
-                                      LeftCppType>::type
-        StaticValueCppType;
-
-    constexpr bool static_value_nullable = value_accessor_on_left ? right_nullable : left_nullable;
-
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(left_nullable || right_nullable),
-        joined_tuple_ids.size());
-    if (static_value_nullable && static_value.isNull()) {
-      result->fillWithNulls();
-      return result;
-    }
-
-    const StaticValueCppType literal = static_value.getLiteral<StaticValueCppType>();
-
-    InvokeOnValueAccessorNotAdapter(
-        value_accessor,
-        [&](auto *value_accessor) -> void {  // NOLINT(build/c++11)
-      constexpr bool va_nullable = value_accessor_on_left ? left_nullable : right_nullable;
-
-      for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-        const void* va_value
-            = value_accessor->template getUntypedValueAtAbsolutePosition<va_nullable>(
-                value_accessor_attr_id,
-                accessor_is_left_relation ? joined_pair.first : joined_pair.second);
-        if (va_nullable && (va_value == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-              = this->castAndApply<value_accessor_on_left>(va_value, &literal);
-        }
-      }
-    });
-    return result;
-  }
-
-  template <bool column_vector_on_left>
-  ColumnVector* applyToColumnVectorAndValueAccessorForJoinHelper(
-      const ColumnVector &column_vector,
-      ValueAccessor *value_accessor,
-      const bool accessor_is_left_relation,
-      const attribute_id value_accessor_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
-    DCHECK(column_vector.isNative());
-    const NativeColumnVector &native_column_vector = static_cast<const NativeColumnVector&>(column_vector);
-    DCHECK_EQ(native_column_vector.size(), joined_tuple_ids.size());
-
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(left_nullable || right_nullable),
-        native_column_vector.size());
-    InvokeOnValueAccessorNotAdapter(
-        value_accessor,
-        [&](auto *value_accessor) -> void {  // NOLINT(build/c++11)
-      constexpr bool cv_nullable = column_vector_on_left ? left_nullable : right_nullable;
-      constexpr bool va_nullable = column_vector_on_left ? right_nullable : left_nullable;
-
-      for (std::size_t pos = 0; pos < native_column_vector.size(); ++pos) {
-        const void *cv_value = native_column_vector.getUntypedValue<cv_nullable>(pos);
-        if (cv_nullable && (cv_value == nullptr)) {
-          result->appendNullValue();
-          continue;
-        }
-        const void *va_value
-            = value_accessor->template getUntypedValueAtAbsolutePosition<va_nullable>(
-                value_accessor_attr_id,
-                accessor_is_left_relation ? joined_tuple_ids[pos].first
-                                          : joined_tuple_ids[pos].second);
-        if (va_nullable && (va_value == nullptr)) {
-          result->appendNullValue();
-          continue;
-        }
-        *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-            = this->castAndApply<column_vector_on_left>(cv_value, va_value);
-      }
-    });
-    return result;
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-  // Actual implementation of accumulator.
-  TypedValue accumulateColumnVectorHelper(
-      const TypedValue &current,
-      const ColumnVector &column_vector,
-      std::size_t *num_tuples_applied,
-      std::true_type) const {
-    DCHECK(num_tuples_applied);
-
-    *num_tuples_applied = 0;
-    if (left_nullable && current.isNull()) {
-      return ResultType::Instance(left_nullable).makeNullValue();
-    }
-
-    LeftCppType accumulated = current.getLiteral<LeftCppType>();
-
-    DCHECK(column_vector.isNative());
-    const NativeColumnVector &native_column_vector = static_cast<const NativeColumnVector&>(column_vector);
-    for (std::size_t pos = 0;
-         pos < native_column_vector.size();
-         ++pos) {
-      const RightCppType *value = static_cast<const RightCppType *>(
-        native_column_vector.getUntypedValue<right_nullable>(pos));
-      if ((right_nullable && value) || !right_nullable) {
-        accumulated = op_functor_(accumulated, *value);
-        ++(*num_tuples_applied);
-      }
-    }
-
-    // Note ResultType::cpptype and LeftCppType are same here.
-    return TypedValue(accumulated);
-  }
-
-  // Unimplemented version of accumlator for use when the result type and left
-  // operand type are not same.
-  TypedValue accumulateColumnVectorHelper(
-      const TypedValue &current,
-      const ColumnVector &column_vector,
-      std::size_t *num_tuples_applied,
-      std::false_type) const {
-    FATAL_ERROR("Unimplemented method UncheckedBinaryOperator::accumulateColumnVectorHelper() "
-        "because ResultType::cpptype and LeftCppType are not same.");
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  // Actual implementation of accumlator.
-  TypedValue accumulateValueAccessorHelper(
-      const TypedValue &current,
-      ValueAccessor *accessor,
-      const attribute_id value_accessor_id,
-      std::size_t *num_tuples_applied,
-      std::true_type) const {
-    DCHECK(num_tuples_applied);
-
-    *num_tuples_applied = 0;
-    if (left_nullable && current.isNull()) {
-      return ResultType::Instance(left_nullable).makeNullValue();
-    }
-
-    LeftCppType accumulated = current.getLiteral<LeftCppType>();
-    InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        accessor,
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-      accessor->beginIteration();
-      while (accessor->next()) {
-        const RightCppType *value = static_cast<const RightCppType *>(
-          accessor->template getUntypedValue<right_nullable>(value_accessor_id));
-        if ((right_nullable && value) || !right_nullable) {
-          accumulated = op_functor_(accumulated, *value);
-          ++(*num_tuples_applied);
-        }
-      }
-    });
-
-    // Note ResultType::cpptype and LeftCppType are same here.
-    return TypedValue(accumulated);
-  }
-
-  // Unimplemented version of accumlator for use when the result type and left
-  // operand type are not same.
-  TypedValue accumulateValueAccessorHelper(
-      const TypedValue &current,
-      ValueAccessor *accessor,
-      const attribute_id value_accessor_id,
-      std::size_t *num_tuples_applied,
-      std::false_type) const {
-    FATAL_ERROR("Unimplemented method UncheckedBinaryOperator::accumulateValueAccessorHelper() "
-        "because ResultType::cpptype and LeftCppType are not same.");
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-  template <bool arguments_in_order>
-  inline typename ResultType::cpptype castAndApply(const void *left, const void *right) const {
-    return op_functor_(*static_cast<const LeftCppType*>(arguments_in_order ? left : right),
-                       *static_cast<const RightCppType*>(arguments_in_order ? right : left));
-  }
-
-  OpFunctor<LeftCppType, RightCppType> op_functor_;
-};
-
-/**
- * @brief The UncheckedBinaryOperator for addition.
- **/
-template <typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-using AddArithmeticUncheckedBinaryOperator
-    = ArithmeticUncheckedBinaryOperator<AddFunctor,
-                                        ResultType,
-                                        LeftCppType, left_nullable,
-                                        RightCppType, right_nullable>;
-
-/**
- * @brief The UncheckedBinaryOperator for subtraction.
- **/
-template <typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-using SubtractArithmeticUncheckedBinaryOperator
-    = ArithmeticUncheckedBinaryOperator<SubtractFunctor,
-                                        ResultType,
-                                        LeftCppType, left_nullable,
-                                        RightCppType, right_nullable>;
-
-/**
- * @brief The UncheckedBinaryOperator for multiplication.
- **/
-template <typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-using MultiplyArithmeticUncheckedBinaryOperator
-    = ArithmeticUncheckedBinaryOperator<MultiplyFunctor,
-                                        ResultType,
-                                        LeftCppType, left_nullable,
-                                        RightCppType, right_nullable>;
-
-/**
- * @brief The UncheckedBinaryOperator for division.
- **/
-template <typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-using DivideArithmeticUncheckedBinaryOperator
-    = ArithmeticUncheckedBinaryOperator<DivideFunctor,
-                                        ResultType,
-                                        LeftCppType, left_nullable,
-                                        RightCppType, right_nullable>;
-
-/**
- * @brief The UncheckedBinaryOperator for integer modulo.
- **/
-template <typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-using IntegerModuloArithmeticUncheckedBinaryOperator
-    = ArithmeticUncheckedBinaryOperator<IntegerModuloFunctor,
-                                        ResultType,
-                                        LeftCppType, left_nullable,
-                                        RightCppType, right_nullable>;
-
-/**
- * @brief The UncheckedBinaryOperator for real number modulo.
- **/
-template <typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-using FloatModuloArithmeticUncheckedBinaryOperator
-    = ArithmeticUncheckedBinaryOperator<FloatModuloFunctor,
-                                        ResultType,
-                                        LeftCppType, left_nullable,
-                                        RightCppType, right_nullable>;
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/AsciiStringBinaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/AsciiStringBinaryOperations.hpp b/types/operations/binary_operations/AsciiStringBinaryOperations.hpp
new file mode 100644
index 0000000..7181bc6
--- /dev/null
+++ b/types/operations/binary_operations/AsciiStringBinaryOperations.hpp
@@ -0,0 +1,130 @@
+/**
+ * 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_TYPES_OPERATIONS_UNARY_OPERATIONS_ASCII_STRING_BINARY_OPERATIONS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ASCII_STRING_BINARY_OPERATIONS_HPP_
+
+#include <cctype>
+#include <cstring>
+#include <string>
+
+#include "types/CharType.hpp"
+#include "types/IntType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/VarCharType.hpp"
+#include "types/operations/OperationUtil.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
+#include "types/port/strnlen.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename LeftT, typename RightT, typename ResultT>
+struct AsciiStringConcatFunctor : public BinaryFunctor<LeftT, RightT, ResultT> {
+  explicit AsciiStringConcatFunctor(const LeftT &left, const RightT &right)
+      : left_max_(left.getStringLength()),
+        right_max_(right.getStringLength()),
+        result_max_(left_max_ + right_max_) {}
+  inline void concat(const void *left, const std::size_t l_len,
+                     const void *right, const std::size_t r_len,
+                     void *result) const {
+    char *result_str = static_cast<char*>(result);
+    std::memcpy(result_str, left, l_len);
+    std::memcpy(result_str + l_len, right, r_len);
+
+    const std::size_t result_len = l_len + r_len;
+    if (ResultT::kStaticTypeID == kVarChar || result_len < result_max_) {
+      result_str[result_len] = 0;
+    }
+  }
+  inline void apply(const void *left, const void *right, void *result) const {
+    concat(left, strnlen(static_cast<const char*>(left), left_max_),
+           right, strnlen(static_cast<const char*>(right), right_max_),
+           result);
+  }
+  inline TypedValue apply(const void *left, const TypedValue &right) const {
+    const std::size_t l_len =
+        strnlen(static_cast<const char*>(left), left_max_);
+    const std::size_t r_len =
+        std::strlen(static_cast<const char*>(right.getOutOfLineData()));
+    const std::size_t buf_len = l_len + r_len + 1;
+    char *buf = static_cast<char*>(std::malloc(buf_len));
+    concat(left, l_len, right.getOutOfLineData(), r_len, buf);
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, buf_len);
+  }
+  inline TypedValue apply(const TypedValue &left, const void *right) const {
+    const std::size_t l_len =
+        std::strlen(static_cast<const char*>(left.getOutOfLineData()));
+    const std::size_t r_len =
+        strnlen(static_cast<const char*>(right), right_max_);;
+    const std::size_t buf_len = l_len + r_len + 1;
+    char *buf = static_cast<char*>(std::malloc(buf_len));
+    concat(left.getOutOfLineData(), l_len, right, r_len, buf);
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, buf_len);
+  }
+  inline TypedValue apply(const TypedValue &left, const TypedValue &right) const {
+    const std::size_t l_len =
+        std::strlen(static_cast<const char*>(left.getOutOfLineData()));
+    const std::size_t r_len =
+        std::strlen(static_cast<const char*>(right.getOutOfLineData()));
+    const std::size_t buf_len = l_len + r_len + 1;
+    char *buf = static_cast<char*>(std::malloc(buf_len));
+    concat(left.getOutOfLineData(), l_len, right.getOutOfLineData(), r_len, buf);
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, buf_len);
+  }
+  inline static std::string GetName() {
+    return "+";
+  }
+  inline static const Type* GetResultType(const Type &left, const Type &right) {
+    DCHECK(left.getTypeID() == LeftT::kStaticTypeID);
+    DCHECK(right.getTypeID() == RightT::kStaticTypeID);
+    const std::size_t result_len =
+        static_cast<const LeftT&>(left).getStringLength() +
+            static_cast<const RightT&>(right).getStringLength();
+    const bool is_nullable = left.isNullable() || right.isNullable();
+    if (LeftT::kStaticTypeID == kChar && RightT::kStaticTypeID == kChar) {
+      return &TypeFactory::GetType(kChar, result_len, is_nullable);
+    } else {
+      return &TypeFactory::GetType(kVarChar, result_len, is_nullable);
+    }
+  }
+  const std::size_t left_max_;
+  const std::size_t right_max_;
+  const std::size_t result_max_;
+};
+
+
+using AsciiStringBinaryFunctorPack = FunctorPack<
+// concat
+    AsciiStringConcatFunctor<CharType, CharType, CharType>,
+    AsciiStringConcatFunctor<CharType, VarCharType, VarCharType>,
+    AsciiStringConcatFunctor<VarCharType, CharType, VarCharType>,
+    AsciiStringConcatFunctor<VarCharType, VarCharType, VarCharType>
+>;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ASCII_STRING_BINARY_OPERATIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/BinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperation.cpp b/types/operations/binary_operations/BinaryOperation.cpp
index 436086f..4f97710 100644
--- a/types/operations/binary_operations/BinaryOperation.cpp
+++ b/types/operations/binary_operations/BinaryOperation.cpp
@@ -19,35 +19,6 @@
 
 #include "types/operations/binary_operations/BinaryOperation.hpp"
 
-#include "types/operations/Operation.pb.h"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
 namespace quickstep {
 
-serialization::BinaryOperation BinaryOperation::getProto() const {
-  serialization::BinaryOperation proto;
-  switch (operation_id_) {
-    case BinaryOperationID::kAdd:
-      proto.set_operation_id(serialization::BinaryOperation::ADD);
-      break;
-    case BinaryOperationID::kSubtract:
-      proto.set_operation_id(serialization::BinaryOperation::SUBTRACT);
-      break;
-    case BinaryOperationID::kMultiply:
-      proto.set_operation_id(serialization::BinaryOperation::MULTIPLY);
-      break;
-    case BinaryOperationID::kDivide:
-      proto.set_operation_id(serialization::BinaryOperation::DIVIDE);
-      break;
-    case BinaryOperationID::kModulo:
-      proto.set_operation_id(serialization::BinaryOperation::MODULO);
-      break;
-    default:
-      FATAL_ERROR("Unrecognized BinaryOperationID");
-  }
-
-  return proto;
-}
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/BinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperation.hpp b/types/operations/binary_operations/BinaryOperation.hpp
index bc8a083..e818634 100644
--- a/types/operations/binary_operations/BinaryOperation.hpp
+++ b/types/operations/binary_operations/BinaryOperation.hpp
@@ -30,8 +30,7 @@
 #include "storage/StorageBlockInfo.hpp"
 #include "types/TypedValue.hpp"
 #include "types/operations/Operation.hpp"
-#include "types/operations/Operation.pb.h"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -44,6 +43,9 @@ class ValueAccessor;
  *  @{
  */
 
+class BinaryOperation;
+typedef std::shared_ptr<const BinaryOperation> BinaryOperationPtr;
+
 /**
  * @brief A binary operator which can be quickly applied to data items WITHOUT
  *        checking their types.
@@ -67,16 +69,6 @@ class UncheckedBinaryOperator {
                                         const TypedValue &right) const = 0;
 
   /**
-   * @brief Apply to two data items via pointers without type-checking.
-   *
-   * @param left The left argument data item to apply to.
-   * @param right The right argument data item to apply to.
-   * @return The literal result of the operation.
-   **/
-  virtual TypedValue applyToDataPtrs(const void *left,
-                                     const void *right) const = 0;
-
-  /**
    * @brief Apply to two equally-sized vectors of values without type-checking.
    *
    * @param left The left argument ColumnVector to apply to.
@@ -182,47 +174,6 @@ class UncheckedBinaryOperator {
       const ColumnVector &right) const = 0;
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  virtual ColumnVector* applyToValueAccessorAndStaticValueForJoin(
-      ValueAccessor *left_accessor,
-      const bool left_accessor_is_left_relation,
-      const attribute_id left_id,
-      const TypedValue &right,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
-
-  virtual ColumnVector* applyToStaticValueAndValueAccessorForJoin(
-      const TypedValue &left,
-      ValueAccessor *right_accessor,
-      const bool right_accessor_is_left_relation,
-      const attribute_id right_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
-
-  virtual ColumnVector* applyToColumnVectorAndValueAccessorForJoin(
-      const ColumnVector &left,
-      ValueAccessor *right_accessor,
-      const bool right_accessor_is_left_relation,
-      const attribute_id right_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
-
-  virtual ColumnVector* applyToValueAccessorAndColumnVectorForJoin(
-      ValueAccessor *left_accessor,
-      const bool left_accessor_is_left_relation,
-      const attribute_id left_id,
-      const ColumnVector &right,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN_WITH_BINARY_EXPRESSIONS
-  virtual ColumnVector* applyToValueAccessorsForJoin(
-      ValueAccessor *left_accessor,
-      const bool left_accessor_is_left_relation,
-      const attribute_id left_id,
-      ValueAccessor *right_accessor,
-      const bool right_accessor_is_left_relation,
-      const attribute_id right_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN_WITH_BINARY_EXPRESSIONS
-
   /**
    * @brief Apply binary operator on column vector accumulating on the current
    * value. This is useful for summing/multiplying the values of the column
@@ -297,241 +248,44 @@ class UncheckedBinaryOperator {
  **/
 class BinaryOperation : public Operation {
  public:
-  /**
-   * @brief Generate a serialized Protocol Buffer representation of
-   *        this BinaryOperation.
-   *
-   * @return The serialized Protocol Buffer representation of
-   *         this BinaryOperation.
-   **/
-  serialization::BinaryOperation getProto() const;
-
-  /**
-   * @brief Determine the ID of this BinaryOperation
-   *
-   * @return The ID of this BinaryOperation
-   **/
-  inline BinaryOperationID getBinaryOperationID() const {
-    return operation_id_;
-  }
-
-  /**
-   * @brief Get a human-readable name for this BinaryOperation.
-   *
-   * @return A human-readable name for this BinaryOperation.
-   **/
   virtual std::string getName() const {
-    return kBinaryOperationNames[static_cast<std::size_t>(operation_id_)];
+    return "NoName";
   }
 
-  /**
-   * @brief Get a human-readable short name (e.g. "+", "-", "*", "/") for this
-   *        BinaryOperation.
-   *
-   * @return A human-readable short name for this BinaryOperation.
-   **/
   virtual std::string getShortName() const {
-    return kBinaryOperationShortNames[static_cast<std::size_t>(operation_id_)];
+    return getName();
   }
 
-  /**
-   * @brief Whether this binary operation is commutative.
-   *
-   * @note The commutative property provides more optimization opportunities,
-   *       e.g. common subexpression elimination. Meanwhile it is always safe
-   *       to assume that a binary operation is not commutative.
-   *
-   * @return True if this binary operation is commutative; false otherwise.
-   */
+  // TODO
   virtual bool isCommutative() const {
     return false;
   }
 
-  /**
-   * @brief Determine whether this BinaryOperation can apply to the specified
-   *        Types.
-   * @note When the Types that an operator can apply to are changed,
-   *       its definition in the function catalog of the query optimizer
-   *       needs to be modified accordingly.
-   *
-   * @param left The left argument Type to check.
-   * @param right The right argument Type to check.
-   * @return Whether this BinaryOperation can apply to left and right.
-   **/
-  virtual bool canApplyToTypes(const Type &left, const Type &right) const = 0;
-
-  /**
-   * @brief Determine the Type of the result from applying this BinaryOperation
-   *        to arguments of the specified Types.
-   *
-   * @param left The left argument Type to check.
-   * @param right The right argument Type to check.
-   * @return The Type of the result from applying this BinaryOperation to left
-   *         and right (NULL if not applicable).
-   **/
-  virtual const Type* resultTypeForArgumentTypes(const Type &left, const Type &right) const = 0;
-
-  /**
-   * @brief Similar to resultTypeForArgumentTypes(), but allows either or both
-   *        argument types to be NULL to indicate an unknown (but presumed
-   *        nullable) type. If the return type can be unambiguously determined
-   *        based on the incomplete information about the argument types, that
-   *        will be returned. Otherwise, NULL will be returned.
-   * @note This method returns NULL in cases where the return Type is
-   *       ambiguous, and also in cases where this BinaryOperation can not
-   *       possibly be applied to argument(s) of one of the known types left or
-   *       right. See also partialTypeSignatureIsPlausible(), which will return
-   *       true in the former case (ambiguous result type) but false in the
-   *       latter (can not apply to specfied argument types).
-   *
-   * @param left The left argument Type, or NULL if unknown (but presumed
-   *        nullable).
-   * @param right The right argument Type, or NULL if unknown (but presumed
-   *        nullable).
-   * @return The result Type for the given partial argument type information,
-   *         if such a result Type can be unambiguously determined. Otherwise
-   *         NULL.
-   **/
-  virtual const Type* resultTypeForPartialArgumentTypes(const Type *left,
-                                                        const Type *right) const = 0;
-
-  /**
-   * @brief Determine if an incomplete type signature is plausible for this
-   *        BinaryOperation. Specifically, checks if for each unknown Type in
-   *        the partial Type signature, there is some concrete Type that could
-   *        be substituted such that the overall signature becomes valid.
-   *
-   * @param result_type The result Type for this BinaryOperation, or NULL if
-   *        unknown.
-   * @param left_argument_type The Type for the left argument to this
-   *        BinaryOperation, or NULL if unknown (but presumed nullable).
-   * @param right_argument_type The Type for the right argument to this
-   *        BinaryOperation, or NULL if unknown (but presumed nullable).
-   * @return true if the specified partial Type signature is plausible.
-   **/
-  virtual bool partialTypeSignatureIsPlausible(const Type *result_type,
-                                               const Type *left_argument_type,
-                                               const Type *right_argument_type) const = 0;
-
-  // TODO(chasseur): This hinting interface is currently rather rudimentary,
-  // and we would like it to be more flexible to capture the full semantics of
-  // binary operations. Three issues immediately come to mind:
-  //
-  //     1. We should be able to use a category of Types or multiple Types as
-  //        a hint. For example, if we hint that the result of
-  //        DivideBinaryOperation should be DatetimeIntervalType, we know that
-  //        the hint for the left argument should be DatetimeIntervalType, but
-  //        the hint for the right argument can be any numeric type.
-  //     2. We should know something about whether or not a BinaryOperation is
-  //        commutative with respect to its argument types. For example,
-  //        AddBinaryOperation is commutative. If we hint that its result
-  //        should be DatetimeType, then one of its arguments should be
-  //        DatetimeType and the other should be one of the interval types, but
-  //        which is left and which is right does not matter (although it does
-  //        matter that there is one of each).
-  //     3. We may want to use "sideways" hinting, where we tentatively resolve
-  //        the types for arguments and note where we encounter "strong"
-  //        information about what a type should be (i.e. from the catalog or
-  //        an explicitly-typed literal in the parser), then using that to help
-  //        hint "weak" guesses about typing for the other argument.
-  //
-  // These issues can be difficult to solve in general, and it may not be worth
-  // doing in the current optimizer if another more sophisticated optimizer
-  // frontend is to be added in the future.
+  virtual bool canApplyTo(const Type &left,
+                          const Type &right,
+                          const std::vector<TypedValue> &static_arguments,
+                          std::string *message) const = 0;
 
-  /**
-   * @brief Get a pair of "hint" Types for the arguments to this
-   *        BinaryOperation based on a hint for this BinaryOperation's result
-   *        type. If possible, returns pointers to Types that, when given to
-   *        this BinaryOperation as arguments, yield values of the desired type
-   *        (i.e. calling resultTypeForArgumentTypes() on the returned types
-   *        should return the original type_hint).
-   *
-   * This method is designed to help the query optimizer's Resolver determine
-   * the type of literal values that appear in SQL queries. The Resolver
-   * propagates known Type information (e.g. from the Catalog) top-down through
-   * parsed expression trees, eventually using a Type hint at a leaf
-   * ParseLiteralValue node in the tree to concretize that literal value as the
-   * appropriate Type. This is especially important for NULL literals that
-   * appear in SQL queries, because a NULL by itself does not convey any type
-   * information and could be an instance of any nullable type. Hints generated
-   * by this method are to be treated as hints only, not as hard or
-   * authoritative requirements. After resolving literals, the Resolver
-   * propagates known concrete Types bottom-up through the tree and explicitly
-   * checks canApplyToTypes() for correctness.
-   *
-   * @note In some cases, either or both arguments could take on more than one
-   *       possible Type and this BinaryOperation would still yield the
-   *       specified result_type_hint (e.g. for arithmetic operations, if
-   *       result_type_hint is DoubleType, then the arguments could both be
-   *       DoubleType, or DoubleType and a different numeric Type, or even
-   *       LongType and FloatType). In such cases, the most precise suitable
-   *       Type is used as the hint.
-   * @note In some cases, the choice of argument Type hints for a given
-   *       result_type_hint is ambiguous and there is no single precise Type
-   *       that can represent values of any of the possible argument types
-   *       (e.g. an addition returning Datetime could be
-   *       Datetime + DatetimeInterval, DatetimeInterval + Datetime,
-   *       Datetime + YearMonthInterval, or YearMonthInterval + Datetime, and
-   *       none of these Types is coercible to the others or to a common more
-   *       precise type). In such cases, NULL is used to represent an ambiguous
-   *       or unknown hint.
-   *
-   * @param result_type_hint A hint about what Type the result of this
-   *        BinaryOperation should have. May be NULL to indicate no preference.
-   * @return A pair of type hints for the left and right arguments to this
-   *         BinaryOperation based on type_hint, or NULL if no suitable Type
-   *         exists or if the hint is ambiguous.
-   **/
-  virtual std::pair<const Type*, const Type*> pushDownTypeHint(
-      const Type *result_type_hint) const = 0;
+  bool canApplyTo(const Type &left,
+                  const Type &right,
+                  const std::vector<TypedValue> &static_arguments = {}) const {
+    std::string message;
+    return canApplyTo(left, right, static_arguments, &message);
+  }
 
-  /**
-   * @brief Apply this BinaryOperation to two TypedValues.
-   * @warning It is an error to call this method if this BinaryOperation can
-   *          not be applied to arguments of the specified types. If in doubt,
-   *          check canApplyToTypes() first.
-   *
-   * @param left The left argument TypedValue to apply to.
-   * @param left_type The Type that left belongs to.
-   * @param right The right argument TypedValue to apply to.
-   * @param right_type The Type that right belongs to.
-   * @return The literal result of the operation.
-   **/
-  virtual TypedValue applyToChecked(const TypedValue &left,
-                                    const Type &left_type,
-                                    const TypedValue &right,
-                                    const Type &right_type) const = 0;
+  virtual const Type* getResultType(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments = {}) const = 0;
 
-  /**
-   * @brief Create an UncheckedBinaryOperator which can apply to items of the
-   *        specified types.
-   * @warning The resulting UncheckedBinaryOperator performs no type-checking
-   *          whatsoever. Nonetheless, it is useful in situations where many
-   *          data items of the same, known types are to be operated on (for
-   *          example, over many tuples in the same table).
-   *
-   * @param left The left argument Type to compare.
-   * @param right The right argument Type to compare.
-   * @return An UncheckedBinaryOperator which applies this BinaryOperation to
-   *         the specified Types.
-   * @exception OperationInapplicableToType This BinaryOperation is not
-   *            applicable to either left or right.
-   **/
-  virtual UncheckedBinaryOperator* makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                                       const Type &right) const = 0;
+  virtual UncheckedBinaryOperator* makeUncheckedBinaryOperator(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments = {}) const = 0;
 
  protected:
-  explicit BinaryOperation(const BinaryOperationID operation_id)
-      : Operation(Operation::kBinaryOperation,
-                  kBinaryOperationNames[
-                      static_cast<typename std::underlying_type<BinaryOperationID>::type>(operation_id)],
-                  kBinaryOperationShortNames[
-                      static_cast<typename std::underlying_type<BinaryOperationID>::type>(operation_id)]),
-        operation_id_(operation_id) {
-  }
-
-  const BinaryOperationID operation_id_;
+  BinaryOperation()
+      : Operation(Operation::kBinaryOperation) {}
 
  private:
   DISALLOW_COPY_AND_ASSIGN(BinaryOperation);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/BinaryOperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationFactory.cpp b/types/operations/binary_operations/BinaryOperationFactory.cpp
deleted file mode 100644
index 51b5a7e..0000000
--- a/types/operations/binary_operations/BinaryOperationFactory.cpp
+++ /dev/null
@@ -1,93 +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 "types/operations/binary_operations/BinaryOperationFactory.hpp"
-
-#include <string>
-
-#include "types/operations/Operation.pb.h"
-#include "types/operations/binary_operations/AddBinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
-#include "types/operations/binary_operations/DivideBinaryOperation.hpp"
-#include "types/operations/binary_operations/ModuloBinaryOperation.hpp"
-#include "types/operations/binary_operations/MultiplyBinaryOperation.hpp"
-#include "types/operations/binary_operations/SubtractBinaryOperation.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-const BinaryOperation& BinaryOperationFactory::GetBinaryOperation(const BinaryOperationID id) {
-  switch (id) {
-    case BinaryOperationID::kAdd:
-      return AddBinaryOperation::Instance();
-    case BinaryOperationID::kSubtract:
-      return SubtractBinaryOperation::Instance();
-    case BinaryOperationID::kMultiply:
-      return MultiplyBinaryOperation::Instance();
-    case BinaryOperationID::kDivide:
-      return DivideBinaryOperation::Instance();
-    case BinaryOperationID::kModulo:
-      return ModuloBinaryOperation::Instance();
-    default:
-      break;  // Prevent compiler from complaining about unhandled case.
-  }
-  // Should never be reached
-  FATAL_ERROR("Unrecognized BinaryOperationID");
-}
-
-bool BinaryOperationFactory::ProtoIsValid(const serialization::BinaryOperation &proto) {
-  // Check that BinaryOperation is fully initialized.
-  if (!proto.IsInitialized()) {
-    return false;
-  }
-
-  // Check that the operation_id is a valid BinaryOperation.
-  if (!proto.BinaryOperationID_IsValid(proto.operation_id())) {
-    return false;
-  }
-
-  return true;
-}
-
-const BinaryOperation& BinaryOperationFactory::ReconstructFromProto(
-    const serialization::BinaryOperation &proto) {
-  DCHECK(ProtoIsValid(proto))
-      << "Attempted to create BinaryOperation from an invalid proto description:\n"
-      << proto.DebugString();
-
-  switch (proto.operation_id()) {
-    case serialization::BinaryOperation::ADD:
-      return GetBinaryOperation(BinaryOperationID::kAdd);
-    case serialization::BinaryOperation::SUBTRACT:
-      return GetBinaryOperation(BinaryOperationID::kSubtract);
-    case serialization::BinaryOperation::MULTIPLY:
-      return GetBinaryOperation(BinaryOperationID::kMultiply);
-    case serialization::BinaryOperation::DIVIDE:
-      return GetBinaryOperation(BinaryOperationID::kDivide);
-    case serialization::BinaryOperation::MODULO:
-      return GetBinaryOperation(BinaryOperationID::kModulo);
-    default:
-      FATAL_ERROR("Unrecognized BinaryOperationID in "
-                  "BinaryOperationFactory::ReconstructFromProto");
-  }
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/BinaryOperationFactory.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationFactory.hpp b/types/operations/binary_operations/BinaryOperationFactory.hpp
deleted file mode 100644
index 578dfb8..0000000
--- a/types/operations/binary_operations/BinaryOperationFactory.hpp
+++ /dev/null
@@ -1,79 +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_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_FACTORY_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_FACTORY_HPP_
-
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-class BinaryOperation;
-namespace serialization { class BinaryOperation; }
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief All-static factory object that provides access to BinaryOperations.
- **/
-class BinaryOperationFactory {
- public:
-  /**
-   * @brief Convenience factory method to get a pointer to a BinaryOperation
-   *        from that BinaryOperation's ID.
-   *
-   * @param id The ID of the desired BinaryOperation.
-   * @return The BinaryOperation corresponding to id.
-   **/
-  static const BinaryOperation& GetBinaryOperation(const BinaryOperationID id);
-
-  /**
-   * @brief Get a reference to a BinaryOperation from that BinaryOperation's
-   *        serialized Protocol Buffer representation.
-   *
-   * @param proto A serialized Protocol Buffer representation of a
-   *        BinaryOperation, originally generated by getProto().
-   * @return The BinaryOperation described by proto.
-   **/
-  static const BinaryOperation& ReconstructFromProto(const serialization::BinaryOperation &proto);
-
-  /**
-   * @brief Check whether a serialization::BinaryOperation is fully-formed and
-   *        all parts are valid.
-   *
-   * @param proto A serialized Protocol Buffer representation of a
-   *        BinaryOperation, originally generated by getProto().
-   * @return Whether proto is fully-formed and valid.
-   **/
-  static bool ProtoIsValid(const serialization::BinaryOperation &proto);
-
- private:
-  BinaryOperationFactory();
-
-  DISALLOW_COPY_AND_ASSIGN(BinaryOperationFactory);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/BinaryOperationID.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationID.cpp b/types/operations/binary_operations/BinaryOperationID.cpp
deleted file mode 100644
index 7ba2e69..0000000
--- a/types/operations/binary_operations/BinaryOperationID.cpp
+++ /dev/null
@@ -1,40 +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 "types/operations/binary_operations/BinaryOperationID.hpp"
-
-namespace quickstep {
-
-const char *kBinaryOperationNames[] = {
-  "Add",
-  "Subtract",
-  "Multiply",
-  "Divide",
-  "Modulo"
-};
-
-const char *kBinaryOperationShortNames[] = {
-  "+",
-  "-",
-  "*",
-  "/",
-  "%"
-};
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/BinaryOperationID.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationID.hpp b/types/operations/binary_operations/BinaryOperationID.hpp
deleted file mode 100644
index 2cf20d1..0000000
--- a/types/operations/binary_operations/BinaryOperationID.hpp
+++ /dev/null
@@ -1,62 +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_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_ID_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_ID_HPP_
-
-#include <type_traits>
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief Concrete BinaryOperations.
- **/
-enum class BinaryOperationID {
-  kAdd = 0,
-  kSubtract,
-  kMultiply,
-  kDivide,
-  kModulo,
-  kNumBinaryOperationIDs  // Not a real BinaryOperationID, exists for counting purposes.
-};
-
-/**
- * @brief Names of comparisons in the same order as BinaryOperationID.
- **/
-extern const char *kBinaryOperationNames[
-    static_cast<typename std::underlying_type<BinaryOperationID>::type>(
-        BinaryOperationID::kNumBinaryOperationIDs)];
-
-/**
- * @brief Short names (i.e. mathematical symbols) of comparisons in the same
- *        order as BinaryOperationID.
- **/
-extern const char *kBinaryOperationShortNames[
-    static_cast<typename std::underlying_type<BinaryOperationID>::type>(
-        BinaryOperationID::kNumBinaryOperationIDs)];
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_ID_HPP_


[12/32] incubator-quickstep git commit: Remove double can. Thanks @zuyu for catching this.

Posted by ji...@apache.org.
Remove double can. Thanks @zuyu for catching this.


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

Branch: refs/heads/new-op
Commit: 4eb1dee1d1c98e8ab8f1100bf18ae9f6647541d3
Parents: 5ee3f11
Author: Jignesh Patel <jm...@hotmail.com>
Authored: Sat Apr 29 09:39:43 2017 -0500
Committer: Jignesh Patel <jm...@hotmail.com>
Committed: Sat Apr 29 09:39:43 2017 -0500

----------------------------------------------------------------------
 README.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4eb1dee1/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index cfb263c..430777f 100644
--- a/README.md
+++ b/README.md
@@ -78,7 +78,7 @@ CREATE TABLE City (cid Integer, name VARCHAR(80), state CHAR(2));
     INSERT INTO Weather VALUES (2, '2015-11-1', 60, 50);
     ```
 
-11. We can now can issue SQL queries such as:
+11. We can now issue SQL queries such as:
   a. Find all weather records for California:
   ```
   SELECT * FROM WEATHER W, City C WHERE C.cid = W.cid AND C.state = 'CA';


[11/32] incubator-quickstep git commit: Assigned partition_id to RebuildWorkOrder.

Posted by ji...@apache.org.
Assigned partition_id to RebuildWorkOrder.


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

Branch: refs/heads/new-op
Commit: 5ee3f11a5f36d56b27df0e7df3a270852911d7a0
Parents: 30021ac
Author: Zuyu Zhang <zu...@apache.org>
Authored: Sat Apr 22 21:09:19 2017 -0700
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Mon Apr 24 17:57:41 2017 -0700

----------------------------------------------------------------------
 query_execution/QueryManagerSingleNode.cpp |  4 +-
 query_execution/Shiftboss.cpp              |  4 +-
 relational_operators/RebuildWorkOrder.hpp  |  5 +++
 storage/InsertDestination.cpp              |  7 ++-
 storage/InsertDestination.hpp              | 58 +++++++++++++++----------
 5 files changed, 51 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5ee3f11a/query_execution/QueryManagerSingleNode.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManagerSingleNode.cpp b/query_execution/QueryManagerSingleNode.cpp
index e3f349f..79c4026 100644
--- a/query_execution/QueryManagerSingleNode.cpp
+++ b/query_execution/QueryManagerSingleNode.cpp
@@ -174,12 +174,13 @@ void QueryManagerSingleNode::getRebuildWorkOrders(const dag_node_index index,
   }
 
   std::vector<MutableBlockReference> partially_filled_block_refs;
+  std::vector<partition_id> part_ids;
 
   DCHECK(query_context_ != nullptr);
   InsertDestination *insert_destination = query_context_->getInsertDestination(insert_destination_index);
   DCHECK(insert_destination != nullptr);
 
-  insert_destination->getPartiallyFilledBlocks(&partially_filled_block_refs);
+  insert_destination->getPartiallyFilledBlocks(&partially_filled_block_refs, &part_ids);
 
   for (std::vector<MutableBlockReference>::size_type i = 0;
        i < partially_filled_block_refs.size();
@@ -189,6 +190,7 @@ void QueryManagerSingleNode::getRebuildWorkOrders(const dag_node_index index,
                              std::move(partially_filled_block_refs[i]),
                              index,
                              op.getOutputRelationID(),
+                             part_ids[i],
                              foreman_client_id_,
                              bus_),
         index);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5ee3f11a/query_execution/Shiftboss.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Shiftboss.cpp b/query_execution/Shiftboss.cpp
index 72d8913..21e7858 100644
--- a/query_execution/Shiftboss.cpp
+++ b/query_execution/Shiftboss.cpp
@@ -405,7 +405,8 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
   DCHECK(insert_destination != nullptr);
 
   vector<MutableBlockReference> partially_filled_block_refs;
-  insert_destination->getPartiallyFilledBlocks(&partially_filled_block_refs);
+  vector<partition_id> part_ids;
+  insert_destination->getPartiallyFilledBlocks(&partially_filled_block_refs, &part_ids);
 
   serialization::InitiateRebuildResponseMessage proto;
   proto.set_query_id(query_id);
@@ -439,6 +440,7 @@ void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
                              move(partially_filled_block_refs[i]),
                              op_index,
                              rel_id,
+                             part_ids[i],
                              shiftboss_client_id_local_,
                              bus_local_);
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5ee3f11a/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index 7f0f7fc..8615d74 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -57,6 +57,7 @@ class RebuildWorkOrder : public WorkOrder {
    *        query plan DAG that produced the output block.
    * @param input_relation_id The ID of the CatalogRelation to which the given
    *        storage block belongs to.
+   * @param part_id The partition_id of the block, if any.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
    * @param bus A pointer to the TMB.
    **/
@@ -65,12 +66,14 @@ class RebuildWorkOrder : public WorkOrder {
       MutableBlockReference &&block_ref,  // NOLINT(whitespace/operators)
       const std::size_t input_operator_index,
       const relation_id input_relation_id,
+      const partition_id part_id,
       const client_id scheduler_client_id,
       MessageBus *bus)
       : WorkOrder(query_id),
         block_ref_(std::move(block_ref)),
         input_operator_index_(input_operator_index),
         input_relation_id_(input_relation_id),
+        part_id_(part_id),
         scheduler_client_id_(scheduler_client_id),
         bus_(bus) {}
 
@@ -88,6 +91,7 @@ class RebuildWorkOrder : public WorkOrder {
     proto.set_block_id(block_ref_->getID());
     proto.set_relation_id(input_relation_id_);
     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();
@@ -114,6 +118,7 @@ class RebuildWorkOrder : public WorkOrder {
   MutableBlockReference block_ref_;
   const std::size_t input_operator_index_;
   const relation_id input_relation_id_;
+  const partition_id part_id_;
   const client_id scheduler_client_id_;
 
   MessageBus *bus_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5ee3f11a/storage/InsertDestination.cpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.cpp b/storage/InsertDestination.cpp
index b6a9e3a..891b5a1 100644
--- a/storage/InsertDestination.cpp
+++ b/storage/InsertDestination.cpp
@@ -29,6 +29,7 @@
 #include "catalog/Catalog.pb.h"
 #include "catalog/CatalogAttribute.hpp"
 #include "catalog/CatalogRelationSchema.hpp"
+#include "catalog/CatalogTypedefs.hpp"
 #include "catalog/PartitionSchemeHeader.hpp"
 #include "query_execution/QueryExecutionMessages.pb.h"
 #include "query_execution/QueryExecutionTypedefs.hpp"
@@ -421,11 +422,15 @@ MutableBlockReference BlockPoolInsertDestination::createNewBlock() {
   return storage_manager_->getBlockMutable(new_id, relation_);
 }
 
-void BlockPoolInsertDestination::getPartiallyFilledBlocks(std::vector<MutableBlockReference> *partial_blocks) {
+void BlockPoolInsertDestination::getPartiallyFilledBlocks(std::vector<MutableBlockReference> *partial_blocks,
+                                                          vector<partition_id> *part_ids) {
   SpinMutexLock lock(mutex_);
   for (std::vector<MutableBlockReference>::size_type i = 0; i < available_block_refs_.size(); ++i) {
     partial_blocks->push_back((std::move(available_block_refs_[i])));
+    // No partition.
+    part_ids->push_back(0u);
   }
+
   available_block_refs_.clear();
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5ee3f11a/storage/InsertDestination.hpp
----------------------------------------------------------------------
diff --git a/storage/InsertDestination.hpp b/storage/InsertDestination.hpp
index dc5a093..66c67d7 100644
--- a/storage/InsertDestination.hpp
+++ b/storage/InsertDestination.hpp
@@ -183,8 +183,11 @@ class InsertDestination : public InsertDestinationInterface {
    *
    * @param partial_blocks A pointer to the vector of block IDs in which the
    *                       partially filled block IDs will be added.
+   * @param part_ids A pointer to the vector of partiiton_ids in which the
+   *                 partially filled block IDs are associated with.
    **/
-  virtual void getPartiallyFilledBlocks(std::vector<MutableBlockReference> *partial_blocks) = 0;
+  virtual void getPartiallyFilledBlocks(std::vector<MutableBlockReference> *partial_blocks,
+                                        std::vector<partition_id> *part_ids) = 0;
 
  protected:
   /**
@@ -335,7 +338,8 @@ class AlwaysCreateBlockInsertDestination : public InsertDestination {
     return returned_block_ids_;
   }
 
-  void getPartiallyFilledBlocks(std::vector<MutableBlockReference> *partial_blocks) override {
+  void getPartiallyFilledBlocks(std::vector<MutableBlockReference> *partial_blocks,
+                                std::vector<partition_id> *part_ids) override {
   }
 
  private:
@@ -421,7 +425,8 @@ class BlockPoolInsertDestination : public InsertDestination {
 
   void returnBlock(MutableBlockReference &&block, const bool full) override;
 
-  void getPartiallyFilledBlocks(std::vector<MutableBlockReference> *partial_blocks) override;
+  void getPartiallyFilledBlocks(std::vector<MutableBlockReference> *partial_blocks,
+                                std::vector<partition_id> *part_ids) override;
 
   const std::vector<block_id>& getTouchedBlocksInternal() override;
 
@@ -488,33 +493,15 @@ class PartitionAwareInsertDestination : public InsertDestination {
     available_block_ids_[part_id].push_back(bid);
   }
 
-  void getPartiallyFilledBlocks(std::vector<MutableBlockReference> *partial_blocks) override {
+  void getPartiallyFilledBlocks(std::vector<MutableBlockReference> *partial_blocks,
+                                std::vector<partition_id> *part_ids) override {
     // Iterate through each partition and return the partially filled blocks
     // in each partition.
     for (partition_id part_id = 0; part_id < partition_scheme_header_->getNumPartitions(); ++part_id) {
-      getPartiallyFilledBlocksInPartition(partial_blocks, part_id);
+      getPartiallyFilledBlocksInPartition(partial_blocks, part_ids, part_id);
     }
   }
 
-  /**
-   * @brief Get the set of blocks that were partially filled by clients of this
-   *        InsertDestination for insertion.
-   * @warning Should only be called AFTER this InsertDestination will no longer
-   *          be used, and all blocks have been returned to it via
-   *          returnBlock() and BEFORE getTouchedBlocks() is called, at all.
-   *
-   * @param partial_blocks A pointer to the vector of block IDs in which the
-   *                       partially filled block IDs will be added.
-   * @param part_id The partition id for which we want the partially filled blocks.
-   **/
-  void getPartiallyFilledBlocksInPartition(std::vector<MutableBlockReference> *partial_blocks, partition_id part_id) {
-    SpinMutexLock lock(mutexes_for_partition_[part_id]);
-    for (std::vector<MutableBlockReference>::size_type i = 0; i < available_block_refs_[part_id].size(); ++i) {
-      partial_blocks->push_back((std::move(available_block_refs_[part_id][i])));
-    }
-    available_block_refs_[part_id].clear();
-  }
-
   PartitionSchemeHeader::PartitionAttributeIds getPartitioningAttributes() const override;
 
   void insertTuple(const Tuple &tuple) override;
@@ -572,6 +559,29 @@ class PartitionAwareInsertDestination : public InsertDestination {
   const std::vector<block_id>& getTouchedBlocksInternalInPartition(partition_id part_id);
 
  private:
+  /**
+   * @brief Get the set of blocks that were partially filled by clients of this
+   *        InsertDestination for insertion.
+   * @warning Should only be called AFTER this InsertDestination will no longer
+   *          be used, and all blocks have been returned to it via
+   *          returnBlock() and BEFORE getTouchedBlocks() is called, at all.
+   *
+   * @param partial_blocks A pointer to the vector of block IDs in which the
+   *                       partially filled block IDs will be added.
+   * @param part_id The partition id for which we want the partially filled blocks.
+   **/
+  void getPartiallyFilledBlocksInPartition(std::vector<MutableBlockReference> *partial_blocks,
+                                           std::vector<partition_id> *part_ids,
+                                           const partition_id part_id) {
+    SpinMutexLock lock(mutexes_for_partition_[part_id]);
+    for (std::vector<MutableBlockReference>::size_type i = 0; i < available_block_refs_[part_id].size(); ++i) {
+      partial_blocks->push_back((std::move(available_block_refs_[part_id][i])));
+      part_ids->push_back(part_id);
+    }
+
+    available_block_refs_[part_id].clear();
+  }
+
   std::unique_ptr<const PartitionSchemeHeader> partition_scheme_header_;
 
   // A vector of available block references for each partition.


[16/32] incubator-quickstep git commit: Fixed a bug regarding the table name containing '_' in Commands.

Posted by ji...@apache.org.
Fixed a bug regarding the table name containing '_' in Commands.


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

Branch: refs/heads/new-op
Commit: 678e2c3956b8ad922388b450d721b7e25535d81e
Parents: 99198ef
Author: Zuyu Zhang <zu...@apache.org>
Authored: Tue May 2 13:51:41 2017 -0700
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue May 2 13:58:08 2017 -0700

----------------------------------------------------------------------
 parser/SqlLexer.lpp                   |    2 +-
 parser/preprocessed/SqlLexer_gen.cpp  | 1058 +++++++++++++------------
 parser/preprocessed/SqlLexer_gen.hpp  |  190 +++--
 parser/preprocessed/SqlParser_gen.cpp | 1162 ++++++++++++++--------------
 parser/preprocessed/SqlParser_gen.hpp |    4 +-
 5 files changed, 1263 insertions(+), 1153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/678e2c39/parser/SqlLexer.lpp
----------------------------------------------------------------------
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index b5c2b3e..c6402e0 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -149,7 +149,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
 }
 
 <CONDITION_COMMAND>{
-  [a-zA-Z0-9\-\.:]+ {
+  [a-zA-Z0-9_\-\.:]+ {
     /* This is a command argument. */
     yylval->string_value_ = new quickstep::ParseString(
         yylloc->first_line, yylloc->first_column, std::string(yytext, yyleng));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/678e2c39/parser/preprocessed/SqlLexer_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.cpp b/parser/preprocessed/SqlLexer_gen.cpp
index 3a85df6..bd77ee1 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 163
 #define YY_END_OF_BUFFER 164
 /* 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[599] =
+static const flex_int16_t yy_accept[599] =
     {   0,
         0,    0,    0,    0,    0,    0,    0,    0,    0,    0,
         0,    0,  164,    2,    2,  162,  162,  161,  160,  162,
@@ -460,7 +509,7 @@ static yyconst flex_int16_t yy_accept[599] =
 
     } ;
 
-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,79 +553,79 @@ static yyconst YY_CHAR yy_meta[72] =
         8
     } ;
 
-static yyconst flex_uint16_t yy_base[614] =
+static const flex_int16_t yy_base[614] =
     {   0,
-        0,    1,   46,    0,  117,  163,    2,    3,  128,  132,
-        6,   10,  211, 1319, 1319,    0, 1319,   13, 1319,  194,
-     1319, 1319, 1319,  194,    6,  130,    4, 1319,  170,  124,
-      161,  215,  170,  207,  265,   92,  167,  162,   96,  107,
-      223,  165,  162,  221,  274,   92,  284,  277,  315,  206,
-      128,  232,    0,  125, 1319,  152,    4,   19,    0,    0,
-        0,  143,    0,    0,  379,    0,    0,  144,    0,   22,
-     1319,    0,  292,  305,  335,   18, 1319, 1319, 1319,    0,
-      174,  264,  180,  186,  209,  271,  229,    0,  277,  333,
-      335,  312,  330,  317,  323,  338,  323,  333,  340, 1319,
-
-      339,  355,  357,  381,  370,  373,  377,  382,  380,  384,
-      383,  383,  383,  431,    0,  398,  383,  390,  406,  402,
-      403,  404,  425,  420,  431,  442,    0,  445,  432,  448,
-      436,  437,  451,  448,  444,  460,  452,  439,  489,  464,
-      468,  469,  468,  462,  455,  477,  501,  493,  489,  494,
-      502,  494,  510,  142,   29,    0,    0, 1319,    0, 1319,
-     1319,   22,   24, 1319, 1319, 1319, 1319, 1319,    0,    0,
-     1319,    0,  520,   26,   28,    0,    0,  508,    0,  512,
-      495,  510,  497,  530,  518,  506,  525,  508,  512,  509,
-      543,  530,  533,  550,    0,  547,  558,  556,  561,  546,
-
-      565,  552,  564,    0,  569,  552,  554,  556,  557,  576,
-      566,  576,  569,  571,  565,  579,  580,  573,  587,  597,
-      598,  604,  594,    0,  593,  594,  612,  613,  616,  603,
-      605,    0,  614,    0,  622,  623,  611,  610,  630,  631,
-      626,  615,  629,  628,  639,  640,  639,  631,  634,  656,
-        0,  654,  648,  659,  658,  670,  672,  668,  666,  667,
-      682,  671,  665,  684,  674,  685,  683,  675,   30,  125,
-        0,  676,  683,  693,  685,  697,  692,  692,    0,  706,
-      697,  706,  704,    0,  705,  711,  725,  713,  722,  717,
-      719,  727,  736,  733,  731,    0,    0,  728,  725,  745,
-
-      742,  729,  730,  736,  743,    0,    0,  739,    0,  742,
-        0,  733,  742,  743,  759,    0,    0,    0,    0,    0,
-      743,    0,  745,  774,  764,  769,  770,    0,  782,  788,
-      791,  796,  780,    0,  794,    0,  782,  777,  782,    0,
-      799,  790,  802,  795,  788,  804,    0,  791,    0,  808,
-      793,  801,  797,  799,  814,  818,  816,    0,  830,  825,
-        0,  828,  136, 1319,    0,  841,  841,  829,  850,  838,
-      849,  853,    0,  844,  841,  855,  856,  848,  854,  864,
-      854,  862,  859,  857,  858,  868,  871,  858,  876,    0,
-        0,    0,  858,  875,    0,  886,    0,    0,  878,  894,
-
-      885,    0,  898,    0,    0,    0,    0,  889,  898,  909,
-      896,  906,    0,  911,  901,    0,  913,  915,  900,  912,
-      905,    0,  904,  906,  912,    0,  924,  925,    0,  912,
-      933,    0,  912,  922,  930,  926,    0,  929,  938,  956,
-      952,  942,    0,  964,    0,  962,  950,  958,  960,  953,
-        0,  954,  971,  973,    0,   93,    0,  957,  969,  965,
-        0,    0,  963,  977,  982,    0,  975,  967,  979,  964,
-      972,  979,    0,  983,    0,    0,  983,  989, 1008, 1013,
-     1011,    0,    0,    0, 1001,    0, 1006, 1009,    0, 1016,
-     1013, 1016, 1018, 1026, 1023,    0,    0, 1028,    0,    0,
-
-     1029, 1026, 1016, 1018,    0, 1025,    0,    0, 1037, 1029,
-     1027,    0, 1031,    0, 1021, 1044, 1041, 1031, 1039, 1042,
-     1043, 1062,    0, 1052, 1066, 1063, 1062, 1065, 1063, 1068,
-     1073,    0, 1070, 1075,    0, 1070, 1084, 1072,    0, 1079,
-     1087,    0, 1090, 1084,    0, 1091, 1083, 1084, 1099, 1096,
-        0, 1098, 1103, 1097, 1106,    0, 1093,    0, 1107, 1107,
-        0,    0, 1111,    0, 1112, 1126,    0,    0, 1124,    0,
-        0,    0, 1121, 1136, 1126,    0,    0, 1136,    0,    0,
-     1126, 1142, 1128,    0,    0, 1135, 1145,    0, 1142, 1145,
-     1135, 1150, 1137,    0, 1138,    0,    0, 1319, 1203, 1213,
-
-     1223, 1233, 1243, 1247, 1250, 1256, 1264, 1274, 1284, 1294,
-     1304, 1309, 1311
+        0,    1,   46,    0,  117,  162,    2,    3,  127,  128,
+        6,   10,  159, 1324, 1324,    0, 1324,   13, 1324,  138,
+     1324, 1324, 1324,  142,    6,  129,    4, 1324,  134,  124,
+      159,  213,  165,  167,  263,   92,  158,  163,   96,  107,
+      218,  160,  186,  223,  221,  155,  281,  264,  279,  275,
+      186,  209,    0,  191, 1324,  147,    4,   19,    0,    0,
+        0,  140,    0,    0,  343,    0,    0,  141,    0,   22,
+     1324,    0,  307,  326,  338,   18, 1324, 1324, 1324,    0,
+      220,  315,  234,  239,  234,  288,  261,    0,  280,  335,
+      330,  342,  346,  332,  333,  382,  332,  331,  339, 1324,
+
+      336,  358,  363,  380,  356,  352,  378,  386,  387,  391,
+      390,  387,  386,  406,    0,  399,  386,  393,  407,  408,
+      401,  402,  410,  408,  431,  440,    0,  443,  431,  447,
+      438,  440,  454,  451,  447,  461,  457,  442,  466,  465,
+      468,  469,  468,  461,  456,  463,  502,  479,  475,  498,
+      496,  500,  504,  144,   29,    0,    0, 1324,    0, 1324,
+     1324,   22,   24, 1324, 1324, 1324, 1324, 1324,    0,    0,
+     1324,    0,  522,   26,   28,    0,    0,  504,    0,  514,
+      498,  513,  500,  521,  520,  508,  525,  508,  512,  507,
+      534,  517,  523,  537,    0,  535,  545,  554,  544,  542,
+
+      565,  552,  564,    0,  571,  555,  559,  559,  560,  579,
+      569,  577,  570,  572,  564,  578,  578,  571,  585,  588,
+      589,  590,  585,    0,  580,  582,  599,  608,  613,  600,
+      605,    0,  614,    0,  622,  625,  614,  615,  633,  634,
+      625,  619,  634,  630,  638,  639,  637,  628,  633,  646,
+        0,  640,  634,  646,  646,  657,  658,  663,  663,  668,
+      678,  670,  665,  685,  676,  689,  687,  678,   30,  125,
+        0,  679,  684,  694,  686,  696,  691,  690,    0,  704,
+      695,  696,  690,    0,  691,  698,  713,  700,  708,  712,
+      716,  724,  736,  733,  731,    0,    0,  728,  726,  747,
+
+      746,  732,  733,  739,  746,    0,    0,  740,    0,  743,
+        0,  734,  741,  742,  756,    0,    0,    0,    0,    0,
+      741,    0,  744,  759,  749,  751,  758,    0,  769,  774,
+      786,  793,  777,    0,  794,    0,  782,  777,  782,    0,
+      800,  792,  806,  798,  791,  807,    0,  794,    0,  809,
+      794,  802,  796,  798,  812,  816,  814,    0,  820,  811,
+        0,  814,  131, 1324,    0,  828,  829,  816,  836,  833,
+      847,  854,    0,  844,  841,  855,  857,  850,  858,  867,
+      857,  865,  862,  858,  859,  869,  870,  857,  874,    0,
+        0,    0,  856,  873,    0,  876,    0,    0,  864,  880,
+
+      873,    0,  885,    0,    0,    0,    0,  874,  893,  906,
+      893,  906,    0,  911,  901,    0,  913,  916,  902,  916,
+      908,    0,  907,  909,  915,    0,  925,  926,    0,  913,
+      932,    0,  911,  920,  928,  924,    0,  919,  924,  942,
+      939,  930,    0,  951,    0,  948,  945,  955,  957,  953,
+        0,  954,  971,  973,    0,   93,    0,  958,  971,  969,
+        0,    0,  966,  980,  985,    0,  978,  968,  980,  965,
+      971,  978,    0,  981,    0,    0,  981,  987,  998,  999,
+      997,    0,    0,    0,  988,    0,  994,  996,    0, 1002,
+     1008, 1013, 1015, 1026, 1023,    0,    0, 1028,    0,    0,
+
+     1029, 1027, 1018, 1022,    0, 1028,    0,    0, 1040, 1032,
+     1030,    0, 1032,    0, 1022, 1045, 1040, 1030, 1037, 1040,
+     1041, 1052,    0, 1038, 1052, 1050, 1050, 1052, 1049, 1063,
+     1070,    0, 1067, 1075,    0, 1070, 1084, 1072,    0, 1080,
+     1089,    0, 1094, 1087,    0, 1094, 1086, 1087, 1100, 1097,
+        0, 1099, 1102, 1096, 1104,    0, 1091,    0, 1105, 1097,
+        0,    0, 1097,    0, 1098, 1113,    0,    0, 1112,    0,
+        0,    0, 1108, 1122, 1121,    0,    0, 1133,    0,    0,
+     1123, 1142, 1128,    0,    0, 1135, 1145,    0, 1143, 1147,
+     1139, 1153, 1140,    0, 1141,    0,    0, 1324, 1206, 1216,
+
+     1226, 1236, 1246, 1250, 1253, 1259, 1269, 1279, 1289, 1299,
+     1309, 1314, 1316
     } ;
 
-static yyconst flex_int16_t yy_def[614] =
+static const flex_int16_t yy_def[614] =
     {   0,
       599,  599,  598,    3,  600,  600,  601,  601,  602,  602,
       603,  603,  598,  598,  598,  604,  598,  598,  598,  598,
@@ -648,7 +697,7 @@ static yyconst flex_int16_t yy_def[614] =
       598,  598,  598
     } ;
 
-static yyconst flex_uint16_t yy_nxt[1391] =
+static const flex_int16_t yy_nxt[1396] =
     {   0,
       598,  598,   15,   15,   61,   61,  155,  155,   67,   62,
        62,   68,   67,  598,   70,   68,   70,   73,   73,   77,
@@ -663,148 +712,150 @@ static yyconst flex_uint16_t yy_nxt[1391] =
 
        38,   39,   40,   41,   42,   43,   44,   45,   46,   47,
        48,   49,   50,   51,   52,   53,   54,   17,   56,   57,
-       58,   17,   17,   17,   17,   17,  111,  116,  117,  134,
-       64,   17,   17,   17,   64,   62,  270,  270,  504,   62,
-       74,   75,   75,  154,   81,  150,  153,  270,  270,  171,
-      158,   76,   82,  154,   83,  111,  116,  117,  134,   84,
-       17,   17,   17,   17,   56,   57,   58,   17,   17,   17,
-       17,   17,   65,   81,  150,  153,   65,   17,   17,   17,
-       76,   82,   85,   83,  112,   79,   86,   95,   84,   87,
-      122,   96,  114,  124,  176,   97,  123,  115,  113,  125,
-
-      179,   98,   88,   72,   99,  180,   17,   17,   17,   71,
-      598,   85,  598,  112,  100,   86,   95,  598,   87,  122,
-       96,  114,  124,  176,   97,  123,  115,  113,  125,  179,
-       98,   88,   89,   99,  180,  101,  147,  102,  148,   90,
-      118,  149,  103,  126,  119,  181,   91,  104,  120,   92,
-       93,  127,   94,  598,  121,  128,  151,  152,  129,  130,
-      184,   89,  598,  598,  101,  147,  102,  148,   90,  118,
-      149,  103,  126,  119,  181,   91,  104,  120,   92,   93,
-      127,   94,  105,  121,  128,  151,  152,  129,  130,  184,
-      106,  131,  177,  107,  182,  132,  108,  598,  139,  109,
-
-      178,  135,  110,   73,   73,  136,  140,  183,  133,  137,
-      598,  105,  185,   76,  141,  138,  173,  173,  598,  106,
-      131,  177,  107,  182,  132,  108,   76,  139,  109,  178,
-      135,  110,  142,  191,  136,  140,  183,  133,  137,  143,
-      144,  185,   76,  141,  138,   74,   75,   75,  192,  145,
-      186,  193,  146,  598,  187,   76,   76,  196,  200,  194,
-      197,  142,  191,  188,  195,  189,  198,  190,  143,  144,
-      201,  202,  598,  199,  203,  204,  205,  192,  145,  186,
-      193,  146,  161,  187,  598,   76,  196,  200,  194,  197,
-      162,  163,  188,  195,  189,  198,  190,  164,  208,  201,
-
-      202,  165,  199,  203,  204,  205,  206,  209,  210,  166,
-      211,  213,  214,  167,  215,  168,  212,  207,  216,  169,
-      218,  217,  598,  223,  224,  225,  164,  208,  226,  227,
-      165,  228,  598,  229,  230,  206,  209,  210,  166,  211,
-      213,  214,  167,  215,  168,  212,  207,  216,  169,  218,
-      217,  219,  223,  224,  225,  231,  232,  226,  227,  233,
-      228,  220,  229,  230,  234,  235,  221,  222,  236,  237,
-      238,  239,  240,  242,  243,  247,  241,  244,  248,  598,
-      219,  252,  245,  246,  231,  232,  253,  254,  233,  255,
-      220,  256,  257,  234,  235,  221,  222,  236,  237,  238,
-
-      239,  240,  242,  243,  247,  241,  244,  248,  249,  258,
-      252,  245,  246,  261,  262,  253,  254,  250,  255,  259,
-      256,  257,  263,  265,  266,  251,  260,  268,  264,  272,
-      267,  173,  173,  273,  274,  275,  276,  249,  258,  279,
-      280,   76,  261,  262,  281,  282,  250,  283,  259,  277,
-      284,  263,  265,  266,  251,  260,  268,  264,  272,  267,
-      285,  278,  273,  274,  275,  276,  286,  287,  279,  280,
-       76,  288,  289,  281,  282,  290,  283,  291,  277,  284,
-      293,  292,  294,  295,  296,  297,  298,  299,  300,  285,
-      278,  301,  302,  303,  304,  286,  287,  305,  306,  307,
-
-      288,  289,  308,  309,  290,  310,  291,  311,  312,  293,
-      292,  294,  295,  296,  297,  298,  299,  300,  313,  314,
-      301,  302,  303,  304,  317,  315,  305,  306,  307,  318,
-      319,  308,  309,  320,  310,  316,  311,  312,  321,  322,
-      323,  324,  325,  326,  327,  328,  329,  313,  314,  330,
-      331,  334,  335,  317,  315,  332,  336,  333,  318,  319,
-      337,  338,  320,  339,  316,  342,  340,  321,  322,  323,
-      324,  325,  326,  327,  328,  329,  341,  343,  330,  331,
-      334,  335,  344,  345,  332,  336,  333,  346,  347,  337,
-      338,  348,  339,  349,  342,  340,  350,  351,  352,  354,
-
-      353,  355,  356,  357,  358,  360,  343,  361,  359,  362,
-      365,  344,  345,  366,  367,  368,  346,  347,  369,  370,
-      348,  371,  349,  372,  373,  350,  351,  352,  354,  353,
-      355,  356,  357,  358,  360,  374,  361,  359,  362,  365,
-      375,  376,  366,  367,  368,  377,  378,  369,  370,  379,
-      371,  380,  372,  373,  381,  382,  383,  384,  385,  386,
-      387,  388,  389,  390,  374,  391,  392,  393,  394,  375,
-      376,  395,  396,  397,  377,  378,  398,  399,  379,  402,
-      380,  403,  400,  381,  382,  383,  384,  385,  386,  387,
-      388,  389,  390,  401,  391,  392,  393,  394,  404,  405,
-
-      395,  396,  397,  406,  407,  398,  399,  408,  402,  409,
-      403,  400,  410,  411,  412,  413,  414,  415,  416,  417,
-      418,  419,  401,  420,  421,  422,  423,  404,  405,  424,
-      425,  426,  406,  407,  427,  428,  408,  429,  409,  430,
-      431,  410,  411,  412,  413,  414,  415,  416,  417,  418,
-      419,  432,  420,  421,  422,  423,  433,  434,  424,  425,
-      426,  435,  436,  427,  428,  437,  429,  438,  430,  431,
-      439,  441,  442,  440,  443,  444,  445,  446,  447,  448,
-      432,  449,  450,  451,  452,  433,  434,  453,  454,  455,
-      435,  436,  456,  457,  437,  458,  438,  459,  460,  439,
-
-      441,  442,  440,  443,  444,  445,  446,  447,  448,  461,
-      449,  450,  451,  452,  462,  463,  453,  454,  455,  466,
-      464,  456,  457,  465,  458,  467,  459,  460,  468,  469,
-      470,  471,  472,  473,  474,  475,  476,  477,  461,  478,
-      479,  480,  481,  462,  463,  482,  483,  484,  466,  464,
-      485,  486,  465,  487,  467,  488,  489,  468,  469,  470,
-      471,  472,  473,  474,  475,  476,  477,  490,  478,  479,
-      480,  481,  491,  492,  482,  483,  484,  493,  494,  485,
-      486,  495,  487,  496,  488,  489,  497,  498,  499,  500,
-      501,  502,  503,  505,  506,  507,  490,  508,  509,  510,
-
-      511,  491,  492,  512,  513,  514,  493,  494,  515,  516,
-      495,  517,  496,  518,  519,  497,  498,  499,  500,  501,
-      502,  503,  505,  506,  507,  520,  508,  509,  510,  511,
-      521,  522,  512,  513,  514,  523,  524,  515,  516,  525,
-      517,  526,  518,  519,  527,  528,  529,  530,  531,  532,
-      533,  534,  535,  536,  520,  537,  538,  539,  540,  521,
-      522,  541,  542,  543,  523,  524,  544,  545,  525,  546,
-      526,  547,  548,  527,  528,  529,  530,  531,  532,  533,
-      534,  535,  536,  549,  537,  538,  539,  540,  550,  551,
-      541,  542,  543,  552,  553,  544,  545,  554,  546,  555,
-
-      547,  548,  556,  557,  558,  559,  560,  561,  562,  563,
-      564,  565,  549,  566,  567,  568,  569,  550,  551,  570,
-      571,  572,  552,  553,  573,  574,  554,  575,  555,  576,
-      577,  556,  557,  558,  559,  560,  561,  562,  563,  564,
-      565,  578,  566,  567,  568,  569,  579,  580,  570,  571,
-      572,  581,  582,  573,  574,  583,  575,  584,  576,  577,
-      585,  586,  587,  588,  589,  590,  591,  592,  593,  594,
-      578,  595,  596,  597,  598,  579,  580,  598,  598,  598,
-      581,  582,  598,  598,  583,  598,  584,  598,  598,  585,
-      586,  587,  588,  589,  590,  591,  592,  593,  594,  598,
-
-      595,  596,  597,   14,   14,   14,   14,   14,   14,   14,
-       14,   14,   14,   59,   59,   59,   59,   59,   59,   59,
-       59,   59,   59,   60,   60,   60,   60,   60,   60,   60,
-       60,   60,   60,   63,   63,   63,   63,   63,   63,   63,
-       63,   63,   63,   66,   66,   66,   66,   66,   66,   66,
-       66,   66,   66,   69,   69,   80,   80,   80,  598,   80,
-      156,  156,  156,  156,  157,  157,  157,  598,  157,  157,
-      157,  157,  157,  157,  159,  159,  159,  598,  159,  159,
-      159,  159,  598,  159,  160,  160,  160,  160,  160,  160,
-      160,  160,  160,  160,  170,  170,  598,  170,  170,  170,
-
-      170,  170,  170,  170,  172,  598,  172,  172,  172,  172,
-      172,  172,  172,  172,  271,  271,  364,  364,   13,  598,
+       58,   17,   17,   17,   17,   17,  111,  116,  117,   64,
+       64,   17,   17,   17,   62,   62,  270,  270,  504,   74,
+       75,   75,  270,  270,   81,  154,  171,  158,  154,   79,
+       76,   72,   82,   71,   83,  111,  116,  117,  598,   84,
+       17,   17,   17,   56,   57,   58,   17,   17,   17,   17,
+       17,   65,   65,   81,  100,  112,   17,   17,   17,   76,
+       85,   82,   95,   83,   86,  122,   96,   87,   84,  113,
+       97,  123,  134,  114,  598,  101,   98,  102,  115,   99,
+
+       88,  598,  103,  150,  112,   17,   17,  104,  598,   85,
+      598,   95,  153,   86,  122,   96,   87,  124,  113,   97,
+      123,  134,  114,  125,  101,   98,  102,  115,   99,   88,
+       89,  103,  150,  151,  152,  118,  104,   90,  131,  119,
+      176,  153,  132,  120,   91,  126,  124,   92,   93,  121,
+       94,  598,  125,  127,  179,  133,  598,  128,  180,   89,
+      129,  130,  151,  152,  118,  598,   90,  131,  119,  176,
+      181,  132,  120,   91,  126,  598,   92,   93,  121,   94,
+      105,  598,  127,  179,  133,  139,  128,  180,  106,  129,
+      130,  107,  184,  140,  108,  598,  142,  109,  135,  181,
+
+      110,  141,  136,  143,  144,  147,  137,  148,  598,  105,
+      149,  182,  138,  145,  139,  185,  146,  106,   73,   73,
+      107,  184,  140,  108,  183,  142,  109,  135,   76,  110,
+      141,  136,  143,  144,  147,  137,  148,  173,  173,  149,
+      182,  138,  145,  177,  185,  146,  161,   76,   74,   75,
+       75,  178,  186,  183,  162,  163,  187,   76,  188,   76,
+      189,  164,  190,  191,  192,  165,  193,  200,  201,  194,
+      202,  203,  177,  166,  195,  598,   76,  167,  204,  168,
+      178,  186,  205,  169,  208,  187,  209,  188,   76,  189,
+      164,  190,  191,  192,  165,  193,  200,  201,  194,  202,
+
+      203,  196,  166,  195,  197,  206,  167,  204,  168,  210,
+      198,  205,  169,  208,  211,  209,  207,  199,  213,  214,
+      212,  215,  216,  218,  223,  217,  219,  224,  225,  226,
+      196,  229,  230,  197,  206,  227,  220,  228,  210,  198,
+      231,  221,  222,  211,  232,  207,  199,  213,  214,  212,
+      215,  216,  218,  223,  217,  219,  224,  225,  226,  233,
+      229,  230,  234,  235,  227,  220,  228,  236,  237,  231,
+      221,  222,  238,  232,  239,  240,  242,  243,  244,  241,
+      247,  248,  252,  245,  246,  249,  253,  254,  233,  255,
+      256,  234,  235,  257,  250,  258,  236,  237,  598,  261,
+
+      262,  238,  251,  239,  240,  242,  243,  244,  241,  247,
+      248,  252,  245,  246,  249,  253,  254,  265,  255,  256,
+      259,  268,  257,  250,  258,  272,  263,  260,  261,  262,
+      266,  251,  264,  173,  173,  273,  267,  274,  275,  276,
+      277,  279,  280,   76,  281,  282,  265,  283,  284,  259,
+      268,  285,  278,  286,  272,  263,  260,  287,  288,  266,
+      289,  264,  290,  293,  273,  267,  274,  275,  276,  277,
+      279,  280,   76,  281,  282,  291,  283,  284,  294,  292,
+      285,  278,  286,  295,  296,  297,  287,  288,  298,  289,
+      299,  290,  293,  300,  301,  302,  303,  304,  305,  306,
+
+      307,  308,  309,  310,  291,  311,  312,  294,  292,  313,
+      314,  315,  295,  296,  297,  317,  318,  298,  319,  299,
+      320,  316,  300,  301,  302,  303,  304,  305,  306,  307,
+      308,  309,  310,  321,  311,  312,  322,  323,  313,  314,
+      315,  324,  325,  326,  317,  318,  327,  319,  328,  320,
+      316,  329,  330,  331,  332,  334,  333,  335,  336,  337,
+      338,  339,  321,  340,  342,  322,  323,  343,  344,  345,
+      324,  325,  326,  341,  346,  327,  347,  328,  348,  349,
+      329,  330,  331,  332,  334,  333,  335,  336,  337,  338,
+      339,  350,  340,  342,  351,  354,  343,  344,  345,  352,
+
+      355,  353,  356,  346,  357,  347,  358,  348,  349,  360,
+      359,  361,  362,  365,  366,  367,  368,  369,  370,  371,
+      350,  372,  373,  351,  354,  374,  375,  376,  352,  355,
+      353,  356,  377,  357,  378,  358,  379,  380,  360,  359,
+      361,  362,  365,  366,  367,  368,  369,  370,  371,  381,
+      372,  373,  382,  383,  374,  375,  376,  384,  385,  386,
+      387,  377,  388,  378,  389,  379,  380,  390,  391,  392,
+      393,  394,  395,  396,  397,  398,  399,  402,  381,  400,
+      403,  382,  383,  404,  405,  406,  384,  385,  386,  387,
+      401,  388,  407,  389,  408,  409,  390,  391,  392,  393,
+
+      394,  395,  396,  397,  398,  399,  402,  410,  400,  403,
+      411,  412,  404,  405,  406,  413,  414,  415,  416,  401,
+      417,  407,  418,  408,  409,  419,  420,  421,  422,  423,
+      424,  425,  426,  427,  428,  429,  410,  430,  431,  411,
+      412,  432,  433,  434,  413,  414,  415,  416,  435,  417,
+      436,  418,  437,  438,  419,  420,  421,  422,  423,  424,
+      425,  426,  427,  428,  429,  439,  430,  431,  440,  441,
+      432,  433,  434,  442,  443,  444,  445,  435,  446,  436,
+      447,  437,  438,  448,  449,  450,  451,  452,  453,  454,
+      455,  456,  457,  458,  439,  459,  460,  440,  441,  461,
+
+      462,  463,  442,  443,  444,  445,  466,  446,  464,  447,
+      467,  465,  448,  449,  450,  451,  452,  453,  454,  455,
+      456,  457,  458,  468,  459,  460,  469,  470,  461,  462,
+      463,  471,  472,  473,  474,  466,  475,  464,  476,  467,
+      465,  477,  478,  479,  480,  481,  482,  483,  484,  485,
+      486,  487,  468,  488,  489,  469,  470,  490,  491,  492,
+      471,  472,  473,  474,  493,  475,  494,  476,  495,  496,
+      477,  478,  479,  480,  481,  482,  483,  484,  485,  486,
+      487,  497,  488,  489,  498,  499,  490,  491,  492,  500,
+      501,  502,  503,  493,  505,  494,  506,  495,  496,  507,
+
+      508,  509,  510,  511,  512,  513,  514,  515,  516,  517,
+      497,  518,  519,  498,  499,  520,  521,  522,  500,  501,
+      502,  503,  523,  505,  524,  506,  525,  526,  507,  508,
+      509,  510,  511,  512,  513,  514,  515,  516,  517,  527,
+      518,  519,  528,  529,  520,  521,  522,  530,  531,  532,
+      533,  523,  534,  524,  535,  525,  526,  536,  537,  538,
+      539,  540,  541,  542,  543,  544,  545,  546,  527,  547,
+      548,  528,  529,  549,  550,  551,  530,  531,  532,  533,
+      552,  534,  553,  535,  554,  555,  536,  537,  538,  539,
+      540,  541,  542,  543,  544,  545,  546,  556,  547,  548,
+
+      557,  558,  549,  550,  551,  559,  560,  561,  562,  552,
+      563,  553,  564,  554,  555,  565,  566,  567,  568,  569,
+      570,  571,  572,  573,  574,  575,  556,  576,  577,  557,
+      558,  578,  579,  580,  559,  560,  561,  562,  581,  563,
+      582,  564,  583,  584,  565,  566,  567,  568,  569,  570,
+      571,  572,  573,  574,  575,  585,  576,  577,  586,  587,
+      578,  579,  580,  588,  589,  590,  591,  581,  592,  582,
+      593,  583,  584,  594,  595,  596,  597,  598,  598,  598,
+      598,  598,  598,  598,  585,  598,  598,  586,  587,  598,
+      598,  598,  588,  589,  590,  591,  598,  592,  598,  593,
+
+      598,  598,  594,  595,  596,  597,   14,   14,   14,   14,
+       14,   14,   14,   14,   14,   14,   59,   59,   59,   59,
+       59,   59,   59,   59,   59,   59,   60,   60,   60,   60,
+       60,   60,   60,   60,   60,   60,   63,   63,   63,   63,
+       63,   63,   63,   63,   63,   63,   66,   66,   66,   66,
+       66,   66,   66,   66,   66,   66,   69,   69,   80,   80,
+       80,  598,   80,  156,  156,  156,  156,  598,  156,  157,
+      157,  157,  598,  157,  157,  157,  157,  157,  157,  159,
+      159,  159,  598,  159,  159,  159,  159,  598,  159,  160,
+      160,  160,  160,  160,  160,  160,  160,  160,  160,  170,
+
+      170,  598,  170,  170,  170,  170,  170,  170,  170,  172,
+      598,  172,  172,  172,  172,  172,  172,  172,  172,  271,
+      271,  364,  364,   13,  598,  598,  598,  598,  598,  598,
       598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
       598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
       598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
       598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
       598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
       598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598
+      598,  598,  598,  598,  598
+
     } ;
 
-static yyconst flex_int16_t yy_chk[1391] =
+static const flex_int16_t yy_chk[1396] =
     {   0,
         0,    0,    1,    2,    7,    8,   57,   57,   11,    7,
         8,   11,   12,    0,   18,   12,   18,   25,   25,   27,
@@ -819,149 +870,151 @@ static yyconst flex_int16_t yy_chk[1391] =
 
         3,    3,    3,    3,    3,    3,    3,    3,    3,    3,
         3,    3,    3,    3,    3,    3,    3,    5,    5,    5,
-        5,    5,    5,    5,    5,    5,   36,   39,   40,   46,
-        9,    5,    5,    5,   10,    9,  270,  270,  456,   10,
-       26,   26,   26,  154,   30,   51,   54,  363,  363,   68,
-       62,   26,   30,   56,   30,   36,   39,   40,   46,   30,
-        5,    5,    5,    6,    6,    6,    6,    6,    6,    6,
-        6,    6,    9,   30,   51,   54,   10,    6,    6,    6,
-       26,   30,   31,   30,   37,   29,   31,   33,   30,   31,
-       42,   33,   38,   43,   81,   33,   42,   38,   37,   43,
-
-       83,   33,   31,   24,   33,   84,    6,    6,    6,   20,
-       13,   31,    0,   37,   34,   31,   33,    0,   31,   42,
-       33,   38,   43,   81,   33,   42,   38,   37,   43,   83,
-       33,   31,   32,   33,   84,   34,   50,   34,   50,   32,
-       41,   50,   34,   44,   41,   85,   32,   34,   41,   32,
-       32,   44,   32,    0,   41,   44,   52,   52,   44,   44,
-       87,   32,    0,    0,   34,   50,   34,   50,   32,   41,
-       50,   34,   44,   41,   85,   32,   34,   41,   32,   32,
-       44,   32,   35,   41,   44,   52,   52,   44,   44,   87,
-       35,   45,   82,   35,   86,   45,   35,    0,   48,   35,
-
-       82,   47,   35,   73,   73,   47,   48,   86,   45,   47,
-        0,   35,   89,   73,   48,   47,   74,   74,    0,   35,
-       45,   82,   35,   86,   45,   35,   74,   48,   35,   82,
-       47,   35,   49,   92,   47,   48,   86,   45,   47,   49,
-       49,   89,   73,   48,   47,   75,   75,   75,   93,   49,
-       90,   94,   49,    0,   90,   74,   75,   96,   97,   95,
-       96,   49,   92,   91,   95,   91,   96,   91,   49,   49,
-       98,   99,    0,   96,  101,  102,  103,   93,   49,   90,
-       94,   49,   65,   90,    0,   75,   96,   97,   95,   96,
-       65,   65,   91,   95,   91,   96,   91,   65,  105,   98,
-
-       99,   65,   96,  101,  102,  103,  104,  106,  107,   65,
-      108,  109,  110,   65,  111,   65,  108,  104,  112,   65,
-      113,  112,    0,  116,  117,  118,   65,  105,  119,  120,
-       65,  120,    0,  121,  122,  104,  106,  107,   65,  108,
-      109,  110,   65,  111,   65,  108,  104,  112,   65,  113,
-      112,  114,  116,  117,  118,  123,  124,  119,  120,  125,
-      120,  114,  121,  122,  126,  128,  114,  114,  129,  130,
-      131,  132,  133,  134,  135,  137,  133,  136,  138,    0,
-      114,  140,  136,  136,  123,  124,  141,  142,  125,  143,
-      114,  144,  145,  126,  128,  114,  114,  129,  130,  131,
-
-      132,  133,  134,  135,  137,  133,  136,  138,  139,  146,
-      140,  136,  136,  148,  149,  141,  142,  139,  143,  147,
-      144,  145,  150,  151,  152,  139,  147,  153,  150,  178,
-      152,  173,  173,  180,  181,  182,  183,  139,  146,  185,
-      186,  173,  148,  149,  187,  188,  139,  189,  147,  184,
-      190,  150,  151,  152,  139,  147,  153,  150,  178,  152,
-      191,  184,  180,  181,  182,  183,  192,  193,  185,  186,
-      173,  194,  196,  187,  188,  197,  189,  198,  184,  190,
-      199,  198,  200,  201,  202,  203,  205,  206,  207,  191,
-      184,  208,  209,  210,  211,  192,  193,  212,  213,  214,
-
-      194,  196,  215,  216,  197,  217,  198,  218,  219,  199,
-      198,  200,  201,  202,  203,  205,  206,  207,  220,  221,
-      208,  209,  210,  211,  223,  222,  212,  213,  214,  225,
-      226,  215,  216,  227,  217,  222,  218,  219,  228,  229,
-      230,  231,  233,  235,  236,  237,  238,  220,  221,  239,
-      240,  242,  243,  223,  222,  241,  244,  241,  225,  226,
-      245,  246,  227,  247,  222,  249,  248,  228,  229,  230,
-      231,  233,  235,  236,  237,  238,  248,  250,  239,  240,
-      242,  243,  252,  253,  241,  244,  241,  254,  255,  245,
-      246,  256,  247,  257,  249,  248,  258,  259,  260,  261,
-
-      260,  262,  263,  264,  265,  266,  250,  267,  265,  268,
-      272,  252,  253,  273,  274,  275,  254,  255,  276,  277,
-      256,  278,  257,  280,  281,  258,  259,  260,  261,  260,
-      262,  263,  264,  265,  266,  282,  267,  265,  268,  272,
-      283,  285,  273,  274,  275,  286,  287,  276,  277,  288,
-      278,  289,  280,  281,  290,  291,  292,  293,  294,  295,
-      298,  299,  300,  301,  282,  302,  303,  304,  305,  283,
-      285,  308,  310,  312,  286,  287,  313,  314,  288,  321,
-      289,  323,  315,  290,  291,  292,  293,  294,  295,  298,
-      299,  300,  301,  315,  302,  303,  304,  305,  324,  325,
-
-      308,  310,  312,  326,  327,  313,  314,  329,  321,  330,
-      323,  315,  331,  332,  333,  335,  337,  338,  339,  341,
-      342,  343,  315,  344,  345,  346,  348,  324,  325,  350,
-      351,  352,  326,  327,  353,  354,  329,  355,  330,  356,
-      357,  331,  332,  333,  335,  337,  338,  339,  341,  342,
-      343,  359,  344,  345,  346,  348,  360,  362,  350,  351,
-      352,  366,  367,  353,  354,  368,  355,  369,  356,  357,
-      370,  371,  372,  370,  374,  375,  376,  377,  378,  379,
-      359,  380,  381,  382,  383,  360,  362,  384,  385,  386,
-      366,  367,  387,  388,  368,  389,  369,  393,  394,  370,
-
-      371,  372,  370,  374,  375,  376,  377,  378,  379,  396,
-      380,  381,  382,  383,  399,  400,  384,  385,  386,  403,
-      401,  387,  388,  401,  389,  408,  393,  394,  409,  410,
-      411,  412,  414,  415,  417,  418,  419,  420,  396,  421,
-      423,  424,  425,  399,  400,  427,  428,  430,  403,  401,
-      431,  433,  401,  434,  408,  435,  436,  409,  410,  411,
-      412,  414,  415,  417,  418,  419,  420,  438,  421,  423,
-      424,  425,  439,  440,  427,  428,  430,  441,  442,  431,
-      433,  444,  434,  446,  435,  436,  447,  448,  449,  450,
-      452,  453,  454,  458,  459,  460,  438,  463,  464,  465,
-
-      467,  439,  440,  468,  469,  470,  441,  442,  471,  472,
-      444,  474,  446,  477,  478,  447,  448,  449,  450,  452,
-      453,  454,  458,  459,  460,  479,  463,  464,  465,  467,
-      480,  481,  468,  469,  470,  485,  487,  471,  472,  488,
-      474,  490,  477,  478,  491,  492,  493,  494,  495,  498,
-      501,  502,  503,  504,  479,  506,  509,  510,  511,  480,
-      481,  513,  515,  516,  485,  487,  517,  518,  488,  519,
-      490,  520,  521,  491,  492,  493,  494,  495,  498,  501,
-      502,  503,  504,  522,  506,  509,  510,  511,  524,  525,
-      513,  515,  516,  526,  527,  517,  518,  528,  519,  529,
-
-      520,  521,  530,  531,  533,  534,  536,  537,  538,  540,
-      541,  543,  522,  544,  546,  547,  548,  524,  525,  549,
-      550,  552,  526,  527,  553,  554,  528,  555,  529,  557,
-      559,  530,  531,  533,  534,  536,  537,  538,  540,  541,
-      543,  560,  544,  546,  547,  548,  563,  565,  549,  550,
-      552,  566,  569,  553,  554,  573,  555,  574,  557,  559,
-      575,  578,  581,  582,  583,  586,  587,  589,  590,  591,
-      560,  592,  593,  595,    0,  563,  565,    0,    0,    0,
-      566,  569,    0,    0,  573,    0,  574,    0,    0,  575,
-      578,  581,  582,  583,  586,  587,  589,  590,  591,    0,
-
-      592,  593,  595,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  600,  600,  600,  600,  600,  600,  600,
-      600,  600,  600,  601,  601,  601,  601,  601,  601,  601,
-      601,  601,  601,  602,  602,  602,  602,  602,  602,  602,
-      602,  602,  602,  603,  603,  603,  603,  603,  603,  603,
-      603,  603,  603,  604,  604,  605,  605,  605,    0,  605,
-      606,  606,  606,  606,  607,  607,  607,    0,  607,  607,
-      607,  607,  607,  607,  608,  608,  608,    0,  608,  608,
-      608,  608,    0,  608,  609,  609,  609,  609,  609,  609,
-      609,  609,  609,  609,  610,  610,    0,  610,  610,  610,
-
-      610,  610,  610,  610,  611,    0,  611,  611,  611,  611,
-      611,  611,  611,  611,  612,  612,  613,  613,  598,  598,
+        5,    5,    5,    5,    5,    5,   36,   39,   40,    9,
+       10,    5,    5,    5,    9,   10,  270,  270,  456,   26,
+       26,   26,  363,  363,   30,  154,   68,   62,   56,   29,
+       26,   24,   30,   20,   30,   36,   39,   40,   13,   30,
+        5,    5,    6,    6,    6,    6,    6,    6,    6,    6,
+        6,    9,   10,   30,   34,   37,    6,    6,    6,   26,
+       31,   30,   33,   30,   31,   42,   33,   31,   30,   37,
+       33,   42,   46,   38,    0,   34,   33,   34,   38,   33,
+
+       31,    0,   34,   51,   37,    6,    6,   34,    0,   31,
+        0,   33,   54,   31,   42,   33,   31,   43,   37,   33,
+       42,   46,   38,   43,   34,   33,   34,   38,   33,   31,
+       32,   34,   51,   52,   52,   41,   34,   32,   45,   41,
+       81,   54,   45,   41,   32,   44,   43,   32,   32,   41,
+       32,    0,   43,   44,   83,   45,    0,   44,   84,   32,
+       44,   44,   52,   52,   41,    0,   32,   45,   41,   81,
+       85,   45,   41,   32,   44,    0,   32,   32,   41,   32,
+       35,    0,   44,   83,   45,   48,   44,   84,   35,   44,
+       44,   35,   87,   48,   35,    0,   49,   35,   47,   85,
+
+       35,   48,   47,   49,   49,   50,   47,   50,    0,   35,
+       50,   86,   47,   49,   48,   89,   49,   35,   73,   73,
+       35,   87,   48,   35,   86,   49,   35,   47,   73,   35,
+       48,   47,   49,   49,   50,   47,   50,   74,   74,   50,
+       86,   47,   49,   82,   89,   49,   65,   74,   75,   75,
+       75,   82,   90,   86,   65,   65,   90,   73,   91,   75,
+       91,   65,   91,   92,   93,   65,   94,   97,   98,   95,
+       99,  101,   82,   65,   95,    0,   74,   65,  102,   65,
+       82,   90,  103,   65,  105,   90,  106,   91,   75,   91,
+       65,   91,   92,   93,   65,   94,   97,   98,   95,   99,
+
+      101,   96,   65,   95,   96,  104,   65,  102,   65,  107,
+       96,  103,   65,  105,  108,  106,  104,   96,  109,  110,
+      108,  111,  112,  113,  116,  112,  114,  117,  118,  119,
+       96,  121,  122,   96,  104,  120,  114,  120,  107,   96,
+      123,  114,  114,  108,  124,  104,   96,  109,  110,  108,
+      111,  112,  113,  116,  112,  114,  117,  118,  119,  125,
+      121,  122,  126,  128,  120,  114,  120,  129,  130,  123,
+      114,  114,  131,  124,  132,  133,  134,  135,  136,  133,
+      137,  138,  140,  136,  136,  139,  141,  142,  125,  143,
+      144,  126,  128,  145,  139,  146,  129,  130,    0,  148,
+
+      149,  131,  139,  132,  133,  134,  135,  136,  133,  137,
+      138,  140,  136,  136,  139,  141,  142,  151,  143,  144,
+      147,  153,  145,  139,  146,  178,  150,  147,  148,  149,
+      152,  139,  150,  173,  173,  180,  152,  181,  182,  183,
+      184,  185,  186,  173,  187,  188,  151,  189,  190,  147,
+      153,  191,  184,  192,  178,  150,  147,  193,  194,  152,
+      196,  150,  197,  199,  180,  152,  181,  182,  183,  184,
+      185,  186,  173,  187,  188,  198,  189,  190,  200,  198,
+      191,  184,  192,  201,  202,  203,  193,  194,  205,  196,
+      206,  197,  199,  207,  208,  209,  210,  211,  212,  213,
+
+      214,  215,  216,  217,  198,  218,  219,  200,  198,  220,
+      221,  222,  201,  202,  203,  223,  225,  205,  226,  206,
+      227,  222,  207,  208,  209,  210,  211,  212,  213,  214,
+      215,  216,  217,  228,  218,  219,  229,  230,  220,  221,
+      222,  231,  233,  235,  223,  225,  236,  226,  237,  227,
+      222,  238,  239,  240,  241,  242,  241,  243,  244,  245,
+      246,  247,  228,  248,  249,  229,  230,  250,  252,  253,
+      231,  233,  235,  248,  254,  236,  255,  237,  256,  257,
+      238,  239,  240,  241,  242,  241,  243,  244,  245,  246,
+      247,  258,  248,  249,  259,  261,  250,  252,  253,  260,
+
+      262,  260,  263,  254,  264,  255,  265,  256,  257,  266,
+      265,  267,  268,  272,  273,  274,  275,  276,  277,  278,
+      258,  280,  281,  259,  261,  282,  283,  285,  260,  262,
+      260,  263,  286,  264,  287,  265,  288,  289,  266,  265,
+      267,  268,  272,  273,  274,  275,  276,  277,  278,  290,
+      280,  281,  291,  292,  282,  283,  285,  293,  294,  295,
+      298,  286,  299,  287,  300,  288,  289,  301,  302,  303,
+      304,  305,  308,  310,  312,  313,  314,  321,  290,  315,
+      323,  291,  292,  324,  325,  326,  293,  294,  295,  298,
+      315,  299,  327,  300,  329,  330,  301,  302,  303,  304,
+
+      305,  308,  310,  312,  313,  314,  321,  331,  315,  323,
+      332,  333,  324,  325,  326,  335,  337,  338,  339,  315,
+      341,  327,  342,  329,  330,  343,  344,  345,  346,  348,
+      350,  351,  352,  353,  354,  355,  331,  356,  357,  332,
+      333,  359,  360,  362,  335,  337,  338,  339,  366,  341,
+      367,  342,  368,  369,  343,  344,  345,  346,  348,  350,
+      351,  352,  353,  354,  355,  370,  356,  357,  370,  371,
+      359,  360,  362,  372,  374,  375,  376,  366,  377,  367,
+      378,  368,  369,  379,  380,  381,  382,  383,  384,  385,
+      386,  387,  388,  389,  370,  393,  394,  370,  371,  396,
+
+      399,  400,  372,  374,  375,  376,  403,  377,  401,  378,
+      408,  401,  379,  380,  381,  382,  383,  384,  385,  386,
+      387,  388,  389,  409,  393,  394,  410,  411,  396,  399,
+      400,  412,  414,  415,  417,  403,  418,  401,  419,  408,
+      401,  420,  421,  423,  424,  425,  427,  428,  430,  431,
+      433,  434,  409,  435,  436,  410,  411,  438,  439,  440,
+      412,  414,  415,  417,  441,  418,  442,  419,  444,  446,
+      420,  421,  423,  424,  425,  427,  428,  430,  431,  433,
+      434,  447,  435,  436,  448,  449,  438,  439,  440,  450,
+      452,  453,  454,  441,  458,  442,  459,  444,  446,  460,
+
+      463,  464,  465,  467,  468,  469,  470,  471,  472,  474,
+      447,  477,  478,  448,  449,  479,  480,  481,  450,  452,
+      453,  454,  485,  458,  487,  459,  488,  490,  460,  463,
+      464,  465,  467,  468,  469,  470,  471,  472,  474,  491,
+      477,  478,  492,  493,  479,  480,  481,  494,  495,  498,
+      501,  485,  502,  487,  503,  488,  490,  504,  506,  509,
+      510,  511,  513,  515,  516,  517,  518,  519,  491,  520,
+      521,  492,  493,  522,  524,  525,  494,  495,  498,  501,
+      526,  502,  527,  503,  528,  529,  504,  506,  509,  510,
+      511,  513,  515,  516,  517,  518,  519,  530,  520,  521,
+
+      531,  533,  522,  524,  525,  534,  536,  537,  538,  526,
+      540,  527,  541,  528,  529,  543,  544,  546,  547,  548,
+      549,  550,  552,  553,  554,  555,  530,  557,  559,  531,
+      533,  560,  563,  565,  534,  536,  537,  538,  566,  540,
+      569,  541,  573,  574,  543,  544,  546,  547,  548,  549,
+      550,  552,  553,  554,  555,  575,  557,  559,  578,  581,
+      560,  563,  565,  582,  583,  586,  587,  566,  589,  569,
+      590,  573,  574,  591,  592,  593,  595,    0,    0,    0,
+        0,    0,    0,    0,  575,    0,    0,  578,  581,    0,
+        0,    0,  582,  583,  586,  587,    0,  589,    0,  590,
+
+        0,    0,  591,  592,  593,  595,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  601,  601,  601,  601,
+      601,  601,  601,  601,  601,  601,  602,  602,  602,  602,
+      602,  602,  602,  602,  602,  602,  603,  603,  603,  603,
+      603,  603,  603,  603,  603,  603,  604,  604,  605,  605,
+      605,    0,  605,  606,  606,  606,  606,    0,  606,  607,
+      607,  607,    0,  607,  607,  607,  607,  607,  607,  608,
+      608,  608,    0,  608,  608,  608,  608,    0,  608,  609,
+      609,  609,  609,  609,  609,  609,  609,  609,  609,  610,
+
+      610,    0,  610,  610,  610,  610,  610,  610,  610,  611,
+        0,  611,  611,  611,  611,  611,  611,  611,  611,  612,
+      612,  613,  613,  598,  598,  598,  598,  598,  598,  598,
       598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
       598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
       598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
       598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
       598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
       598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598
+      598,  598,  598,  598,  598
+
     } ;
 
 /* Table of booleans, true if rule could match eol. */
-static yyconst flex_int32_t yy_rule_can_match_eol[164] =
+static const flex_int32_t yy_rule_can_match_eol[164] =
     {   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, 
@@ -1082,17 +1135,14 @@ class UnaryOperation;
     yycolumn += yyleng;                                   \
   }
 
+#line 1138 "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 1096 "SqlLexer_gen.cpp"
+#line 1145 "SqlLexer_gen.cpp"
 
 #define INITIAL 0
 #define CONDITION_SQL 1
@@ -1126,8 +1176,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;
@@ -1151,7 +1201,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.*/
@@ -1161,48 +1211,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.
@@ -1210,9 +1260,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
 
@@ -1221,19 +1271,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
@@ -1253,7 +1302,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,
@@ -1264,7 +1313,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; \
@@ -1277,7 +1326,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) \
 				{ \
@@ -1319,7 +1368,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)
@@ -1383,7 +1432,7 @@ YY_DECL
 #line 132 "../SqlLexer.lpp"
 
 
-#line 1387 "SqlLexer_gen.cpp"
+#line 1435 "SqlLexer_gen.cpp"
 
 	while ( /*CONSTCOND*/1 )		/* loops until end-of-file is reached */
 		{
@@ -1411,9 +1460,9 @@ yy_match:
 				{
 				yy_current_state = (int) yy_def[yy_current_state];
 				if ( yy_current_state >= 599 )
-					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 != 598 );
@@ -1427,10 +1476,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)
@@ -2425,7 +2474,7 @@ YY_RULE_SETUP
 #line 469 "../SqlLexer.lpp"
 YY_FATAL_ERROR( "flex scanner jammed" );
 	YY_BREAK
-#line 2429 "SqlLexer_gen.cpp"
+#line 2477 "SqlLexer_gen.cpp"
 
 	case YY_END_OF_BUFFER:
 		{
@@ -2570,7 +2619,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] )
@@ -2599,7 +2648,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++);
@@ -2612,7 +2661,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 )
@@ -2626,7 +2675,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;
@@ -2635,11 +2684,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(
@@ -2681,10 +2730,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()" );
 	}
@@ -2720,9 +2769,9 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 			{
 			yy_current_state = (int) yy_def[yy_current_state];
 			if ( yy_current_state >= 599 )
-				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;
@@ -2749,9 +2798,9 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 		{
 		yy_current_state = (int) yy_def[yy_current_state];
 		if ( yy_current_state >= 599 )
-			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 == 598);
 
 	(void)yyg;
@@ -2787,7 +2836,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 ) )
@@ -2811,7 +2860,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;
@@ -2834,7 +2883,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)
@@ -2922,12 +2971,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()" );
 
@@ -3083,15 +3132,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;
@@ -3120,7 +3169,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)
 {
@@ -3130,16 +3179,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;
@@ -3159,10 +3208,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
@@ -3172,15 +3221,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()" );
@@ -3206,7 +3255,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;
@@ -3248,7 +3297,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;
     
@@ -3261,7 +3310,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;
     
@@ -3289,7 +3338,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;
@@ -3407,9 +3456,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;
@@ -3436,9 +3483,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;
 
@@ -3448,20 +3493,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 );
 }
 
@@ -3472,10 +3517,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;
 
@@ -3488,8 +3533,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
@@ -3533,7 +3578,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;
@@ -3545,7 +3590,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 )
@@ -3559,7 +3604,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)
@@ -3574,7 +3619,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)
@@ -3589,4 +3634,3 @@ void quickstep_yyfree (void * ptr , yyscan_t yyscanner)
 #line 469 "../SqlLexer.lpp"
 
 
-

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/678e2c39/parser/preprocessed/SqlLexer_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.hpp b/parser/preprocessed/SqlLexer_gen.hpp
index f8fc00b..86e36f0 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 469 "../SqlLexer.lpp"
 
 
-#line 367 "SqlLexer_gen.hpp"
+#line 432 "SqlLexer_gen.hpp"
 #undef quickstep_yyIN_HEADER
 #endif /* quickstep_yyHEADER_H */


[27/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index a6d12e2..2a1f528 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -81,103 +81,104 @@ extern int quickstep_yydebug;
     TOKEN_CSB_TREE = 291,
     TOKEN_BY = 292,
     TOKEN_CASE = 293,
-    TOKEN_CHARACTER = 294,
-    TOKEN_CHECK = 295,
-    TOKEN_COLUMN = 296,
-    TOKEN_CONSTRAINT = 297,
-    TOKEN_COPY = 298,
-    TOKEN_CREATE = 299,
-    TOKEN_CURRENT = 300,
-    TOKEN_DATE = 301,
-    TOKEN_DATETIME = 302,
-    TOKEN_DAY = 303,
-    TOKEN_DECIMAL = 304,
-    TOKEN_DEFAULT = 305,
-    TOKEN_DELETE = 306,
-    TOKEN_DELIMITER = 307,
-    TOKEN_DESC = 308,
-    TOKEN_DISTINCT = 309,
-    TOKEN_DOUBLE = 310,
-    TOKEN_DROP = 311,
-    TOKEN_ELSE = 312,
-    TOKEN_END = 313,
-    TOKEN_ESCAPE_STRINGS = 314,
-    TOKEN_EXISTS = 315,
-    TOKEN_EXTRACT = 316,
-    TOKEN_FALSE = 317,
-    TOKEN_FIRST = 318,
-    TOKEN_FLOAT = 319,
-    TOKEN_FOLLOWING = 320,
-    TOKEN_FOR = 321,
-    TOKEN_FOREIGN = 322,
-    TOKEN_FROM = 323,
-    TOKEN_FULL = 324,
-    TOKEN_GROUP = 325,
-    TOKEN_HASH = 326,
-    TOKEN_HAVING = 327,
-    TOKEN_HOUR = 328,
-    TOKEN_IN = 329,
-    TOKEN_INDEX = 330,
-    TOKEN_INNER = 331,
-    TOKEN_INSERT = 332,
-    TOKEN_INTEGER = 333,
-    TOKEN_INTERVAL = 334,
-    TOKEN_INTO = 335,
-    TOKEN_JOIN = 336,
-    TOKEN_KEY = 337,
-    TOKEN_LAST = 338,
-    TOKEN_LEFT = 339,
-    TOKEN_LIMIT = 340,
-    TOKEN_LONG = 341,
-    TOKEN_MINUTE = 342,
-    TOKEN_MONTH = 343,
-    TOKEN_NULL = 344,
-    TOKEN_NULLS = 345,
-    TOKEN_OFF = 346,
-    TOKEN_ON = 347,
-    TOKEN_ORDER = 348,
-    TOKEN_OUTER = 349,
-    TOKEN_OVER = 350,
-    TOKEN_PARTITION = 351,
-    TOKEN_PARTITIONS = 352,
-    TOKEN_PERCENT = 353,
-    TOKEN_PRECEDING = 354,
-    TOKEN_PRIMARY = 355,
-    TOKEN_PRIORITY = 356,
-    TOKEN_QUIT = 357,
-    TOKEN_RANGE = 358,
-    TOKEN_REAL = 359,
-    TOKEN_REFERENCES = 360,
-    TOKEN_RIGHT = 361,
-    TOKEN_ROW = 362,
-    TOKEN_ROW_DELIMITER = 363,
-    TOKEN_ROWS = 364,
-    TOKEN_SECOND = 365,
-    TOKEN_SELECT = 366,
-    TOKEN_SET = 367,
-    TOKEN_SMA = 368,
-    TOKEN_SMALLINT = 369,
-    TOKEN_SUBSTRING = 370,
-    TOKEN_TABLE = 371,
-    TOKEN_THEN = 372,
-    TOKEN_TIME = 373,
-    TOKEN_TIMESTAMP = 374,
-    TOKEN_TRUE = 375,
-    TOKEN_TUPLESAMPLE = 376,
-    TOKEN_UNBOUNDED = 377,
-    TOKEN_UNIQUE = 378,
-    TOKEN_UPDATE = 379,
-    TOKEN_USING = 380,
-    TOKEN_VALUES = 381,
-    TOKEN_VARCHAR = 382,
-    TOKEN_WHEN = 383,
-    TOKEN_WHERE = 384,
-    TOKEN_WINDOW = 385,
-    TOKEN_WITH = 386,
-    TOKEN_YEAR = 387,
-    TOKEN_YEARMONTH = 388,
-    TOKEN_EOF = 389,
-    TOKEN_LEX_ERROR = 390
+    TOKEN_CAST = 294,
+    TOKEN_CHARACTER = 295,
+    TOKEN_CHECK = 296,
+    TOKEN_COLUMN = 297,
+    TOKEN_CONSTRAINT = 298,
+    TOKEN_COPY = 299,
+    TOKEN_CREATE = 300,
+    TOKEN_CURRENT = 301,
+    TOKEN_DATE = 302,
+    TOKEN_DATETIME = 303,
+    TOKEN_DAY = 304,
+    TOKEN_DECIMAL = 305,
+    TOKEN_DEFAULT = 306,
+    TOKEN_DELETE = 307,
+    TOKEN_DELIMITER = 308,
+    TOKEN_DESC = 309,
+    TOKEN_DISTINCT = 310,
+    TOKEN_DOUBLE = 311,
+    TOKEN_DROP = 312,
+    TOKEN_ELSE = 313,
+    TOKEN_END = 314,
+    TOKEN_ESCAPE_STRINGS = 315,
+    TOKEN_EXISTS = 316,
+    TOKEN_EXTRACT = 317,
+    TOKEN_FALSE = 318,
+    TOKEN_FIRST = 319,
+    TOKEN_FLOAT = 320,
+    TOKEN_FOLLOWING = 321,
+    TOKEN_FOR = 322,
+    TOKEN_FOREIGN = 323,
+    TOKEN_FROM = 324,
+    TOKEN_FULL = 325,
+    TOKEN_GROUP = 326,
+    TOKEN_HASH = 327,
+    TOKEN_HAVING = 328,
+    TOKEN_HOUR = 329,
+    TOKEN_IN = 330,
+    TOKEN_INDEX = 331,
+    TOKEN_INNER = 332,
+    TOKEN_INSERT = 333,
+    TOKEN_INTEGER = 334,
+    TOKEN_INTERVAL = 335,
+    TOKEN_INTO = 336,
+    TOKEN_JOIN = 337,
+    TOKEN_KEY = 338,
+    TOKEN_LAST = 339,
+    TOKEN_LEFT = 340,
+    TOKEN_LIMIT = 341,
+    TOKEN_LONG = 342,
+    TOKEN_MINUTE = 343,
+    TOKEN_MONTH = 344,
+    TOKEN_NULL = 345,
+    TOKEN_NULLS = 346,
+    TOKEN_OFF = 347,
+    TOKEN_ON = 348,
+    TOKEN_ORDER = 349,
+    TOKEN_OUTER = 350,
+    TOKEN_OVER = 351,
+    TOKEN_PARTITION = 352,
+    TOKEN_PARTITIONS = 353,
+    TOKEN_PERCENT = 354,
+    TOKEN_PRECEDING = 355,
+    TOKEN_PRIMARY = 356,
+    TOKEN_PRIORITY = 357,
+    TOKEN_QUIT = 358,
+    TOKEN_RANGE = 359,
+    TOKEN_REAL = 360,
+    TOKEN_REFERENCES = 361,
+    TOKEN_RIGHT = 362,
+    TOKEN_ROW = 363,
+    TOKEN_ROW_DELIMITER = 364,
+    TOKEN_ROWS = 365,
+    TOKEN_SECOND = 366,
+    TOKEN_SELECT = 367,
+    TOKEN_SET = 368,
+    TOKEN_SMA = 369,
+    TOKEN_SMALLINT = 370,
+    TOKEN_SUBSTRING = 371,
+    TOKEN_TABLE = 372,
+    TOKEN_THEN = 373,
+    TOKEN_TIME = 374,
+    TOKEN_TIMESTAMP = 375,
+    TOKEN_TRUE = 376,
+    TOKEN_TUPLESAMPLE = 377,
+    TOKEN_UNBOUNDED = 378,
+    TOKEN_UNIQUE = 379,
+    TOKEN_UPDATE = 380,
+    TOKEN_USING = 381,
+    TOKEN_VALUES = 382,
+    TOKEN_VARCHAR = 383,
+    TOKEN_WHEN = 384,
+    TOKEN_WHERE = 385,
+    TOKEN_WINDOW = 386,
+    TOKEN_WITH = 387,
+    TOKEN_YEAR = 388,
+    TOKEN_YEARMONTH = 389,
+    TOKEN_EOF = 390,
+    TOKEN_LEX_ERROR = 391
   };
 #endif
 
@@ -186,7 +187,7 @@ extern int quickstep_yydebug;
 
 union YYSTYPE
 {
-#line 121 "../SqlParser.ypp" /* yacc.c:1915  */
+#line 115 "../SqlParser.ypp" /* yacc.c:1915  */
 
   quickstep::ParseString *string_value_;
 
@@ -259,8 +260,8 @@ union YYSTYPE
   quickstep::ParseStatementQuit *quit_statement_;
 
   const quickstep::Comparison *comparison_;
-  const quickstep::UnaryOperation *unary_operation_;
-  const quickstep::BinaryOperation *binary_operation_;
+  quickstep::ParseString *unary_operation_;
+  quickstep::ParseString *binary_operation_;
 
   quickstep::ParseFunctionCall *function_call_;
   quickstep::PtrList<quickstep::ParseExpression> *expression_list_;
@@ -288,7 +289,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 292 "SqlParser_gen.hpp" /* yacc.c:1915  */
+#line 293 "SqlParser_gen.hpp" /* yacc.c:1915  */
 };
 
 typedef union YYSTYPE YYSTYPE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/LogicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/LogicalGenerator.cpp b/query_optimizer/LogicalGenerator.cpp
index abeca53..aaad96a 100644
--- a/query_optimizer/LogicalGenerator.cpp
+++ b/query_optimizer/LogicalGenerator.cpp
@@ -51,9 +51,12 @@ L::LogicalPtr LogicalGenerator::generatePlan(
     const CatalogDatabase &catalog_database,
     const ParseStatement &parse_statement) {
   resolver::Resolver resolver(catalog_database, optimizer_context_);
-  DVLOG(4) << "Parse tree:\n" << parse_statement.toString();
+//  DVLOG(4) << "Parse tree:\n" << parse_statement.toString();
+  std::cerr << "Parse tree:\n" << parse_statement.toString();
   logical_plan_ = resolver.resolve(parse_statement);
-  DVLOG(4) << "Initial logical plan:\n" << logical_plan_->toString();
+//  DVLOG(4) << "Initial logical plan:\n" << logical_plan_->toString();
+  std::cerr << "Initial logical plan:\n" << logical_plan_->toString();
+//  exit(0);
 
   optimizePlan();
   DVLOG(4) << "Optimized logical plan:\n" << logical_plan_->toString();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/expressions/BinaryExpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/BinaryExpression.cpp b/query_optimizer/expressions/BinaryExpression.cpp
index f49c6a2..24fee40 100644
--- a/query_optimizer/expressions/BinaryExpression.cpp
+++ b/query_optimizer/expressions/BinaryExpression.cpp
@@ -31,8 +31,8 @@
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/Expression.hpp"
 #include "query_optimizer/expressions/PatternMatcher.hpp"
+#include "query_optimizer/expressions/ScalarLiteral.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 #include "utility/HashPair.hpp"
 
 #include "glog/logging.h"
@@ -44,37 +44,8 @@ class Type;
 namespace optimizer {
 namespace expressions {
 
-BinaryExpression::BinaryExpression(const BinaryOperation &operation,
-                                   const ScalarPtr &left,
-                                   const ScalarPtr &right)
-    : operation_(operation), left_(left), right_(right) {
-  DCHECK(operation_.canApplyToTypes(left_->getValueType(),
-                                    right_->getValueType()))
-      << toString();
-  addChild(left_);
-  addChild(right_);
-}
-
 std::string BinaryExpression::getName() const {
-  switch (operation_.getBinaryOperationID()) {
-    case BinaryOperationID::kAdd:
-      return "Add";
-    case BinaryOperationID::kSubtract:
-      return "Subtract";
-    case BinaryOperationID::kMultiply:
-      return "Multiply";
-    case BinaryOperationID::kDivide:
-      return "Divide";
-    case BinaryOperationID::kModulo:
-      return "Modulo";
-    default:
-      LOG(FATAL) << "Unknown binary operation";
-  }
-}
-
-const Type &BinaryExpression::getValueType() const {
-  return *operation_.resultTypeForArgumentTypes(left_->getValueType(),
-                                                right_->getValueType());
+  return op_signature_->getName();
 }
 
 ExpressionPtr BinaryExpression::copyWithNewChildren(
@@ -83,9 +54,12 @@ ExpressionPtr BinaryExpression::copyWithNewChildren(
   DCHECK(SomeScalar::Matches(new_children[0]));
   DCHECK(SomeScalar::Matches(new_children[1]));
   return BinaryExpression::Create(
+      op_signature_,
       operation_,
       std::static_pointer_cast<const Scalar>(new_children[0]),
-      std::static_pointer_cast<const Scalar>(new_children[1]));
+      std::static_pointer_cast<const Scalar>(new_children[1]),
+      static_arguments_,
+      static_argument_types_);
 }
 
 std::vector<AttributeReferencePtr> BinaryExpression::getReferencedAttributes() const {
@@ -102,33 +76,42 @@ std::vector<AttributeReferencePtr> BinaryExpression::getReferencedAttributes() c
 ::quickstep::Scalar *BinaryExpression::concretize(
     const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const {
   return new ::quickstep::ScalarBinaryExpression(
+      op_signature_,
       operation_,
       left_->concretize(substitution_map),
-      right_->concretize(substitution_map));
+      right_->concretize(substitution_map),
+      static_arguments_);
 }
 
 std::size_t BinaryExpression::computeHash() const {
+  std::size_t hash_code = op_signature_->hash();
   std::size_t left_hash = left_->hash();
   std::size_t right_hash = right_->hash();
 
-  if (operation_.isCommutative() && left_hash > right_hash) {
+  if (operation_->isCommutative() && left_hash > right_hash) {
     std::swap(left_hash, right_hash);
   }
+  hash_code = CombineHashes(hash_code, left_hash);
+  hash_code = CombineHashes(hash_code, right_hash);
 
-  return CombineHashes(
-      CombineHashes(static_cast<std::size_t>(ExpressionType::kBinaryExpression),
-                    static_cast<std::size_t>(operation_.getBinaryOperationID())),
-      CombineHashes(left_hash, right_hash));
+  for (const TypedValue &st_arg : *static_arguments_) {
+    if (!st_arg.isNull()) {
+      hash_code = CombineHashes(hash_code, st_arg.getHash());
+    }
+  }
+  return hash_code;
 }
 
 bool BinaryExpression::equals(const ScalarPtr &other) const {
+  // TODO
   BinaryExpressionPtr expr;
   if (SomeBinaryExpression::MatchesWithConditionalCast(other, &expr) &&
-      &operation_ == &expr->operation_) {
+      *op_signature_ == *expr->op_signature_ &&
+      *static_arguments_ == *expr->static_arguments_) {
     ScalarPtr left = left_;
     ScalarPtr right = right_;
 
-    if (operation_.isCommutative()) {
+    if (operation_->isCommutative()) {
       const bool self_order = (left_->hash() < right_->hash());
       const bool other_order = (expr->left_->hash() < expr->right_->hash());
       if (self_order ^ other_order) {
@@ -148,8 +131,26 @@ void BinaryExpression::getFieldStringItems(
     std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
     std::vector<std::string> *container_child_field_names,
     std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
-  container_child_field_names->push_back("");
-  container_child_fields->push_back({left_, right_});
+  inline_field_names->emplace_back("op_signature");
+  inline_field_values->emplace_back(op_signature_->toString());
+
+  inline_field_names->emplace_back("result_type");
+  inline_field_values->emplace_back(result_type_.getName());
+
+  non_container_child_field_names->emplace_back("left_operand");
+  non_container_child_fields->emplace_back(left_);
+  non_container_child_field_names->emplace_back("right_operand");
+  non_container_child_fields->emplace_back(right_);
+
+  if (!static_arguments_->empty()) {
+    container_child_field_names->emplace_back("static_arguments");
+    container_child_fields->emplace_back();
+    for (std::size_t i = 0; i < static_arguments_->size(); ++i) {
+      container_child_fields->back().emplace_back(
+          ScalarLiteral::Create(static_arguments_->at(i),
+                                *static_argument_types_->at(i)));
+    }
+  }
 }
 
 }  // namespace expressions

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/expressions/BinaryExpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/BinaryExpression.hpp b/query_optimizer/expressions/BinaryExpression.hpp
index 6a37679..6ee6690 100644
--- a/query_optimizer/expressions/BinaryExpression.hpp
+++ b/query_optimizer/expressions/BinaryExpression.hpp
@@ -31,6 +31,8 @@
 #include "query_optimizer/expressions/Expression.hpp"
 #include "query_optimizer/expressions/ExpressionType.hpp"
 #include "query_optimizer/expressions/Scalar.hpp"
+#include "types/operations/OperationSignature.hpp"
+#include "types/operations/binary_operations/BinaryOperation.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -61,7 +63,9 @@ class BinaryExpression : public Scalar {
 
   std::string getName() const override;
 
-  const Type& getValueType() const override;
+  const Type& getValueType() const override {
+    return result_type_;
+  }
 
   bool isConstant() const override {
     return left_->isConstant() && right_->isConstant();
@@ -70,7 +74,7 @@ class BinaryExpression : public Scalar {
   /**
    * @return The binary operation.
    */
-  const BinaryOperation& operation() const { return operation_; }
+  const BinaryOperationPtr& operation() const { return operation_; }
 
   /**
    * @return The left operand.
@@ -92,10 +96,34 @@ class BinaryExpression : public Scalar {
 
   bool equals(const ScalarPtr &other) const override;
 
-  static BinaryExpressionPtr Create(const BinaryOperation &operation,
-                                    const ScalarPtr &left,
-                                    const ScalarPtr &right) {
-    return BinaryExpressionPtr(new BinaryExpression(operation, left, right));
+  static BinaryExpressionPtr Create(
+      const OperationSignaturePtr &op_signature,
+      const BinaryOperationPtr &operation,
+      const ScalarPtr &left,
+      const ScalarPtr &right,
+      const std::shared_ptr<const std::vector<TypedValue>> &static_arguments,
+      const std::shared_ptr<const std::vector<const Type*>> &static_argument_types) {
+    return BinaryExpressionPtr(
+        new BinaryExpression(op_signature,
+                             operation,
+                             left,
+                             right,
+                             static_arguments,
+                             static_argument_types));
+  }
+
+  static BinaryExpressionPtr Create(
+      const OperationSignaturePtr &op_signature,
+      const BinaryOperationPtr &operation,
+      const ScalarPtr &left,
+      const ScalarPtr &right) {
+    return BinaryExpressionPtr(
+        new BinaryExpression(op_signature,
+                             operation,
+                             left,
+                             right,
+                             std::make_shared<const std::vector<TypedValue>>(),
+                             std::make_shared<const std::vector<const Type*>>()));
   }
 
  protected:
@@ -110,14 +138,32 @@ class BinaryExpression : public Scalar {
       std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const override;
 
  private:
-  BinaryExpression(const BinaryOperation &operation,
+  BinaryExpression(const OperationSignaturePtr &op_signature,
+                   const BinaryOperationPtr &operation,
                    const ScalarPtr &left,
-                   const ScalarPtr &right);
-
-  const BinaryOperation &operation_;
+                   const ScalarPtr &right,
+                   const std::shared_ptr<const std::vector<TypedValue>> &static_arguments,
+                   const std::shared_ptr<const std::vector<const Type*>> &static_argument_types)
+      : op_signature_(op_signature),
+        operation_(operation),
+        left_(left),
+        right_(right),
+        static_arguments_(static_arguments),
+        static_argument_types_(static_argument_types),
+        result_type_(*(operation_->getResultType(left_->getValueType(),
+                                                 right_->getValueType(),
+                                                 *static_arguments))) {
+    addChild(left);
+    addChild(right);
+  }
 
-  ScalarPtr left_;
-  ScalarPtr right_;
+  const OperationSignaturePtr op_signature_;
+  const BinaryOperationPtr operation_;
+  const ScalarPtr left_;
+  const ScalarPtr right_;
+  const std::shared_ptr<const std::vector<TypedValue>> static_arguments_;
+  const std::shared_ptr<const std::vector<const Type*>> static_argument_types_;
+  const Type &result_type_;
 
   DISALLOW_COPY_AND_ASSIGN(BinaryExpression);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/CMakeLists.txt b/query_optimizer/expressions/CMakeLists.txt
index 3e7f8e4..dc722e7 100644
--- a/query_optimizer/expressions/CMakeLists.txt
+++ b/query_optimizer/expressions/CMakeLists.txt
@@ -20,7 +20,6 @@ add_library(quickstep_queryoptimizer_expressions_AggregateFunction AggregateFunc
 add_library(quickstep_queryoptimizer_expressions_Alias Alias.cpp Alias.hpp)
 add_library(quickstep_queryoptimizer_expressions_AttributeReference AttributeReference.cpp AttributeReference.hpp)
 add_library(quickstep_queryoptimizer_expressions_BinaryExpression BinaryExpression.cpp BinaryExpression.hpp)
-add_library(quickstep_queryoptimizer_expressions_Cast Cast.cpp Cast.hpp)
 add_library(quickstep_queryoptimizer_expressions_CommonSubexpression
             CommonSubexpression.cpp
             CommonSubexpression.hpp)
@@ -97,23 +96,9 @@ target_link_libraries(quickstep_queryoptimizer_expressions_BinaryExpression
                       quickstep_queryoptimizer_expressions_ExpressionType
                       quickstep_queryoptimizer_expressions_PatternMatcher
                       quickstep_queryoptimizer_expressions_Scalar
+                      quickstep_queryoptimizer_expressions_ScalarLiteral
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_HashPair
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_queryoptimizer_expressions_Cast
-                      glog
-                      quickstep_expressions_scalar_Scalar
-                      quickstep_expressions_scalar_ScalarUnaryExpression
-                      quickstep_queryoptimizer_OptimizerTree
-                      quickstep_queryoptimizer_expressions_AttributeReference
-                      quickstep_queryoptimizer_expressions_ExprId
-                      quickstep_queryoptimizer_expressions_Expression
-                      quickstep_queryoptimizer_expressions_ExpressionType
-                      quickstep_queryoptimizer_expressions_PatternMatcher
-                      quickstep_queryoptimizer_expressions_Scalar
-                      quickstep_types_Type
-                      quickstep_types_operations_unaryoperations_NumericCastOperation
                       quickstep_utility_HashPair
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_CommonSubexpression
@@ -325,8 +310,9 @@ target_link_libraries(quickstep_queryoptimizer_expressions_UnaryExpression
                       quickstep_queryoptimizer_expressions_ExpressionType
                       quickstep_queryoptimizer_expressions_PatternMatcher
                       quickstep_queryoptimizer_expressions_Scalar
+                      quickstep_queryoptimizer_expressions_ScalarLiteral
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_HashPair
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_WindowAggregateFunction
@@ -350,7 +336,6 @@ target_link_libraries(quickstep_queryoptimizer_expressions
                       quickstep_queryoptimizer_expressions_Alias
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_BinaryExpression
-                      quickstep_queryoptimizer_expressions_Cast
                       quickstep_queryoptimizer_expressions_CommonSubexpression
                       quickstep_queryoptimizer_expressions_ComparisonExpression
                       quickstep_queryoptimizer_expressions_Exists

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/expressions/Cast.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Cast.cpp b/query_optimizer/expressions/Cast.cpp
deleted file mode 100644
index e6eb1bd..0000000
--- a/query_optimizer/expressions/Cast.cpp
+++ /dev/null
@@ -1,91 +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 "query_optimizer/expressions/Cast.hpp"
-
-#include <cstddef>
-#include <string>
-#include <unordered_map>
-#include <vector>
-
-#include "expressions/scalar/Scalar.hpp"
-#include "expressions/scalar/ScalarUnaryExpression.hpp"
-#include "query_optimizer/OptimizerTree.hpp"
-#include "query_optimizer/expressions/AttributeReference.hpp"
-#include "query_optimizer/expressions/ExprId.hpp"
-#include "query_optimizer/expressions/Expression.hpp"
-#include "query_optimizer/expressions/PatternMatcher.hpp"
-#include "query_optimizer/expressions/Scalar.hpp"
-#include "types/Type.hpp"
-#include "types/operations/unary_operations/NumericCastOperation.hpp"
-#include "utility/HashPair.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-namespace optimizer {
-namespace expressions {
-
-ExpressionPtr Cast::copyWithNewChildren(
-    const std::vector<ExpressionPtr> &new_children) const {
-  DCHECK_EQ(getNumChildren(), new_children.size());
-  ScalarPtr scalar;
-  CHECK(SomeScalar::MatchesWithConditionalCast(new_children[0], &scalar))
-      << new_children[0]->toString();
-  return Create(scalar, target_type_);
-}
-
-::quickstep::Scalar *Cast::concretize(
-    const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const {
-  return new ::quickstep::ScalarUnaryExpression(::quickstep::NumericCastOperation::Instance(target_type_),
-                                                operand_->concretize(substitution_map));
-}
-
-std::size_t Cast::computeHash() const {
-  return CombineHashes(
-      CombineHashes(static_cast<std::size_t>(ExpressionType::kCast),
-                    operand_->hash()),
-      static_cast<std::size_t>(target_type_.getTypeID()));
-}
-
-bool Cast::equals(const ScalarPtr &other) const {
-  CastPtr expr;
-  if (SomeCast::MatchesWithConditionalCast(other, &expr)) {
-    return operand_->equals(expr->operand_) && target_type_.equals(expr->target_type_);
-  }
-  return false;
-}
-
-void Cast::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 {
-  inline_field_names->push_back("target_type");
-  inline_field_values->push_back(target_type_.getName());
-
-  non_container_child_field_names->push_back("operand");
-  non_container_child_fields->push_back(operand_);
-}
-
-}  // namespace expressions
-}  // namespace optimizer
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/expressions/Cast.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Cast.hpp b/query_optimizer/expressions/Cast.hpp
deleted file mode 100644
index 11be775..0000000
--- a/query_optimizer/expressions/Cast.hpp
+++ /dev/null
@@ -1,125 +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_QUERY_OPTIMIZER_EXPRESSIONS_CAST_HPP_
-#define QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_CAST_HPP_
-
-#include <memory>
-#include <string>
-#include <unordered_map>
-#include <vector>
-
-#include "query_optimizer/OptimizerTree.hpp"
-#include "query_optimizer/expressions/AttributeReference.hpp"
-#include "query_optimizer/expressions/ExprId.hpp"
-#include "query_optimizer/expressions/Expression.hpp"
-#include "query_optimizer/expressions/ExpressionType.hpp"
-#include "query_optimizer/expressions/Scalar.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-class CatalogAttribute;
-class Type;
-
-namespace optimizer {
-namespace expressions {
-
-/** \addtogroup OptimizerExpressions
- *  @{
- */
-
-class Cast;
-typedef std::shared_ptr<const Cast> CastPtr;
-
-/**
- * @brief Converts a value of a type to another type.
- */
-class Cast : public Scalar {
- public:
-  ExpressionType getExpressionType() const override { return ExpressionType::kCast; }
-
-  std::string getName() const override { return "Cast"; }
-
-  const Type& getValueType() const override { return target_type_; }
-
-  bool isConstant() const override { return operand_->isConstant(); }
-
-  /**
-   * @return The expression to be coerced.
-   */
-  const ScalarPtr& operand() const { return operand_; }
-
-  std::vector<AttributeReferencePtr> getReferencedAttributes() const override {
-    return operand_->getReferencedAttributes();
-  }
-
-  ExpressionPtr copyWithNewChildren(
-      const std::vector<ExpressionPtr> &new_children) const override;
-
-  ::quickstep::Scalar* concretize(
-      const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const override;
-
-  bool equals(const ScalarPtr &other) const override;
-
-  /**
-   * @brief Creates a Cast expression that converts \p operand to \p target_type.
-   *
-   * @param operand The input expression to be coerced.
-   * @param target_type The target type that the expression is converted to.
-   * @return A Cast expression.
-   */
-  static CastPtr Create(const ScalarPtr &operand, const Type &target_type) {
-    return CastPtr(new Cast(operand, target_type));
-  }
-
- protected:
-  std::size_t computeHash() const override;
-
-  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:
-  Cast(const ScalarPtr &operand, const Type &target_type)
-      : operand_(operand),
-        target_type_(target_type) {
-    addChild(operand);
-    DCHECK(target_type.isCoercibleFrom(operand->getValueType()));
-  }
-
-  ScalarPtr operand_;
-  const Type &target_type_;
-
-  DISALLOW_COPY_AND_ASSIGN(Cast);
-};
-
-/** @} */
-
-}  // namespace expressions
-}  // namespace optimizer
-}  // namespace quickstep
-
-#endif /* QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_CAST_HPP_ */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/expressions/UnaryExpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/UnaryExpression.cpp b/query_optimizer/expressions/UnaryExpression.cpp
index b448553..e1ad014 100644
--- a/query_optimizer/expressions/UnaryExpression.cpp
+++ b/query_optimizer/expressions/UnaryExpression.cpp
@@ -30,8 +30,8 @@
 #include "query_optimizer/expressions/Expression.hpp"
 #include "query_optimizer/expressions/PatternMatcher.hpp"
 #include "query_optimizer/expressions/Scalar.hpp"
+#include "query_optimizer/expressions/ScalarLiteral.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
 #include "utility/HashPair.hpp"
 
 #include "glog/logging.h"
@@ -41,7 +41,7 @@ namespace optimizer {
 namespace expressions {
 
 std::string UnaryExpression::getName() const {
-  return operation_.getName();
+  return op_signature_->getName();
 }
 
 ExpressionPtr UnaryExpression::copyWithNewChildren(
@@ -49,26 +49,39 @@ ExpressionPtr UnaryExpression::copyWithNewChildren(
   DCHECK_EQ(new_children.size(), children().size());
   DCHECK(SomeScalar::Matches(new_children[0]));
   return UnaryExpression::Create(
-      operation_, std::static_pointer_cast<const Scalar>(new_children[0]));
+      op_signature_,
+      operation_,
+      std::static_pointer_cast<const Scalar>(new_children[0]),
+      static_arguments_,
+      static_argument_types_);
 }
 
 ::quickstep::Scalar* UnaryExpression::concretize(
     const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const {
   return new ::quickstep::ScalarUnaryExpression(
-      operation_, operand_->concretize(substitution_map));
+      op_signature_,
+      operation_,
+      operand_->concretize(substitution_map),
+      static_arguments_);
 }
 
 std::size_t UnaryExpression::computeHash() const {
-  return CombineHashes(
-      CombineHashes(static_cast<std::size_t>(ExpressionType::kUnaryExpression),
-                    static_cast<std::size_t>(operation_.getUnaryOperationID())),
-      operand_->hash());
+  std::size_t hash_code = CombineHashes(op_signature_->hash(),
+                                        operand_->hash());
+  for (const TypedValue &st_arg : *static_arguments_) {
+    if (!st_arg.isNull()) {
+      hash_code = CombineHashes(hash_code, st_arg.getHash());
+    }
+  }
+  return hash_code;
 }
 
 bool UnaryExpression::equals(const ScalarPtr &other) const {
   UnaryExpressionPtr expr;
   if (SomeUnaryExpression::MatchesWithConditionalCast(other, &expr)) {
-    return &operation_ == &expr->operation_ && operand_->equals(expr->operand_);
+    return *op_signature_ == *expr->op_signature_
+        && operand_->equals(expr->operand_)
+        && *static_arguments_ == *expr->static_arguments_;
   }
   return false;
 }
@@ -80,8 +93,24 @@ void UnaryExpression::getFieldStringItems(
     std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
     std::vector<std::string> *container_child_field_names,
     std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
-  non_container_child_field_names->push_back("Operand");
-  non_container_child_fields->push_back(operand_);
+  inline_field_names->emplace_back("op_signature");
+  inline_field_values->emplace_back(op_signature_->toString());
+
+  inline_field_names->emplace_back("result_type");
+  inline_field_values->emplace_back(result_type_.getName());
+
+  non_container_child_field_names->emplace_back("operand");
+  non_container_child_fields->emplace_back(operand_);
+
+  if (!static_arguments_->empty()) {
+    container_child_field_names->emplace_back("static_arguments");
+    container_child_fields->emplace_back();
+    for (std::size_t i = 0; i < static_arguments_->size(); ++i) {
+      container_child_fields->back().emplace_back(
+          ScalarLiteral::Create(static_arguments_->at(i),
+                                *static_argument_types_->at(i)));
+    }
+  }
 }
 
 }  // namespace expressions

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/expressions/UnaryExpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/UnaryExpression.hpp b/query_optimizer/expressions/UnaryExpression.hpp
index 14201ff..bbb1841 100644
--- a/query_optimizer/expressions/UnaryExpression.hpp
+++ b/query_optimizer/expressions/UnaryExpression.hpp
@@ -31,6 +31,7 @@
 #include "query_optimizer/expressions/Expression.hpp"
 #include "query_optimizer/expressions/ExpressionType.hpp"
 #include "query_optimizer/expressions/Scalar.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
 #include "utility/Macros.hpp"
 
@@ -65,7 +66,7 @@ class UnaryExpression : public Scalar {
   /**
    * @return The unary operator.
    */
-  const UnaryOperation& operation() const { return operation_; }
+  const UnaryOperationPtr& operation() const { return operation_; }
 
   /**
    * @return The operand of the unary operator.
@@ -73,7 +74,7 @@ class UnaryExpression : public Scalar {
   const ScalarPtr& operand() const { return operand_; }
 
   const Type& getValueType() const override {
-    return *(operation_.resultTypeForArgumentType(operand_->getValueType()));
+    return result_type_;
   }
 
   ExpressionPtr copyWithNewChildren(
@@ -96,9 +97,18 @@ class UnaryExpression : public Scalar {
    * @return An immutable UnaryExpression that applies the operation to the
    *         operand.
    */
-  static UnaryExpressionPtr Create(const UnaryOperation &operation,
-                                   const ScalarPtr &operand) {
-    return UnaryExpressionPtr(new UnaryExpression(operation, operand));
+  static UnaryExpressionPtr Create(
+      const OperationSignaturePtr &op_signature,
+      const UnaryOperationPtr &operation,
+      const ScalarPtr &operand,
+      const std::shared_ptr<const std::vector<TypedValue>> &static_arguments,
+      const std::shared_ptr<const std::vector<const Type*>> &static_argument_types) {
+    return UnaryExpressionPtr(
+        new UnaryExpression(op_signature,
+                            operation,
+                            operand,
+                            static_arguments,
+                            static_argument_types));
   }
 
  protected:
@@ -113,15 +123,26 @@ class UnaryExpression : public Scalar {
       std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const override;
 
  private:
-  UnaryExpression(const UnaryOperation &operation,
-                  const ScalarPtr &operand)
-      : operation_(operation), operand_(operand) {
-    DCHECK(operation_.canApplyToType(operand_->getValueType())) << toString();
+  UnaryExpression(const OperationSignaturePtr &op_signature,
+                  const UnaryOperationPtr &operation,
+                  const ScalarPtr &operand,
+                  const std::shared_ptr<const std::vector<TypedValue>> &static_arguments,
+                  const std::shared_ptr<const std::vector<const Type*>> &static_argument_types)
+      : op_signature_(op_signature),
+        operation_(operation),
+        operand_(operand),
+        static_arguments_(static_arguments),
+        static_argument_types_(static_argument_types),
+        result_type_(*(operation_->getResultType(operand_->getValueType(), *static_arguments_))) {
     addChild(operand);
   }
 
-  const UnaryOperation &operation_;
-  ScalarPtr operand_;
+  const OperationSignaturePtr op_signature_;
+  const UnaryOperationPtr operation_;
+  const ScalarPtr operand_;
+  const std::shared_ptr<const std::vector<TypedValue>> static_arguments_;
+  const std::shared_ptr<const std::vector<const Type*>> static_argument_types_;
+  const Type &result_type_;
 
   DISALLOW_COPY_AND_ASSIGN(UnaryExpression);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/resolver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/CMakeLists.txt b/query_optimizer/resolver/CMakeLists.txt
index 4e364a6..64ffcdf 100644
--- a/query_optimizer/resolver/CMakeLists.txt
+++ b/query_optimizer/resolver/CMakeLists.txt
@@ -74,7 +74,6 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_queryoptimizer_expressions_Alias
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_BinaryExpression
-                      quickstep_queryoptimizer_expressions_Cast
                       quickstep_queryoptimizer_expressions_ComparisonExpression
                       quickstep_queryoptimizer_expressions_Exists
                       quickstep_queryoptimizer_expressions_ExprId
@@ -122,14 +121,15 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_storage_StorageConstants
                       quickstep_types_IntType
                       quickstep_types_Type
+                      quickstep_types_TypeUtil
                       quickstep_types_TypedValue
                       quickstep_types_TypeFactory
+                      quickstep_types_operations_OperationFactory
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_binaryoperations_BinaryOperation
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
-                      quickstep_types_operations_unaryoperations_DateExtractOperation
-                      quickstep_types_operations_unaryoperations_SubstringOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_utility_Macros
                       quickstep_utility_PtrList

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index 0f65255..d361679 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -70,7 +70,6 @@
 #include "query_optimizer/expressions/Alias.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/BinaryExpression.hpp"
-#include "query_optimizer/expressions/Cast.hpp"
 #include "query_optimizer/expressions/ComparisonExpression.hpp"
 #include "query_optimizer/expressions/Exists.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
@@ -117,14 +116,15 @@
 #include "storage/StorageConstants.hpp"
 #include "types/IntType.hpp"
 #include "types/Type.hpp"
-#include "types/TypedValue.hpp"
 #include "types/TypeFactory.hpp"
+#include "types/TypeUtil.hpp"
+#include "types/TypedValue.hpp"
+#include "types/operations/OperationFactory.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
-#include "types/operations/unary_operations/DateExtractOperation.hpp"
-#include "types/operations/unary_operations/SubstringOperation.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
 #include "utility/PtrList.hpp"
 #include "utility/PtrVector.hpp"
@@ -143,6 +143,26 @@ namespace E = ::quickstep::optimizer::expressions;
 namespace L = ::quickstep::optimizer::logical;
 namespace S = ::quickstep::serialization;
 
+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
+
 struct Resolver::ExpressionResolutionInfo {
   /**
    * @brief Constructs an ExpressionResolutionInfo that disallows aggregate
@@ -491,26 +511,6 @@ L::LogicalPtr Resolver::resolveCreateTable(
   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
@@ -875,14 +875,15 @@ L::LogicalPtr Resolver::resolveInsertSelection(
           selection_type.getSuperTypeID() == Type::SuperTypeID::kNumeric &&
           destination_type.isSafelyCoercibleFrom(selection_type)) {
         // Add cast operation
-        const E::AttributeReferencePtr attr = selection_attributes[aid];
-        const E::ExpressionPtr cast_expr =
-            E::Cast::Create(attr, destination_type);
-        cast_expressions.emplace_back(
-            E::Alias::Create(context_->nextExprId(),
-                             cast_expr,
-                             attr->attribute_name(),
-                             attr->attribute_alias()));
+//        const E::AttributeReferencePtr attr = selection_attributes[aid];
+//        const E::ExpressionPtr cast_expr =
+//            E::Cast::Create(attr, destination_type);
+//        cast_expressions.emplace_back(
+//            E::Alias::Create(context_->nextExprId(),
+//                             cast_expr,
+//                             attr->attribute_name(),
+//                             attr->attribute_alias()));
+        THROW_SQL_ERROR_AT(insert_statement.relation_name()) << "TODO: not handled";
       } else {
         THROW_SQL_ERROR_AT(insert_statement.relation_name())
             << "The assigned value for the column "
@@ -1020,8 +1021,9 @@ L::LogicalPtr Resolver::resolveUpdate(
     // Coerce the assignment expression if its Type is not equal to that of the
     // assigned attribute.
     if (!assignment_expression->getValueType().equals(attribute->getValueType())) {
-      assignment_expression =
-          E::Cast::Create(assignment_expression, attribute->getValueType());
+//      assignment_expression =
+//          E::Cast::Create(assignment_expression, attribute->getValueType());
+      THROW_SQL_ERROR_AT(&assignment) << "TODO: not handled";
     }
     if (assignee_ids.find(attribute->id()) != assignee_ids.end()) {
       THROW_SQL_ERROR_AT(&assignment) << "Multiple assignments to the column "
@@ -1439,11 +1441,12 @@ L::LogicalPtr Resolver::resolveSetOperations(
       if (possible_type.equals(current_type)) {
         cast_expressions.emplace_back(current_attr);
       } else {
-        cast_expressions.emplace_back(
-            E::Alias::Create(context_->nextExprId(),
-                             E::Cast::Create(current_attr, possible_type),
-                             current_attr->attribute_name(),
-                             current_attr->attribute_alias()));
+//        cast_expressions.emplace_back(
+//            E::Alias::Create(context_->nextExprId(),
+//                             E::Cast::Create(current_attr, possible_type),
+//                             current_attr->attribute_name(),
+//                             current_attr->attribute_alias()));
+        LOG(FATAL) << "TODO: not handled";
       }
     }
     resolved_operations[opid] = L::Project::Create(resolved_operations[opid], cast_expressions);
@@ -2281,109 +2284,6 @@ E::ScalarPtr Resolver::resolveExpression(
           parse_attribute_scalar.attr_name(),
           parse_attribute_scalar.rel_name());
     }
-    case ParseExpression::kBinaryExpression: {
-      const ParseBinaryExpression &parse_binary_scalar =
-          static_cast<const ParseBinaryExpression&>(parse_expression);
-
-      std::pair<const Type*, const Type*> argument_type_hints
-          = parse_binary_scalar.op().pushDownTypeHint(type_hint);
-
-      ExpressionResolutionInfo left_resolution_info(
-          *expression_resolution_info);
-      E::ScalarPtr left_argument = resolveExpression(
-          *parse_binary_scalar.left_operand(),
-           argument_type_hints.first,
-           &left_resolution_info);
-
-      ExpressionResolutionInfo right_resolution_info(
-          *expression_resolution_info);
-      E::ScalarPtr right_argument = resolveExpression(
-          *parse_binary_scalar.right_operand(),
-          argument_type_hints.second,
-          &right_resolution_info);
-
-      if (left_resolution_info.hasAggregate()) {
-        expression_resolution_info->parse_aggregate_expression =
-            left_resolution_info.parse_aggregate_expression;
-      } else if (right_resolution_info.hasAggregate()) {
-        expression_resolution_info->parse_aggregate_expression =
-            right_resolution_info.parse_aggregate_expression;
-      }
-
-      // Check if either argument is a NULL literal of an unknown type.
-      const bool left_is_nulltype = (left_argument->getValueType().getTypeID() == kNullType);
-      const bool right_is_nulltype = (right_argument->getValueType().getTypeID() == kNullType);
-
-      // If either argument is a NULL of unknown type, we try to resolve the
-      // type of this BinaryExpression as follows:
-      //
-      //     1. If there is only one possible result type for the expression
-      //        based on what is known about its argument types, then the
-      //        result is a NULL of that type.
-      //     2. Otherwise, if there is a type hint for the BinaryExpression's
-      //        result, and if it is a plausible result type based on what we
-      //        know about argument types, then the result is a NULL of the
-      //        hint type.
-      //     3. Otherwise, check if the BinaryExpression can plausibly be
-      //        applied to the known argument types at all. If so, then the
-      //        result is a NULL of unknown type (i.e. NullType).
-      //     4. If all of the above steps fail, then the BinaryExpression is
-      //        not possibly applicable to the given arguments.
-      //
-      // NOTE(chasseur): Step #3 above does not completely capture knowledge
-      // about the result type of a BinaryExpression with one or more unknown
-      // arguments. For instance, DivideBinaryOperation can never return a
-      // DateTime or any string type, so even if we do not know its specific
-      // return type, we do know that there are some restrictions on what it
-      // may be. However, NullType is implicitly convertable to ANY Type, so
-      // such restrictions could be violated if a parent node in the expression
-      // tree converts a value of NullType to something that it shouldn't be.
-      if (left_is_nulltype || right_is_nulltype) {
-        const Type *fixed_result_type
-            = parse_binary_scalar.op().resultTypeForPartialArgumentTypes(
-                left_is_nulltype ? nullptr : &(left_argument->getValueType()),
-                right_is_nulltype ? nullptr : &(right_argument->getValueType()));
-        if (fixed_result_type != nullptr) {
-          return E::ScalarLiteral::Create(fixed_result_type->makeNullValue(),
-                                          *fixed_result_type);
-        }
-
-        if (type_hint != nullptr) {
-          const Type &nullable_type_hint = type_hint->getNullableVersion();
-          if (parse_binary_scalar.op().partialTypeSignatureIsPlausible(
-                  &nullable_type_hint,
-                  left_is_nulltype ? nullptr : &(left_argument->getValueType()),
-                  right_is_nulltype ? nullptr : &(right_argument->getValueType()))) {
-            return E::ScalarLiteral::Create(nullable_type_hint.makeNullValue(),
-                                            nullable_type_hint);
-          }
-        }
-
-        if (parse_binary_scalar.op().partialTypeSignatureIsPlausible(
-                nullptr,
-                left_is_nulltype ? nullptr : &(left_argument->getValueType()),
-                right_is_nulltype ? nullptr : &(right_argument->getValueType()))) {
-          const Type &null_type = TypeFactory::GetType(kNullType, true);
-          return E::ScalarLiteral::Create(null_type.makeNullValue(),
-                                          null_type);
-        }
-
-        // If nothing above worked, fall through to canApplyToTypes() below,
-        // which should fail.
-      }
-
-      if (!parse_binary_scalar.op().canApplyToTypes(left_argument->getValueType(),
-                                                    right_argument->getValueType())) {
-        THROW_SQL_ERROR_AT(&parse_binary_scalar)
-            << "Can not apply binary operation \"" << parse_binary_scalar.op().getName()
-            << "\" to arguments of types " << left_argument->getValueType().getName()
-            << " and " << right_argument->getValueType().getName();
-      }
-
-      return E::BinaryExpression::Create(parse_binary_scalar.op(),
-                                         left_argument,
-                                         right_argument);
-    }
     case ParseExpression::kScalarLiteral: {
       const ParseScalarLiteral &parse_literal_scalar =
           static_cast<const ParseScalarLiteral&>(parse_expression);
@@ -2408,57 +2308,6 @@ E::ScalarPtr Resolver::resolveExpression(
           type_hint,
           expression_resolution_info);
     }
-    case ParseExpression::kUnaryExpression: {
-      const ParseUnaryExpression &parse_unary_expr =
-          static_cast<const ParseUnaryExpression&>(parse_expression);
-
-      E::ScalarPtr argument = resolveExpression(
-          *parse_unary_expr.operand(),
-          parse_unary_expr.op().pushDownTypeHint(type_hint),
-          expression_resolution_info);
-
-      // If the argument is a NULL of unknown Type, try to resolve result Type
-      // of this UnaryExpression as follows:
-      //
-      //     1. If the UnaryExpression can only return one type, then the
-      //        result is a NULL of that type.
-      //     2. If there is a type hint for the UnaryExpression's result, and
-      //        it is possible for the UnaryExpression to return the hinted
-      //        type, then the result is a NULL of that type.
-      //     3. Otherwise, the result is a NULL of unknown type (i.e.
-      //        NullType).
-      //
-      // NOTE(chasseur): As with binary expressions above, step #3 does not
-      // always completely capture information about what types the NULL result
-      // can take on, since NullType is implicitly convertable to any Type.
-      if (argument->getValueType().getTypeID() == kNullType) {
-        const Type *fixed_result_type = parse_unary_expr.op().fixedNullableResultType();
-        if (fixed_result_type != nullptr) {
-          return E::ScalarLiteral::Create(fixed_result_type->makeNullValue(),
-                                          *fixed_result_type);
-        }
-
-        if (type_hint != nullptr) {
-          const Type &nullable_type_hint = type_hint->getNullableVersion();
-          if (parse_unary_expr.op().resultTypeIsPlausible(nullable_type_hint)) {
-            return E::ScalarLiteral::Create(nullable_type_hint.makeNullValue(),
-                                            nullable_type_hint);
-          }
-        }
-
-        const Type &null_type = TypeFactory::GetType(kNullType, true);
-        return E::ScalarLiteral::Create(null_type.makeNullValue(),
-                                        null_type);
-      }
-
-      if (!parse_unary_expr.op().canApplyToType(argument->getValueType())) {
-        THROW_SQL_ERROR_AT(&parse_unary_expr)
-            << "Can not apply unary operation \"" << parse_unary_expr.op().getName()
-            << "\" to argument of type " << argument->getValueType().getName();
-      }
-
-      return E::UnaryExpression::Create(parse_unary_expr.op(), argument);
-    }
     case ParseExpression::kFunctionCall: {
       return resolveFunctionCall(
           static_cast<const ParseFunctionCall&>(parse_expression),
@@ -2472,75 +2321,6 @@ E::ScalarPtr Resolver::resolveExpression(
           expression_resolution_info,
           true /* has_single_column */);
     }
-    case ParseExpression::kExtract: {
-      const ParseExtractFunction &parse_extract =
-          static_cast<const ParseExtractFunction&>(parse_expression);
-
-      const ParseString &extract_field = *parse_extract.extract_field();
-      const std::string lowered_unit = ToLower(extract_field.value());
-      DateExtractUnit extract_unit;
-      if (lowered_unit == "year") {
-        extract_unit = DateExtractUnit::kYear;
-      } else if (lowered_unit == "month") {
-        extract_unit = DateExtractUnit::kMonth;
-      } else if (lowered_unit == "day") {
-        extract_unit = DateExtractUnit::kDay;
-      } else if (lowered_unit == "hour") {
-        extract_unit = DateExtractUnit::kHour;
-      } else if (lowered_unit == "minute") {
-        extract_unit = DateExtractUnit::kMinute;
-      } else if (lowered_unit == "second") {
-        extract_unit = DateExtractUnit::kSecond;
-      } else {
-        THROW_SQL_ERROR_AT(&extract_field)
-            << "Invalid extract unit: " << extract_field.value();
-      }
-
-      const DateExtractOperation &op = DateExtractOperation::Instance(extract_unit);
-      const E::ScalarPtr argument = resolveExpression(
-          *parse_extract.date_expression(),
-          op.pushDownTypeHint(type_hint),
-          expression_resolution_info);
-
-      if (!op.canApplyToType(argument->getValueType())) {
-        THROW_SQL_ERROR_AT(parse_extract.date_expression())
-            << "Can not extract from argument of type: "
-            << argument->getValueType().getName();
-      }
-
-      return E::UnaryExpression::Create(op, argument);
-    }
-    case ParseExpression::kSubstring: {
-      const ParseSubstringFunction &parse_substring =
-          static_cast<const ParseSubstringFunction&>(parse_expression);
-
-      // Validate start position and substring length.
-      if (parse_substring.start_position() <= 0) {
-        THROW_SQL_ERROR_AT(&parse_expression)
-            << "The start position must be greater than 0";
-      }
-      if (parse_substring.length() <= 0) {
-        THROW_SQL_ERROR_AT(&parse_expression)
-            << "The substring length must be greater than 0";
-      }
-
-      // Convert 1-base position to 0-base position
-      const std::size_t zero_base_start_position = parse_substring.start_position() - 1;
-      const SubstringOperation &op =
-          SubstringOperation::Instance(zero_base_start_position,
-                                       parse_substring.length());
-
-      const E::ScalarPtr argument =
-          resolveExpression(*parse_substring.operand(),
-                            op.pushDownTypeHint(type_hint),
-                            expression_resolution_info);
-      if (!op.canApplyToType(argument->getValueType())) {
-        THROW_SQL_ERROR_AT(&parse_substring)
-            << "Can not apply substring function to argument of type "
-            << argument->getValueType().getName();
-      }
-      return E::UnaryExpression::Create(op, argument);
-    }
     default:
       LOG(FATAL) << "Unknown scalar type: "
                  << parse_expression.getExpressionType();
@@ -2634,13 +2414,15 @@ E::ScalarPtr Resolver::resolveSearchedCaseExpression(
   // Cast all the result expressions to the same type.
   for (E::ScalarPtr &conditional_result_expression : conditional_result_expressions) {
     if (conditional_result_expression->getValueType().getTypeID() != result_data_type->getTypeID()) {
-      conditional_result_expression =
-          E::Cast::Create(conditional_result_expression, *result_data_type);
+//      conditional_result_expression =
+//          E::Cast::Create(conditional_result_expression, *result_data_type);
+      LOG(FATAL) << "TODO: not handled";
     }
   }
   if (else_result_expression != nullptr
       && else_result_expression->getValueType().getTypeID() != result_data_type->getTypeID()) {
-    else_result_expression = E::Cast::Create(else_result_expression, *result_data_type);
+//    else_result_expression = E::Cast::Create(else_result_expression, *result_data_type);
+    LOG(FATAL) << "TODO: not handled";
   }
 
   if (else_result_expression == nullptr) {
@@ -2778,13 +2560,15 @@ E::ScalarPtr Resolver::resolveSimpleCaseExpression(
   // Cast all the result expressions to the same type.
   for (E::ScalarPtr &conditional_result_expression : conditional_result_expressions) {
     if (conditional_result_expression->getValueType().getTypeID() != result_data_type->getTypeID()) {
-      conditional_result_expression =
-          E::Cast::Create(conditional_result_expression, *result_data_type);
+//      conditional_result_expression =
+//          E::Cast::Create(conditional_result_expression, *result_data_type);
+      LOG(FATAL) << "TODO: not handled";
     }
   }
   if (else_result_expression != nullptr
       && else_result_expression->getValueType().getTypeID() != result_data_type->getTypeID()) {
-    else_result_expression = E::Cast::Create(else_result_expression, *result_data_type);
+//    else_result_expression = E::Cast::Create(else_result_expression, *result_data_type);
+    LOG(FATAL) << "TODO: not handled";
   }
 
   if (else_result_expression == nullptr) {
@@ -2800,6 +2584,85 @@ E::ScalarPtr Resolver::resolveSimpleCaseExpression(
                                *result_data_type);
 }
 
+E::ScalarPtr Resolver::resolveScalarFunction(
+    const ParseFunctionCall &parse_function_call,
+    const std::string &function_name,
+    const std::vector<E::ScalarPtr> &resolved_arguments,
+    ExpressionResolutionInfo *expression_resolution_info) {
+  const std::size_t arity = resolved_arguments.size();
+  std::vector<const Type*> argument_types;
+  std::size_t first_static_argument_position = 0;
+  for (std::size_t i = 0; i < arity; ++i) {
+    const E::ScalarPtr &argument = resolved_arguments[i];
+    if (argument->getExpressionType() != E::ExpressionType::kScalarLiteral) {
+      first_static_argument_position = i + 1;
+    }
+    argument_types.emplace_back(&argument->getValueType());
+  }
+
+  std::vector<TypedValue> static_arguments;
+  for (std::size_t i = first_static_argument_position; i < arity; ++i) {
+    static_arguments.emplace_back(
+        std::static_pointer_cast<const E::ScalarLiteral>(
+            resolved_arguments[i])->value());
+    DCHECK(static_arguments.back().getTypeID() == argument_types[i]->getTypeID());
+  }
+
+  std::shared_ptr<const std::vector<const Type*>> coerced_argument_types;
+  std::shared_ptr<const std::vector<TypedValue>> coerced_static_arguments;
+  std::string message;
+  const OperationSignaturePtr op_signature =
+      OperationFactory::Instance().resolveOperation(
+          function_name,
+          std::make_shared<const std::vector<const Type*>>(std::move(argument_types)),
+          std::make_shared<const std::vector<TypedValue>>(std::move(static_arguments)),
+          &coerced_argument_types,
+          &coerced_static_arguments,
+          &message);
+
+  if (op_signature == nullptr) {
+    if (message.empty()) {
+      THROW_SQL_ERROR_AT(&parse_function_call) << message;
+    } else {
+      THROW_SQL_ERROR_AT(&parse_function_call)
+          << "Cannot resolve scalar function " << function_name;
+    }
+  }
+
+  // TODO: add cast if neccessary.
+
+  const auto coerced_static_argument_types =
+      std::make_shared<const std::vector<const Type*>>(
+          coerced_argument_types->begin() + op_signature->getNonStaticArity(),
+          coerced_argument_types->end());
+
+  const OperationPtr operation =
+      OperationFactory::Instance().getOperation(op_signature);
+  switch (operation->getOperationSuperTypeID()) {
+    case Operation::kUnaryOperation:
+      return E::UnaryExpression::Create(
+          op_signature,
+          std::static_pointer_cast<const UnaryOperation>(operation),
+          resolved_arguments[0],
+          coerced_static_arguments,
+          coerced_static_argument_types);
+    case Operation::kBinaryOperation:
+      return E::BinaryExpression::Create(
+          op_signature,
+          std::static_pointer_cast<const BinaryOperation>(operation),
+          resolved_arguments[0],
+          resolved_arguments[1],
+          coerced_static_arguments,
+          coerced_static_argument_types);
+    default: {
+      const auto operation_id =
+         static_cast<std::underlying_type_t<Operation::OperationSuperTypeID>>(
+             operation->getOperationSuperTypeID());
+      LOG(FATAL) << "Unknown opeation super type id: " << operation_id;
+    }
+  }
+}
+
 // TODO(chasseur): For now this only handles resolving aggregate functions. In
 // the future it should be extended to resolve scalar functions as well.
 // TODO(Shixuan): This will handle resolving window aggregation function as well,
@@ -2820,8 +2683,7 @@ E::ScalarPtr Resolver::resolveFunctionCall(
   }
 
   std::vector<E::ScalarPtr> resolved_arguments;
-  const PtrList<ParseExpression> *unresolved_arguments =
-      parse_function_call.arguments();
+  const PtrList<ParseExpression> *unresolved_arguments = parse_function_call.arguments();
   // The first aggregate function and window aggregate function in the arguments.
   const ParseTreeNode *first_aggregate_function = nullptr;
   const ParseTreeNode *first_window_aggregate_function = nullptr;
@@ -2829,7 +2691,7 @@ E::ScalarPtr Resolver::resolveFunctionCall(
     for (const ParseExpression &unresolved_argument : *unresolved_arguments) {
       ExpressionResolutionInfo expr_resolution_info(
           *expression_resolution_info);
-      resolved_arguments.push_back(
+      resolved_arguments.emplace_back(
           resolveExpression(unresolved_argument,
                             nullptr,  // No Type hint.
                             &expr_resolution_info));
@@ -2848,6 +2710,17 @@ E::ScalarPtr Resolver::resolveFunctionCall(
     }
   }
 
+  if (OperationFactory::Instance().hasOperation(function_name,
+                                                resolved_arguments.size())) {
+    E::ScalarPtr scalar = resolveScalarFunction(parse_function_call,
+                                                function_name,
+                                                resolved_arguments,
+                                                expression_resolution_info);
+    expression_resolution_info->parse_aggregate_expression = first_aggregate_function;
+    expression_resolution_info->parse_window_aggregate_expression = first_window_aggregate_function;
+    return scalar;
+  }
+
   if (count_star && !resolved_arguments.empty()) {
     THROW_SQL_ERROR_AT(&parse_function_call)
         << "COUNT aggregate has both star (*) and non-star arguments.";

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/resolver/Resolver.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.hpp b/query_optimizer/resolver/Resolver.hpp
index 1ae565a..bae4839 100644
--- a/query_optimizer/resolver/Resolver.hpp
+++ b/query_optimizer/resolver/Resolver.hpp
@@ -471,9 +471,6 @@ class Resolver {
    * @brief Resolves a function call. For a non-scalar function, the returned
    *        expression is an AttributeReference to the actual resolved expression.
    *
-   * @note This currently only handles resolving aggregate functions and window
-   *       aggregate functions.
-   *
    * @param parse_function_call The function call to be resolved.
    * @param expression_resolution_info Resolution info that contains the name
    *                                   resolver and info to be updated after
@@ -484,6 +481,12 @@ class Resolver {
       const ParseFunctionCall &parse_function_call,
       ExpressionResolutionInfo *expression_resolution_info);
 
+  expressions::ScalarPtr resolveScalarFunction(
+      const ParseFunctionCall &parse_function_call,
+      const std::string &function_name,
+      const std::vector<expressions::ScalarPtr> &resolved_arguments,
+      ExpressionResolutionInfo *expression_resolution_info);
+
   /**
    * @brief Resolves a window aggregate function.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 36e5959..96f066c 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -258,9 +258,9 @@ target_link_libraries(quickstep_queryoptimizer_rules_ReuseAggregateExpressions
                       quickstep_queryoptimizer_physical_Selection
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_rules_BottomUpRule
+                      quickstep_types_operations_OperationFactory
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_utility_HashError
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_Rule

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/rules/ReuseAggregateExpressions.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReuseAggregateExpressions.cpp b/query_optimizer/rules/ReuseAggregateExpressions.cpp
index a7c62c6..d63715a 100644
--- a/query_optimizer/rules/ReuseAggregateExpressions.cpp
+++ b/query_optimizer/rules/ReuseAggregateExpressions.cpp
@@ -44,9 +44,9 @@
 #include "query_optimizer/physical/PhysicalType.hpp"
 #include "query_optimizer/physical/Selection.hpp"
 #include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "types/operations/OperationFactory.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 #include "utility/HashError.hpp"
 
 #include "gflags/gflags.h"
@@ -317,12 +317,19 @@ P::PhysicalPtr ReuseAggregateExpressions::applyToNode(
           }
 
           // Obtain AVG by evaluating SUM/COUNT in Selection.
-          const BinaryOperation &divide_op =
-              BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide);
+          const E::AttributeReferencePtr &count_attr = agg_attrs[agg_ref->second_ref];
+          const std::vector<TypeID> operand_tids =
+              { sum_attr->getValueType().getTypeID(), count_attr->getValueType().getTypeID() };
+          const OperationSignaturePtr op_sig =
+              OperationSignature::Create("/", operand_tids, 0);
+
+          const BinaryOperationPtr &divide_op =
+              OperationFactory::Instance().getBinaryOperation(op_sig);
           const E::BinaryExpressionPtr avg_expr =
-              E::BinaryExpression::Create(divide_op,
+              E::BinaryExpression::Create(op_sig,
+                                          divide_op,
                                           sum_attr,
-                                          agg_attrs[agg_ref->second_ref]);
+                                          count_attr);
           new_select_exprs.emplace_back(
               E::Alias::Create(agg_expr->id(),
                                avg_expr,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/rules/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/tests/CMakeLists.txt b/query_optimizer/rules/tests/CMakeLists.txt
index 0d913e2..0db39c3 100644
--- a/query_optimizer/rules/tests/CMakeLists.txt
+++ b/query_optimizer/rules/tests/CMakeLists.txt
@@ -84,8 +84,6 @@ target_link_libraries(quickstep_queryoptimizer_rules_tests
                       quickstep_queryoptimizer_rules_tests_PhysicalRuleTest
                       quickstep_queryoptimizer_rules_tests_RuleTest
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/strategy/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/strategy/tests/CMakeLists.txt b/query_optimizer/strategy/tests/CMakeLists.txt
index 97675f0..f502583 100644
--- a/query_optimizer/strategy/tests/CMakeLists.txt
+++ b/query_optimizer/strategy/tests/CMakeLists.txt
@@ -73,8 +73,6 @@ target_link_libraries(quickstep_queryoptimizer_strategy_tests
                       quickstep_queryoptimizer_strategy_tests_StrategyTest
                       quickstep_types_TypeID
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/CMakeLists.txt b/query_optimizer/tests/CMakeLists.txt
index 5ef1d0a..bde0495 100644
--- a/query_optimizer/tests/CMakeLists.txt
+++ b/query_optimizer/tests/CMakeLists.txt
@@ -50,8 +50,6 @@ target_link_libraries(quickstep_queryoptimizer_tests_OptimizerTest
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/query_optimizer/tests/OptimizerTest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/OptimizerTest.cpp b/query_optimizer/tests/OptimizerTest.cpp
index 7eb7a11..4c94f2d 100644
--- a/query_optimizer/tests/OptimizerTest.cpp
+++ b/query_optimizer/tests/OptimizerTest.cpp
@@ -43,8 +43,6 @@
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index a6a3cd0..388e7f2 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -681,8 +681,6 @@ target_link_libraries(AggregationOperator_unittest
                       quickstep_types_LongType
                       quickstep_types_TypedValue
                       quickstep_types_containers_Tuple
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
                       quickstep_utility_Macros

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index 4296ba0..4a656f4 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -825,7 +825,7 @@ target_link_libraries(quickstep_storage_PackedPayloadHashTable
                       quickstep_utility_HashPair
                       quickstep_utility_Macros
                       quickstep_utility_PrimeNumber
-                      quickstep_utility_TemplateUtil)
+                      quickstep_utility_meta_Dispatchers)
 target_link_libraries(quickstep_storage_PartitionedHashTablePool
                       glog
                       quickstep_storage_HashTableBase
@@ -869,9 +869,8 @@ target_link_libraries(quickstep_storage_SMAIndexSubBlock
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
+                      quickstep_types_operations_OperationFactory
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/storage/PackedPayloadHashTable.cpp
----------------------------------------------------------------------
diff --git a/storage/PackedPayloadHashTable.cpp b/storage/PackedPayloadHashTable.cpp
index 1df20d0..77f512b 100644
--- a/storage/PackedPayloadHashTable.cpp
+++ b/storage/PackedPayloadHashTable.cpp
@@ -40,7 +40,7 @@
 #include "utility/Alignment.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PrimeNumber.hpp"
-#include "utility/TemplateUtil.hpp"
+#include "utility/meta/Dispatchers.hpp"
 
 #include "glog/logging.h"
 
@@ -246,7 +246,7 @@ bool PackedPayloadHashTable::upsertValueAccessorCompositeKey(
     derived_accessor->beginIterationVirtual();
   }
 
-  return InvokeOnBools(
+  return meta::InvokeOnBools(
       handles_.empty(),
       !all_keys_inline_,
       [&](auto key_only,  // NOLINT(build/c++11)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/storage/SMAIndexSubBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/SMAIndexSubBlock.cpp b/storage/SMAIndexSubBlock.cpp
index 3b3b879..0724138 100644
--- a/storage/SMAIndexSubBlock.cpp
+++ b/storage/SMAIndexSubBlock.cpp
@@ -44,9 +44,8 @@
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
+#include "types/operations/OperationFactory.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
@@ -359,9 +358,10 @@ SMAIndexSubBlock::SMAIndexSubBlock(const TupleStorageSubBlock &tuple_store,
       TypeID attr_sum_typeid = sma_internal::getTypeForSum(attr_typeid);
       if (add_operations_.elementIsNullAt(attr_typeid)) {
         add_operations_.replaceElement(attr_typeid,
-          BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-              .makeUncheckedBinaryOperatorForTypes(TypeFactory::GetType(attr_typeid),
-                                                   TypeFactory::GetType(attr_sum_typeid)));
+            OperationFactory::Instance().getBinaryOperation(
+                "+", {attr_typeid, attr_sum_typeid})
+                    ->makeUncheckedBinaryOperator(TypeFactory::GetType(attr_typeid),
+                                                  TypeFactory::GetType(attr_sum_typeid)));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/AsciiStringSuperType.hpp
----------------------------------------------------------------------
diff --git a/types/AsciiStringSuperType.hpp b/types/AsciiStringSuperType.hpp
new file mode 100644
index 0000000..959c288
--- /dev/null
+++ b/types/AsciiStringSuperType.hpp
@@ -0,0 +1,78 @@
+/**
+ * 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_TYPES_ASCII_STRING_SUPER_TYPE_HPP_
+#define QUICKSTEP_TYPES_ASCII_STRING_SUPER_TYPE_HPP_
+
+#include <cstddef>
+
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypeSynthesizer.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+/**
+ * @brief A superclass for ASCII string types.
+ **/
+template <TypeID type_id>
+class AsciiStringSuperType : public TypeSynthesizer<type_id> {
+ public:
+  bool isCoercibleFrom(const Type &original_type) const override {
+    if (original_type.isNullable() && !this->nullable_) {
+      return false;
+    }
+    return (original_type.getSuperTypeID() == Type::kAsciiString)
+           || (original_type.getTypeID() == kNullType);
+  }
+
+  /**
+   * @brief Get the character-length of this string type.
+   *
+   * @return The maximum length of a string of this type.
+   **/
+  inline std::size_t getStringLength() const {
+    return length_;
+  }
+
+ protected:
+  AsciiStringSuperType(const bool nullable,
+                       const std::size_t minimum_byte_length,
+                       const std::size_t maximum_byte_length,
+                       const std::size_t string_length)
+      : TypeSynthesizer<type_id>(
+            nullable, minimum_byte_length, maximum_byte_length, string_length),
+        length_(string_length) {
+  }
+
+  const std::size_t length_;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(AsciiStringSuperType);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_ASCII_STRING_SUPER_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/BoolType.cpp
----------------------------------------------------------------------
diff --git a/types/BoolType.cpp b/types/BoolType.cpp
new file mode 100644
index 0000000..83cf060
--- /dev/null
+++ b/types/BoolType.cpp
@@ -0,0 +1,63 @@
+/**
+ * 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 "types/BoolType.hpp"
+
+#include <cstdio>
+#include <string>
+
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "utility/StringUtil.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+std::string BoolType::printValueToString(const TypedValue &value) const {
+  DCHECK(!value.isNull());
+
+  return value.getLiteral<bool>() ? "true" : "false";
+}
+
+void BoolType::printValueToFile(const TypedValue &value,
+                                FILE *file,
+                                const int padding) const {
+  DCHECK(!value.isNull());
+
+  std::fprintf(file,
+               "%*s",
+               static_cast<int>(padding),
+               value.getLiteral<bool>() ? "true" : "false");
+}
+
+bool BoolType::parseValueFromString(const std::string &value_string,
+                                    TypedValue *value) const {
+  const std::string lo_value = ToLower(value_string);
+  if (lo_value == "true") {
+    *value = TypedValue(true);
+    return true;
+  } else if (lo_value == "false") {
+    *value = TypedValue(false);
+    return true;
+  }
+  return false;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/BoolType.hpp
----------------------------------------------------------------------
diff --git a/types/BoolType.hpp b/types/BoolType.hpp
new file mode 100644
index 0000000..f149e76
--- /dev/null
+++ b/types/BoolType.hpp
@@ -0,0 +1,73 @@
+/**
+ * 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_TYPES_BOOL_TYPE_HPP_
+#define QUICKSTEP_TYPES_BOOL_TYPE_HPP_
+
+#include <array>
+#include <cstdio>
+#include <limits>
+#include <string>
+
+#include "types/NumericSuperType.hpp"
+#include "types/TypeID.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class Type;
+class TypedValue;
+
+/** \addtogroup Types
+ *  @{
+ */
+
+/**
+ * @brief A type representing a 8-bit bool.
+ **/
+class BoolType : public NumericSuperType<kBool> {
+ public:
+  int getPrintWidth() const override {
+    // "true" or "false"
+    return 5;
+  }
+
+  std::string printValueToString(const TypedValue &value) const override;
+
+  void printValueToFile(const TypedValue &value,
+                        FILE *file,
+                        const int padding = 0) const override;
+
+  bool parseValueFromString(const std::string &value_string,
+                            TypedValue *value) const override;
+
+ private:
+  explicit BoolType(const bool nullable)
+      : NumericSuperType<kBool>(nullable) {}
+
+  template <typename, bool> friend class TypeInstance;
+
+  DISALLOW_COPY_AND_ASSIGN(BoolType);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_BOOL_TYPE_HPP_



[28/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index 5cd7426..1301447 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -138,22 +138,16 @@ typedef struct YYLTYPE {
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
-#include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
-#include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationFactory.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
 #include "utility/PtrList.hpp"
 #include "utility/PtrVector.hpp"
 
 // Needed for Bison 2.6 and higher, which do not automatically provide this typedef.
 typedef void* yyscan_t;
 
-#line 157 "SqlParser_gen.cpp" /* yacc.c:339  */
+#line 151 "SqlParser_gen.cpp" /* yacc.c:339  */
 
 # ifndef YY_NULLPTR
 #  if defined __cplusplus && 201103L <= __cplusplus
@@ -224,103 +218,104 @@ extern int quickstep_yydebug;
     TOKEN_CSB_TREE = 291,
     TOKEN_BY = 292,
     TOKEN_CASE = 293,
-    TOKEN_CHARACTER = 294,
-    TOKEN_CHECK = 295,
-    TOKEN_COLUMN = 296,
-    TOKEN_CONSTRAINT = 297,
-    TOKEN_COPY = 298,
-    TOKEN_CREATE = 299,
-    TOKEN_CURRENT = 300,
-    TOKEN_DATE = 301,
-    TOKEN_DATETIME = 302,
-    TOKEN_DAY = 303,
-    TOKEN_DECIMAL = 304,
-    TOKEN_DEFAULT = 305,
-    TOKEN_DELETE = 306,
-    TOKEN_DELIMITER = 307,
-    TOKEN_DESC = 308,
-    TOKEN_DISTINCT = 309,
-    TOKEN_DOUBLE = 310,
-    TOKEN_DROP = 311,
-    TOKEN_ELSE = 312,
-    TOKEN_END = 313,
-    TOKEN_ESCAPE_STRINGS = 314,
-    TOKEN_EXISTS = 315,
-    TOKEN_EXTRACT = 316,
-    TOKEN_FALSE = 317,
-    TOKEN_FIRST = 318,
-    TOKEN_FLOAT = 319,
-    TOKEN_FOLLOWING = 320,
-    TOKEN_FOR = 321,
-    TOKEN_FOREIGN = 322,
-    TOKEN_FROM = 323,
-    TOKEN_FULL = 324,
-    TOKEN_GROUP = 325,
-    TOKEN_HASH = 326,
-    TOKEN_HAVING = 327,
-    TOKEN_HOUR = 328,
-    TOKEN_IN = 329,
-    TOKEN_INDEX = 330,
-    TOKEN_INNER = 331,
-    TOKEN_INSERT = 332,
-    TOKEN_INTEGER = 333,
-    TOKEN_INTERVAL = 334,
-    TOKEN_INTO = 335,
-    TOKEN_JOIN = 336,
-    TOKEN_KEY = 337,
-    TOKEN_LAST = 338,
-    TOKEN_LEFT = 339,
-    TOKEN_LIMIT = 340,
-    TOKEN_LONG = 341,
-    TOKEN_MINUTE = 342,
-    TOKEN_MONTH = 343,
-    TOKEN_NULL = 344,
-    TOKEN_NULLS = 345,
-    TOKEN_OFF = 346,
-    TOKEN_ON = 347,
-    TOKEN_ORDER = 348,
-    TOKEN_OUTER = 349,
-    TOKEN_OVER = 350,
-    TOKEN_PARTITION = 351,
-    TOKEN_PARTITIONS = 352,
-    TOKEN_PERCENT = 353,
-    TOKEN_PRECEDING = 354,
-    TOKEN_PRIMARY = 355,
-    TOKEN_PRIORITY = 356,
-    TOKEN_QUIT = 357,
-    TOKEN_RANGE = 358,
-    TOKEN_REAL = 359,
-    TOKEN_REFERENCES = 360,
-    TOKEN_RIGHT = 361,
-    TOKEN_ROW = 362,
-    TOKEN_ROW_DELIMITER = 363,
-    TOKEN_ROWS = 364,
-    TOKEN_SECOND = 365,
-    TOKEN_SELECT = 366,
-    TOKEN_SET = 367,
-    TOKEN_SMA = 368,
-    TOKEN_SMALLINT = 369,
-    TOKEN_SUBSTRING = 370,
-    TOKEN_TABLE = 371,
-    TOKEN_THEN = 372,
-    TOKEN_TIME = 373,
-    TOKEN_TIMESTAMP = 374,
-    TOKEN_TRUE = 375,
-    TOKEN_TUPLESAMPLE = 376,
-    TOKEN_UNBOUNDED = 377,
-    TOKEN_UNIQUE = 378,
-    TOKEN_UPDATE = 379,
-    TOKEN_USING = 380,
-    TOKEN_VALUES = 381,
-    TOKEN_VARCHAR = 382,
-    TOKEN_WHEN = 383,
-    TOKEN_WHERE = 384,
-    TOKEN_WINDOW = 385,
-    TOKEN_WITH = 386,
-    TOKEN_YEAR = 387,
-    TOKEN_YEARMONTH = 388,
-    TOKEN_EOF = 389,
-    TOKEN_LEX_ERROR = 390
+    TOKEN_CAST = 294,
+    TOKEN_CHARACTER = 295,
+    TOKEN_CHECK = 296,
+    TOKEN_COLUMN = 297,
+    TOKEN_CONSTRAINT = 298,
+    TOKEN_COPY = 299,
+    TOKEN_CREATE = 300,
+    TOKEN_CURRENT = 301,
+    TOKEN_DATE = 302,
+    TOKEN_DATETIME = 303,
+    TOKEN_DAY = 304,
+    TOKEN_DECIMAL = 305,
+    TOKEN_DEFAULT = 306,
+    TOKEN_DELETE = 307,
+    TOKEN_DELIMITER = 308,
+    TOKEN_DESC = 309,
+    TOKEN_DISTINCT = 310,
+    TOKEN_DOUBLE = 311,
+    TOKEN_DROP = 312,
+    TOKEN_ELSE = 313,
+    TOKEN_END = 314,
+    TOKEN_ESCAPE_STRINGS = 315,
+    TOKEN_EXISTS = 316,
+    TOKEN_EXTRACT = 317,
+    TOKEN_FALSE = 318,
+    TOKEN_FIRST = 319,
+    TOKEN_FLOAT = 320,
+    TOKEN_FOLLOWING = 321,
+    TOKEN_FOR = 322,
+    TOKEN_FOREIGN = 323,
+    TOKEN_FROM = 324,
+    TOKEN_FULL = 325,
+    TOKEN_GROUP = 326,
+    TOKEN_HASH = 327,
+    TOKEN_HAVING = 328,
+    TOKEN_HOUR = 329,
+    TOKEN_IN = 330,
+    TOKEN_INDEX = 331,
+    TOKEN_INNER = 332,
+    TOKEN_INSERT = 333,
+    TOKEN_INTEGER = 334,
+    TOKEN_INTERVAL = 335,
+    TOKEN_INTO = 336,
+    TOKEN_JOIN = 337,
+    TOKEN_KEY = 338,
+    TOKEN_LAST = 339,
+    TOKEN_LEFT = 340,
+    TOKEN_LIMIT = 341,
+    TOKEN_LONG = 342,
+    TOKEN_MINUTE = 343,
+    TOKEN_MONTH = 344,
+    TOKEN_NULL = 345,
+    TOKEN_NULLS = 346,
+    TOKEN_OFF = 347,
+    TOKEN_ON = 348,
+    TOKEN_ORDER = 349,
+    TOKEN_OUTER = 350,
+    TOKEN_OVER = 351,
+    TOKEN_PARTITION = 352,
+    TOKEN_PARTITIONS = 353,
+    TOKEN_PERCENT = 354,
+    TOKEN_PRECEDING = 355,
+    TOKEN_PRIMARY = 356,
+    TOKEN_PRIORITY = 357,
+    TOKEN_QUIT = 358,
+    TOKEN_RANGE = 359,
+    TOKEN_REAL = 360,
+    TOKEN_REFERENCES = 361,
+    TOKEN_RIGHT = 362,
+    TOKEN_ROW = 363,
+    TOKEN_ROW_DELIMITER = 364,
+    TOKEN_ROWS = 365,
+    TOKEN_SECOND = 366,
+    TOKEN_SELECT = 367,
+    TOKEN_SET = 368,
+    TOKEN_SMA = 369,
+    TOKEN_SMALLINT = 370,
+    TOKEN_SUBSTRING = 371,
+    TOKEN_TABLE = 372,
+    TOKEN_THEN = 373,
+    TOKEN_TIME = 374,
+    TOKEN_TIMESTAMP = 375,
+    TOKEN_TRUE = 376,
+    TOKEN_TUPLESAMPLE = 377,
+    TOKEN_UNBOUNDED = 378,
+    TOKEN_UNIQUE = 379,
+    TOKEN_UPDATE = 380,
+    TOKEN_USING = 381,
+    TOKEN_VALUES = 382,
+    TOKEN_VARCHAR = 383,
+    TOKEN_WHEN = 384,
+    TOKEN_WHERE = 385,
+    TOKEN_WINDOW = 386,
+    TOKEN_WITH = 387,
+    TOKEN_YEAR = 388,
+    TOKEN_YEARMONTH = 389,
+    TOKEN_EOF = 390,
+    TOKEN_LEX_ERROR = 391
   };
 #endif
 
@@ -329,7 +324,7 @@ extern int quickstep_yydebug;
 
 union YYSTYPE
 {
-#line 121 "../SqlParser.ypp" /* yacc.c:355  */
+#line 115 "../SqlParser.ypp" /* yacc.c:355  */
 
   quickstep::ParseString *string_value_;
 
@@ -402,8 +397,8 @@ union YYSTYPE
   quickstep::ParseStatementQuit *quit_statement_;
 
   const quickstep::Comparison *comparison_;
-  const quickstep::UnaryOperation *unary_operation_;
-  const quickstep::BinaryOperation *binary_operation_;
+  quickstep::ParseString *unary_operation_;
+  quickstep::ParseString *binary_operation_;
 
   quickstep::ParseFunctionCall *function_call_;
   quickstep::PtrList<quickstep::ParseExpression> *expression_list_;
@@ -431,7 +426,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 435 "SqlParser_gen.cpp" /* yacc.c:355  */
+#line 430 "SqlParser_gen.cpp" /* yacc.c:355  */
 };
 
 typedef union YYSTYPE YYSTYPE;
@@ -460,13 +455,13 @@ int quickstep_yyparse (yyscan_t yyscanner, quickstep::ParseStatement **parsedSta
 #endif /* !YY_QUICKSTEP_YY_SQLPARSER_GEN_HPP_INCLUDED  */
 
 /* Copy the second part of user declarations.  */
-#line 223 "../SqlParser.ypp" /* yacc.c:358  */
+#line 217 "../SqlParser.ypp" /* yacc.c:358  */
 
 /* This header needs YYSTYPE, which is defined by the %union directive above */
 #include "SqlLexer_gen.hpp"
 void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string &feature);
 
-#line 470 "SqlParser_gen.cpp" /* yacc.c:358  */
+#line 465 "SqlParser_gen.cpp" /* yacc.c:358  */
 
 #ifdef short
 # undef short
@@ -710,21 +705,21 @@ union yyalloc
 /* YYFINAL -- State number of the termination state.  */
 #define YYFINAL  49
 /* YYLAST -- Last index in YYTABLE.  */
-#define YYLAST   1207
+#define YYLAST   1432
 
 /* YYNTOKENS -- Number of terminals.  */
-#define YYNTOKENS  147
+#define YYNTOKENS  148
 /* YYNNTS -- Number of nonterminals.  */
-#define YYNNTS  109
+#define YYNNTS  110
 /* YYNRULES -- Number of rules.  */
-#define YYNRULES  293
+#define YYNRULES  296
 /* YYNSTATES -- Number of states.  */
-#define YYNSTATES  543
+#define YYNSTATES  552
 
 /* YYTRANSLATE[YYX] -- Symbol number corresponding to YYX as returned
    by yylex, with out-of-bounds checking.  */
 #define YYUNDEFTOK  2
-#define YYMAXUTOK   390
+#define YYMAXUTOK   391
 
 #define YYTRANSLATE(YYX)                                                \
   ((unsigned int) (YYX) <= YYMAXUTOK ? yytranslate[YYX] : YYUNDEFTOK)
@@ -734,11 +729,11 @@ union yyalloc
 static const yytype_uint8 yytranslate[] =
 {
        0,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-     142,     2,     2,     2,     2,     2,     2,     2,     2,     2,
+     143,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,   146,     2,     2,
-     143,   144,    23,    21,   145,    22,    27,    24,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,     2,     2,   141,
+       2,     2,     2,     2,     2,     2,     2,   147,     2,     2,
+     144,   145,    23,    21,   146,    22,    27,    24,     2,     2,
+       2,     2,     2,     2,     2,     2,     2,     2,     2,   142,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
@@ -772,43 +767,43 @@ static const yytype_uint8 yytranslate[] =
      110,   111,   112,   113,   114,   115,   116,   117,   118,   119,
      120,   121,   122,   123,   124,   125,   126,   127,   128,   129,
      130,   131,   132,   133,   134,   135,   136,   137,   138,   139,
-     140
+     140,   141
 };
 
 #if YYDEBUG
   /* YYRLINE[YYN] -- Source line where rule number YYN was defined.  */
 static const yytype_uint16 yyrline[] =
 {
-       0,   639,   639,   643,   647,   651,   655,   658,   665,   668,
-     671,   674,   677,   680,   683,   686,   689,   692,   698,   704,
-     711,   717,   724,   733,   738,   747,   752,   757,   761,   767,
-     772,   775,   778,   783,   786,   789,   792,   795,   798,   801,
-     804,   807,   810,   822,   825,   828,   846,   866,   869,   872,
-     877,   882,   888,   894,   903,   907,   913,   916,   921,   926,
-     931,   938,   945,   949,   955,   958,   963,   966,   971,   974,
-     979,   982,  1001,  1004,  1009,  1013,  1019,  1022,  1025,  1030,
-    1033,  1040,  1045,  1056,  1061,  1065,  1069,  1075,  1078,  1084,
-    1092,  1095,  1098,  1104,  1109,  1112,  1117,  1121,  1125,  1129,
-    1135,  1140,  1145,  1149,  1155,  1164,  1167,  1172,  1175,  1194,
-    1199,  1203,  1209,  1215,  1224,  1229,  1237,  1243,  1249,  1252,
-    1255,  1260,  1263,  1268,  1272,  1278,  1281,  1284,  1289,  1294,
-    1299,  1302,  1305,  1310,  1313,  1316,  1319,  1322,  1325,  1328,
-    1331,  1336,  1339,  1344,  1348,  1352,  1355,  1359,  1362,  1367,
-    1370,  1375,  1378,  1383,  1387,  1393,  1396,  1401,  1404,  1409,
-    1412,  1417,  1420,  1439,  1442,  1447,  1451,  1457,  1463,  1468,
-    1471,  1476,  1479,  1484,  1487,  1492,  1495,  1500,  1501,  1504,
-    1509,  1510,  1513,  1518,  1522,  1528,  1535,  1538,  1541,  1546,
-    1549,  1552,  1558,  1561,  1566,  1571,  1580,  1585,  1594,  1599,
-    1602,  1607,  1610,  1615,  1621,  1627,  1630,  1633,  1636,  1639,
-    1642,  1648,  1657,  1660,  1665,  1668,  1673,  1676,  1681,  1684,
-    1687,  1690,  1694,  1698,  1701,  1704,  1707,  1710,  1715,  1719,
-    1723,  1726,  1731,  1736,  1740,  1746,  1749,  1754,  1758,  1764,
-    1769,  1773,  1779,  1784,  1787,  1792,  1796,  1802,  1805,  1808,
-    1811,  1823,  1827,  1846,  1859,  1874,  1877,  1880,  1883,  1886,
-    1889,  1894,  1898,  1904,  1907,  1912,  1916,  1923,  1926,  1929,
-    1932,  1935,  1938,  1941,  1944,  1947,  1950,  1955,  1966,  1969,
-    1974,  1977,  1980,  1986,  1990,  1996,  1999,  2007,  2010,  2013,
-    2016,  2022,  2027,  2032
+       0,   633,   633,   637,   641,   645,   649,   652,   659,   662,
+     665,   668,   671,   674,   677,   680,   683,   686,   692,   698,
+     705,   711,   718,   727,   732,   741,   746,   751,   755,   761,
+     766,   769,   772,   777,   780,   783,   786,   789,   792,   795,
+     798,   801,   804,   816,   819,   822,   840,   860,   863,   866,
+     871,   876,   882,   888,   897,   901,   907,   910,   915,   920,
+     925,   932,   939,   943,   949,   952,   957,   960,   965,   968,
+     973,   976,   995,   998,  1003,  1007,  1013,  1016,  1019,  1024,
+    1027,  1034,  1039,  1050,  1055,  1059,  1063,  1069,  1072,  1078,
+    1086,  1089,  1092,  1098,  1103,  1106,  1111,  1115,  1119,  1123,
+    1129,  1134,  1139,  1143,  1149,  1158,  1161,  1166,  1169,  1188,
+    1193,  1197,  1203,  1209,  1218,  1223,  1231,  1237,  1243,  1246,
+    1249,  1254,  1257,  1262,  1266,  1272,  1275,  1278,  1283,  1288,
+    1293,  1296,  1299,  1304,  1307,  1310,  1313,  1316,  1319,  1322,
+    1325,  1330,  1333,  1338,  1342,  1346,  1349,  1353,  1356,  1361,
+    1364,  1369,  1372,  1377,  1381,  1387,  1390,  1395,  1398,  1403,
+    1406,  1411,  1414,  1433,  1436,  1441,  1445,  1451,  1457,  1462,
+    1465,  1470,  1473,  1478,  1481,  1486,  1489,  1494,  1495,  1498,
+    1503,  1504,  1507,  1512,  1516,  1522,  1529,  1532,  1535,  1540,
+    1543,  1546,  1552,  1555,  1560,  1565,  1574,  1579,  1588,  1593,
+    1596,  1601,  1604,  1609,  1615,  1621,  1624,  1627,  1630,  1633,
+    1636,  1642,  1651,  1657,  1662,  1668,  1673,  1678,  1683,  1686,
+    1689,  1692,  1696,  1700,  1703,  1706,  1709,  1712,  1715,  1720,
+    1724,  1728,  1731,  1736,  1750,  1761,  1772,  1780,  1791,  1794,
+    1799,  1803,  1809,  1814,  1818,  1824,  1829,  1832,  1837,  1841,
+    1847,  1850,  1853,  1856,  1868,  1872,  1891,  1904,  1919,  1922,
+    1925,  1928,  1931,  1934,  1939,  1943,  1949,  1952,  1957,  1961,
+    1968,  1971,  1974,  1977,  1980,  1983,  1986,  1989,  1992,  1995,
+    2000,  2011,  2014,  2019,  2022,  2025,  2031,  2035,  2041,  2044,
+    2052,  2055,  2058,  2061,  2067,  2072,  2077
 };
 #endif
 
@@ -827,20 +822,20 @@ static const char *const yytname[] =
   "TOKEN_ALTER", "TOKEN_AS", "TOKEN_ASC", "TOKEN_BIGINT", "TOKEN_BIT",
   "TOKEN_BITWEAVING", "TOKEN_BLOCKPROPERTIES", "TOKEN_BLOCKSAMPLE",
   "TOKEN_BLOOM_FILTER", "TOKEN_CSB_TREE", "TOKEN_BY", "TOKEN_CASE",
-  "TOKEN_CHARACTER", "TOKEN_CHECK", "TOKEN_COLUMN", "TOKEN_CONSTRAINT",
-  "TOKEN_COPY", "TOKEN_CREATE", "TOKEN_CURRENT", "TOKEN_DATE",
-  "TOKEN_DATETIME", "TOKEN_DAY", "TOKEN_DECIMAL", "TOKEN_DEFAULT",
-  "TOKEN_DELETE", "TOKEN_DELIMITER", "TOKEN_DESC", "TOKEN_DISTINCT",
-  "TOKEN_DOUBLE", "TOKEN_DROP", "TOKEN_ELSE", "TOKEN_END",
-  "TOKEN_ESCAPE_STRINGS", "TOKEN_EXISTS", "TOKEN_EXTRACT", "TOKEN_FALSE",
-  "TOKEN_FIRST", "TOKEN_FLOAT", "TOKEN_FOLLOWING", "TOKEN_FOR",
-  "TOKEN_FOREIGN", "TOKEN_FROM", "TOKEN_FULL", "TOKEN_GROUP", "TOKEN_HASH",
-  "TOKEN_HAVING", "TOKEN_HOUR", "TOKEN_IN", "TOKEN_INDEX", "TOKEN_INNER",
-  "TOKEN_INSERT", "TOKEN_INTEGER", "TOKEN_INTERVAL", "TOKEN_INTO",
-  "TOKEN_JOIN", "TOKEN_KEY", "TOKEN_LAST", "TOKEN_LEFT", "TOKEN_LIMIT",
-  "TOKEN_LONG", "TOKEN_MINUTE", "TOKEN_MONTH", "TOKEN_NULL", "TOKEN_NULLS",
-  "TOKEN_OFF", "TOKEN_ON", "TOKEN_ORDER", "TOKEN_OUTER", "TOKEN_OVER",
-  "TOKEN_PARTITION", "TOKEN_PARTITIONS", "TOKEN_PERCENT",
+  "TOKEN_CAST", "TOKEN_CHARACTER", "TOKEN_CHECK", "TOKEN_COLUMN",
+  "TOKEN_CONSTRAINT", "TOKEN_COPY", "TOKEN_CREATE", "TOKEN_CURRENT",
+  "TOKEN_DATE", "TOKEN_DATETIME", "TOKEN_DAY", "TOKEN_DECIMAL",
+  "TOKEN_DEFAULT", "TOKEN_DELETE", "TOKEN_DELIMITER", "TOKEN_DESC",
+  "TOKEN_DISTINCT", "TOKEN_DOUBLE", "TOKEN_DROP", "TOKEN_ELSE",
+  "TOKEN_END", "TOKEN_ESCAPE_STRINGS", "TOKEN_EXISTS", "TOKEN_EXTRACT",
+  "TOKEN_FALSE", "TOKEN_FIRST", "TOKEN_FLOAT", "TOKEN_FOLLOWING",
+  "TOKEN_FOR", "TOKEN_FOREIGN", "TOKEN_FROM", "TOKEN_FULL", "TOKEN_GROUP",
+  "TOKEN_HASH", "TOKEN_HAVING", "TOKEN_HOUR", "TOKEN_IN", "TOKEN_INDEX",
+  "TOKEN_INNER", "TOKEN_INSERT", "TOKEN_INTEGER", "TOKEN_INTERVAL",
+  "TOKEN_INTO", "TOKEN_JOIN", "TOKEN_KEY", "TOKEN_LAST", "TOKEN_LEFT",
+  "TOKEN_LIMIT", "TOKEN_LONG", "TOKEN_MINUTE", "TOKEN_MONTH", "TOKEN_NULL",
+  "TOKEN_NULLS", "TOKEN_OFF", "TOKEN_ON", "TOKEN_ORDER", "TOKEN_OUTER",
+  "TOKEN_OVER", "TOKEN_PARTITION", "TOKEN_PARTITIONS", "TOKEN_PERCENT",
   "TOKEN_PRECEDING", "TOKEN_PRIMARY", "TOKEN_PRIORITY", "TOKEN_QUIT",
   "TOKEN_RANGE", "TOKEN_REAL", "TOKEN_REFERENCES", "TOKEN_RIGHT",
   "TOKEN_ROW", "TOKEN_ROW_DELIMITER", "TOKEN_ROWS", "TOKEN_SECOND",
@@ -878,7 +873,7 @@ static const char *const yytname[] =
   "opt_order_direction", "opt_nulls_first", "opt_where_clause",
   "where_clause", "or_expression", "and_expression", "not_expression",
   "predicate_expression_base", "add_expression", "multiply_expression",
-  "unary_expression", "expression_base", "function_call",
+  "unary_expression", "expression_base", "function_call", "cast_function",
   "extract_function", "substr_function", "case_expression",
   "simple_when_clause_list", "simple_when_clause",
   "searched_when_clause_list", "searched_when_clause", "opt_else_clause",
@@ -909,14 +904,14 @@ static const yytype_uint16 yytoknum[] =
      360,   361,   362,   363,   364,   365,   366,   367,   368,   369,
      370,   371,   372,   373,   374,   375,   376,   377,   378,   379,
      380,   381,   382,   383,   384,   385,   386,   387,   388,   389,
-     390,    59,    10,    40,    41,    44,    37
+     390,   391,    59,    10,    40,    41,    44,    37
 };
 # endif
 
-#define YYPACT_NINF -244
+#define YYPACT_NINF -253
 
 #define yypact_value_is_default(Yystate) \
-  (!!((Yystate) == (-244)))
+  (!!((Yystate) == (-253)))
 
 #define YYTABLE_NINF -134
 
@@ -927,61 +922,62 @@ static const yytype_uint16 yytoknum[] =
      STATE-NUM.  */
 static const yytype_int16 yypact[] =
 {
-     807,  -244,  -244,   -55,   203,   -26,    -2,    -6,    43,  -244,
-      86,   203,   203,  -244,   136,   125,  -244,  -244,  -244,  -244,
-    -244,  -244,  -244,  -244,  -244,  -244,   -22,    -3,   128,  -244,
-     -78,   182,   203,  -244,  -244,   130,   203,   203,   203,   203,
-     203,  -244,  -244,   633,   107,    89,  -244,   209,   120,  -244,
-    -244,  -244,   197,    -3,    86,   161,  -244,   137,  -244,  -244,
-    -244,    27,   265,   180,   141,   170,  -244,   -25,  -244,  -244,
-     301,   313,  -244,  -244,  -244,   660,   188,  -244,   249,  -244,
-    -244,   206,  -244,  -244,   357,  -244,  -244,  -244,  -244,   221,
-    -244,  -244,   222,   282,   783,   366,   299,   231,  -244,  -244,
-     272,     7,  -244,  -244,   277,  -244,  -244,  -244,  -244,  -244,
-     933,    -5,   203,   203,   237,   203,   203,  -244,   137,   374,
-    -244,   233,   239,   246,   203,   203,   551,  -244,  -244,   244,
-     203,  -244,  -244,  -244,   551,    47,    40,  -244,   376,  -244,
-     153,   153,   960,   381,  -244,     8,    24,  -244,    23,   170,
-     960,  -244,  -244,   203,   960,  -244,  -244,  -244,  -244,   960,
-      36,   313,  -244,   203,   363,    94,  -244,   378,  -244,   137,
-    -244,   144,  -244,   137,   128,  -244,   203,   -21,   203,   203,
-     248,  -244,   250,  -244,   147,  1069,   810,   237,   469,   387,
-     388,  -244,  -244,   329,   380,  1042,   152,    45,   960,    48,
-    -244,   960,  -244,   333,   257,  -244,  -244,  -244,  -244,  -244,
-    -244,   330,  -244,   168,   260,  -244,  -244,    19,    66,   146,
-    -244,   264,    66,    51,   335,  -244,  -244,     7,  -244,   304,
-    -244,  -244,   267,   960,  -244,   279,   158,   203,  -244,   960,
-    -244,   203,  -244,  -244,   270,   331,   334,   273,  -244,  -244,
-    -244,   165,   203,   290,   -21,   203,  -244,    77,  -244,  -244,
-       2,    64,   551,   551,   193,  -244,  -244,  -244,  -244,  -244,
-    -244,  -244,  -244,   960,   281,   960,    31,  -244,   162,   294,
-     960,    71,  -244,   364,   279,  -244,  -244,   960,   419,  -244,
-     110,   203,  -244,  -244,   332,  -244,   336,   337,   342,    23,
-    -244,   423,   426,    66,   392,   361,   397,   296,   343,  -244,
-     166,  -244,   960,  -244,   279,  -244,   551,   300,   302,   203,
-     437,   119,   169,  -244,   172,   421,    60,  -244,   305,   316,
-    -244,   355,   310,  1042,  -244,   368,   203,  -244,  -244,    77,
-    -244,  -244,   388,  -244,  -244,  -244,   960,   315,    95,   783,
-    -244,   279,   362,  -244,  -244,  1042,   317,   279,   960,  -244,
-      26,   -15,  -244,  -244,  -244,  -244,  -244,    23,   146,   356,
-     358,  -244,   960,   551,   365,   960,  -244,   420,   111,  -244,
-     279,     4,   203,   203,   174,  -244,  -244,  -244,  -244,  -244,
-    -244,  -244,   201,  -244,   203,  -244,  -244,  -244,  -244,   321,
-     -21,   427,   369,  -244,   551,  -244,  -244,   323,  -244,   238,
-     783,  -244,   960,   179,  -244,  -244,  1042,   279,  -244,   460,
-    -244,   384,  -244,  -244,   326,   387,   435,   394,   326,   960,
-    -244,  -244,  -244,   459,  -244,   181,   183,  -244,   480,   119,
-    -244,   203,  -244,  -244,   345,   447,  -244,     6,   203,   960,
-     185,   279,  -244,   192,   349,   551,   960,   489,   367,   353,
-    -244,   240,    12,   389,  -244,  -244,  -244,   208,  -244,  -244,
-    -244,  -244,    11,   203,    22,  -244,   359,   279,  -244,  -244,
-    -244,   387,   353,  -244,   203,  -244,   367,  -244,   960,  -244,
-    -244,   405,   404,   398,   407,   500,   203,  -244,   203,  -244,
-    -244,   203,  -244,   210,  -244,  -244,   371,  -244,   482,  -244,
-    -244,   129,  -244,  -244,  -244,  -244,    13,   373,  -244,   212,
-    -244,   203,   375,  -244,  -244,   449,   410,   454,  -244,   203,
-    -244,   214,   304,  -244,  -244,  -244,   216,   424,   383,  -244,
-     521,  -244,  -244
+      86,  -253,  -253,   -32,   242,   -40,    45,    69,    96,  -253,
+      42,   242,   242,  -253,   132,   139,  -253,  -253,  -253,  -253,
+    -253,  -253,  -253,  -253,  -253,  -253,   -26,     9,   191,  -253,
+     131,   224,   242,  -253,  -253,   168,   242,   242,   242,   242,
+     242,  -253,  -253,   723,   144,   111,  -253,   240,   136,  -253,
+    -253,  -253,   190,     9,    42,   177,  -253,   170,  -253,  -253,
+    -253,    41,   285,   205,   163,   179,  -253,    75,  -253,  -253,
+     310,   312,  -253,  -253,  -253,   816,   181,   188,  -253,   236,
+    -253,  -253,   192,  -253,  -253,   361,  -253,  -253,  -253,  -253,
+     225,  -253,  -253,   234,   297,   909,   379,   316,   252,  -253,
+    -253,   289,     6,  -253,  -253,   302,  -253,  -253,  -253,  -253,
+    -253,  -253,  1095,   -13,   242,   242,   260,   242,   242,  -253,
+     170,   398,  -253,    46,   153,   269,   242,   242,   630,  -253,
+    -253,   264,   242,  -253,  -253,  -253,   630,    24,    -7,  -253,
+    1188,   402,  -253,   145,   145,  1188,   404,  -253,    -3,    27,
+    -253,    17,   179,  1188,  -253,  -253,   242,  1188,  -253,  -253,
+    -253,  -253,  1188,    21,   312,  -253,   242,   412,   -50,  -253,
+     401,  -253,   170,  -253,    60,  -253,   170,   191,  -253,   242,
+     162,   242,   242,   270,  -253,   271,  -253,    65,  1293,  1002,
+     260,   537,   405,   411,  -253,  -253,   498,   403,  1241,   156,
+      34,  1188,     3,  -253,  1188,  -253,   357,    62,   277,  -253,
+    -253,  -253,  -253,  -253,  -253,   350,  -253,    32,   280,  -253,
+    -253,    16,    82,   138,  -253,   281,    82,    39,   352,  -253,
+    -253,     6,  -253,   327,  -253,  -253,   286,  1188,  -253,   284,
+     167,   242,  -253,  1188,  -253,   242,  -253,  -253,   288,   348,
+     351,   294,  -253,  -253,  -253,    15,   242,   309,   162,   242,
+    -253,   134,  -253,  -253,     4,   346,   630,   630,   214,  -253,
+    -253,  -253,  -253,  -253,  -253,  -253,  -253,  1188,   298,  1188,
+      89,  -253,   182,   311,  1188,    47,  -253,   377,   284,  -253,
+     263,  -253,  1188,   437,  -253,   -11,   242,  -253,  -253,   345,
+    -253,   349,   354,   363,    17,  -253,   444,   445,    82,   416,
+     381,   418,   317,   362,  -253,   184,  -253,  1188,  -253,   284,
+    -253,   630,   319,   322,   242,   463,   146,   194,  -253,   196,
+     443,    68,  -253,   325,   337,  -253,   380,   332,  1241,  -253,
+     392,   242,  -253,  -253,   134,  -253,  -253,   411,  -253,  -253,
+    -253,  1188,   339,   176,   909,  -253,   284,   386,  -253,  -253,
+    1241,   340,   284,  1188,  -253,   341,   342,    29,   -29,  -253,
+    -253,  -253,  -253,  -253,    17,   138,   384,   388,  -253,  1188,
+     630,   391,  1188,  -253,   451,    87,  -253,   284,     5,   242,
+     242,   198,  -253,  -253,  -253,  -253,  -253,  -253,  -253,   212,
+    -253,   242,  -253,  -253,  -253,  -253,   355,   162,   447,   393,
+    -253,   630,  -253,  -253,   356,  -253,   244,   909,  -253,  1188,
+     200,  -253,  -253,  1241,   284,  -253,  -253,  -253,   487,  -253,
+     400,  -253,  -253,   359,   405,   460,   427,   359,  1188,  -253,
+    -253,  -253,   502,  -253,   204,   207,  -253,   518,   146,  -253,
+     242,  -253,  -253,   383,   482,  -253,    23,   242,  1188,   226,
+     284,  -253,   229,   385,   630,  1188,   521,   389,   394,  -253,
+     275,    10,   420,  -253,  -253,  -253,   231,  -253,  -253,  -253,
+    -253,    12,   242,   127,  -253,   390,   284,  -253,  -253,  -253,
+     405,   394,  -253,   242,  -253,   389,  -253,  1188,  -253,  -253,
+     433,   434,   425,   441,   539,   242,  -253,   242,  -253,  -253,
+     242,  -253,   235,  -253,  -253,   406,  -253,   516,  -253,  -253,
+     -14,  -253,  -253,  -253,  -253,    53,   408,  -253,   247,  -253,
+     242,   409,  -253,  -253,   483,   442,   489,  -253,   242,  -253,
+     249,   327,  -253,  -253,  -253,   255,   458,   417,  -253,   556,
+    -253,  -253
 };
 
   /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM.
@@ -989,93 +985,94 @@ static const yytype_int16 yypact[] =
      means the default is an error.  */
 static const yytype_uint16 yydefact[] =
 {
-       0,     6,   293,     0,     0,     0,     0,     0,     0,    18,
+       0,     6,   296,     0,     0,     0,     0,     0,     0,    18,
      118,     0,     0,     7,     0,     0,    15,     8,    10,    11,
       13,    14,     9,    17,    12,    16,     0,   107,   114,   116,
-       0,   291,     0,   285,   286,     0,     0,     0,     0,     0,
+       0,   294,     0,   288,   289,     0,     0,     0,     0,     0,
        0,   119,   120,     0,     0,   109,   110,     0,   151,     1,
        3,     2,     0,   107,   118,     0,   105,     0,     5,     4,
-     292,     0,     0,     0,     0,   192,    25,     0,   251,   248,
-       0,   277,   121,    40,    29,     0,     0,    30,    31,    34,
-      36,     0,    37,    39,     0,    41,   247,    35,    38,     0,
-      32,    33,     0,     0,     0,     0,     0,   122,   123,   227,
-     127,   213,   215,   217,   220,   223,   224,   225,   219,   218,
-       0,   263,     0,     0,     0,     0,     0,   106,     0,     0,
-     115,     0,     0,    94,     0,     0,     0,   101,   193,     0,
-       0,    91,   249,   250,     0,     0,   243,   240,     0,    43,
-       0,   252,     0,     0,    44,     0,     0,   254,     0,   192,
-       0,   278,   279,     0,     0,   126,   281,   282,   280,     0,
-       0,     0,   216,     0,     0,   192,   103,     0,   111,     0,
-     112,     0,   283,     0,   113,   108,     0,     0,     0,     0,
-       0,    93,    66,    27,     0,     0,     0,     0,     0,   194,
-     196,   198,   200,     0,   218,     0,     0,     0,     0,   243,
-     237,     0,   241,     0,     0,   257,   258,   259,   256,   260,
-     255,     0,   253,     0,     0,   129,   226,     0,     0,   153,
-     142,   128,   147,   130,   155,   124,   125,   212,   214,   169,
-     221,   264,     0,     0,   228,   245,     0,     0,   100,     0,
-     152,     0,    92,    19,     0,     0,     0,     0,    20,    21,
-      22,     0,     0,     0,    64,     0,    42,    56,   199,   207,
-       0,     0,     0,     0,     0,   267,   269,   270,   271,   272,
-     268,   273,   275,     0,     0,     0,     0,   261,     0,     0,
-       0,     0,   238,     0,   244,   236,    45,     0,     0,    46,
-     133,     0,   143,   149,   139,   134,   135,   137,     0,     0,
-     146,     0,     0,   145,     0,   157,     0,     0,   171,   229,
-       0,   230,     0,   102,   104,   284,     0,     0,     0,     0,
-       0,     0,     0,   265,     0,   263,     0,    63,    65,    68,
-      28,     0,     0,     0,    47,     0,     0,    49,    55,    57,
-      26,   206,   195,   197,   274,   276,     0,     0,     0,     0,
-     208,   205,     0,   204,    90,     0,     0,   242,     0,   235,
-       0,     0,   148,   150,   140,   136,   138,     0,   154,     0,
-       0,   144,     0,     0,   159,     0,   222,     0,   173,   231,
-     246,     0,     0,     0,     0,    96,   289,   290,   288,   287,
-      97,    95,     0,    67,     0,    83,    84,    85,    86,    87,
-       0,     0,    70,    48,     0,    51,    50,     0,    54,     0,
-       0,   210,     0,     0,   203,   262,     0,   239,   232,     0,
-     233,     0,   131,   132,   156,   158,     0,   161,   170,     0,
-     176,   175,   168,     0,    61,     0,     0,    58,     0,     0,
-     266,     0,    24,    62,     0,     0,    23,     0,     0,     0,
-       0,   201,   209,     0,     0,     0,     0,     0,   163,   172,
-     183,   186,     0,     0,    59,    98,    99,     0,    74,    76,
-      77,    78,     0,     0,     0,    52,     0,   202,   211,    89,
-     234,   141,   160,   162,     0,   117,   164,   165,     0,   187,
-     188,   189,     0,     0,     0,     0,     0,    88,     0,    82,
-      80,     0,    79,     0,    72,    73,     0,    53,     0,   166,
-     184,     0,   185,   177,   179,   178,     0,     0,    75,     0,
-      69,     0,     0,   190,   191,     0,     0,     0,   174,     0,
-      81,     0,   169,   180,   182,   181,     0,     0,     0,    60,
-       0,   167,    71
+     295,     0,     0,     0,     0,   192,    25,     0,   254,   251,
+       0,   280,   121,    40,    29,     0,     0,     0,    30,    31,
+      34,    36,     0,    37,    39,     0,    41,   250,    35,    38,
+       0,    32,    33,     0,     0,     0,     0,     0,   122,   123,
+     228,   127,   213,   215,   217,   220,   223,   224,   225,   226,
+     219,   218,     0,   266,     0,     0,     0,     0,     0,   106,
+       0,     0,   115,     0,     0,    94,     0,     0,     0,   101,
+     193,     0,     0,    91,   252,   253,     0,     0,   246,   243,
+       0,     0,    43,     0,   255,     0,     0,    44,     0,     0,
+     257,     0,   192,     0,   281,   282,     0,     0,   126,   284,
+     285,   283,     0,     0,     0,   216,     0,     0,   192,   103,
+       0,   111,     0,   112,     0,   286,     0,   113,   108,     0,
+       0,     0,     0,     0,    93,    66,    27,     0,     0,     0,
+       0,     0,   194,   196,   198,   200,     0,   218,     0,     0,
+       0,     0,   246,   240,     0,   244,     0,     0,     0,   260,
+     261,   262,   259,   263,   258,     0,   256,     0,     0,   129,
+     227,     0,     0,   153,   142,   128,   147,   130,   155,   124,
+     125,   212,   214,   169,   221,   267,     0,     0,   229,   248,
+       0,     0,   100,     0,   152,     0,    92,    19,     0,     0,
+       0,     0,    20,    21,    22,     0,     0,     0,    64,     0,
+      42,    56,   199,   207,     0,     0,     0,     0,     0,   270,
+     272,   273,   274,   275,   271,   276,   278,     0,     0,     0,
+       0,   264,     0,     0,     0,     0,   241,     0,   247,   239,
+       0,    45,     0,     0,    46,   133,     0,   143,   149,   139,
+     134,   135,   137,     0,     0,   146,     0,     0,   145,     0,
+     157,     0,     0,   171,   230,     0,   231,     0,   102,   104,
+     287,     0,     0,     0,     0,     0,     0,     0,   268,     0,
+     266,     0,    63,    65,    68,    28,     0,     0,     0,    47,
+       0,     0,    49,    55,    57,    26,   206,   195,   197,   277,
+     279,     0,     0,     0,     0,   208,   205,     0,   204,    90,
+       0,     0,   245,     0,   238,     0,     0,     0,     0,   148,
+     150,   140,   136,   138,     0,   154,     0,     0,   144,     0,
+       0,   159,     0,   222,     0,   173,   232,   249,     0,     0,
+       0,     0,    96,   292,   293,   291,   290,    97,    95,     0,
+      67,     0,    83,    84,    85,    86,    87,     0,     0,    70,
+      48,     0,    51,    50,     0,    54,     0,     0,   210,     0,
+       0,   203,   265,     0,   242,   233,   234,   235,     0,   236,
+       0,   131,   132,   156,   158,     0,   161,   170,     0,   176,
+     175,   168,     0,    61,     0,     0,    58,     0,     0,   269,
+       0,    24,    62,     0,     0,    23,     0,     0,     0,     0,
+     201,   209,     0,     0,     0,     0,     0,   163,   172,   183,
+     186,     0,     0,    59,    98,    99,     0,    74,    76,    77,
+      78,     0,     0,     0,    52,     0,   202,   211,    89,   237,
+     141,   160,   162,     0,   117,   164,   165,     0,   187,   188,
+     189,     0,     0,     0,     0,     0,    88,     0,    82,    80,
+       0,    79,     0,    72,    73,     0,    53,     0,   166,   184,
+       0,   185,   177,   179,   178,     0,     0,    75,     0,    69,
+       0,     0,   190,   191,     0,     0,     0,   174,     0,    81,
+       0,   169,   180,   182,   181,     0,     0,     0,    60,     0,
+     167,    71
 };
 
   /* YYPGOTO[NTERM-NUM].  */
 static const yytype_int16 yypgoto[] =
 {
-    -244,  -244,  -244,  -244,  -244,  -244,  -244,  -244,  -132,  -244,
-     346,   191,  -244,  -244,  -243,  -244,  -244,  -244,  -244,  -244,
-    -244,    59,    35,  -244,  -244,  -244,  -244,  -244,  -244,  -244,
-    -244,  -244,  -244,  -244,  -244,   303,  -244,   483,  -244,  -244,
-     428,   200,   425,    -8,   485,  -244,  -244,   395,  -244,   -98,
-    -244,  -244,  -165,   175,  -179,   -10,  -244,  -244,  -244,  -244,
-    -244,  -244,  -244,    58,     5,  -244,  -244,  -244,  -244,  -244,
-    -244,    90,    61,  -244,  -244,    39,  -244,  -133,   285,   287,
-     379,   -37,   400,   403,   438,  -144,  -244,  -244,  -244,  -244,
-     352,  -244,   430,   360,  -212,  -186,   429,   148,  -121,  -244,
-    -244,  -244,  -244,  -244,  -127,    -4,   132,  -244,  -244
+    -253,  -253,  -253,  -253,  -253,  -253,  -253,  -253,  -135,  -253,
+    -111,   220,  -253,  -253,  -252,  -253,  -253,  -253,  -253,  -253,
+    -253,    83,    59,  -253,  -253,  -253,  -253,  -253,  -253,  -253,
+    -253,  -253,  -253,  -253,  -253,   326,  -253,   515,  -253,  -253,
+     454,    74,   450,    -5,   517,  -253,  -253,   421,  -253,   -92,
+    -253,  -253,  -184,   201,  -187,    -8,  -253,  -253,  -253,  -253,
+    -253,  -253,  -253,    81,    36,  -253,  -253,  -253,  -253,  -253,
+    -253,   114,    88,  -253,  -253,   -76,  -253,  -133,   318,   320,
+     397,   -28,   426,   429,   476,  -140,  -253,  -253,  -253,  -253,
+    -253,   395,  -253,   455,   399,  -228,  -193,   452,   171,  -126,
+    -253,  -253,  -253,  -253,  -253,  -131,    -4,   147,  -253,  -253
 };
 
   /* YYDEFGOTO[NTERM-NUM].  */
 static const yytype_int16 yydefgoto[] =
 {
-      -1,    14,    15,    16,    17,    18,    19,    20,   183,   184,
-      95,   338,   339,   340,   248,   328,   329,   253,   402,   446,
-     506,   467,   468,   469,   470,   471,   399,   442,    21,    22,
-     181,   322,    23,    24,   165,   166,    25,    56,    26,    45,
-      46,   145,    28,    29,    43,    96,    97,    98,   149,    99,
-     303,   298,   219,   220,   292,   293,   221,   305,   374,   427,
-     458,   485,   486,   487,   307,   308,   378,   432,   433,   495,
-     528,   459,   460,   491,   512,   127,   128,   189,   190,   191,
-     192,   193,   101,   102,   103,   104,   105,   106,   107,   199,
-     200,   136,   137,   203,   236,   108,   211,   278,   109,   324,
-     275,   110,   154,   159,   171,   111,   390,    30,    31
+      -1,    14,    15,    16,    17,    18,    19,    20,   186,   187,
+      96,   343,   344,   345,   252,   333,   334,   257,   409,   455,
+     515,   476,   477,   478,   479,   480,   406,   451,    21,    22,
+     184,   327,    23,    24,   168,   169,    25,    56,    26,    45,
+      46,   148,    28,    29,    43,    97,    98,    99,   152,   100,
+     308,   303,   223,   224,   297,   298,   225,   310,   381,   436,
+     467,   494,   495,   496,   312,   313,   385,   441,   442,   504,
+     537,   468,   469,   500,   521,   129,   130,   192,   193,   194,
+     195,   196,   102,   103,   104,   105,   106,   107,   108,   109,
+     202,   203,   138,   139,   206,   240,   110,   215,   282,   111,
+     329,   279,   112,   157,   162,   174,   113,   397,    30,    31
 };
 
   /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM.  If
@@ -1083,348 +1080,395 @@ static const yytype_int16 yydefgoto[] =
      number is the opposite.  If YYTABLE_NINF, syntax error.  */
 static const yytype_int16 yytable[] =
 {
-      35,   197,    47,   196,   222,   194,   100,    44,    48,   277,
-     262,   327,   262,   194,   262,    33,   170,    34,   499,   492,
-     525,   310,   163,    33,   244,    34,    54,    33,    61,    34,
-     156,   157,    63,    64,    65,    66,    67,    54,   135,   500,
-      33,   352,    34,   300,   243,   151,   152,   151,   152,   120,
-     218,   245,   290,   262,    36,   260,   419,   146,   121,   131,
-      52,    58,   493,   526,    59,   194,    32,   194,   151,   152,
-      33,    38,    34,   222,   264,   265,   266,   267,   268,   269,
-     270,   271,   272,   273,   246,   151,   152,   331,   122,   259,
-     301,    10,   151,   152,    10,    37,   155,   395,   504,   291,
-     396,   397,   201,    47,   412,   213,   129,   247,   167,    48,
-     201,   172,   173,   100,    41,    39,   151,   152,   130,   218,
-     182,   185,   332,   330,   371,   353,   172,   235,    40,   420,
-     505,   323,   333,    55,   368,    10,    49,   413,   164,   494,
-     527,   194,   194,   274,   223,    42,   341,   405,   434,   226,
-     475,   261,   215,   158,   501,   222,   230,   443,    57,   231,
-     424,   281,   217,   428,   284,   242,   217,   280,   216,   415,
-     418,   334,   185,   134,   249,   250,   350,   302,   398,   229,
-     198,   198,   335,   381,   294,    60,   386,   336,   224,   151,
-     152,   295,   384,   358,   164,   194,   235,   523,   450,   296,
-      27,   218,   314,    62,   238,   337,   205,    33,   216,    34,
-     344,   345,   346,   223,    48,   387,   388,   524,    48,   430,
-     294,   297,   320,   222,   112,   431,    53,   295,   126,   321,
-     277,   206,  -133,   167,   113,   296,   348,   315,   351,   237,
-     425,   288,   114,   357,   389,   207,   208,   449,   325,   411,
-     360,   185,   194,    10,   362,   435,   436,   297,   438,   151,
-     152,   151,   152,   115,    50,   439,    51,   119,   209,   218,
-     123,   447,   347,   440,   489,   380,    33,   124,    34,   176,
-     177,   363,   116,   194,   125,   178,   179,    48,   240,   241,
-     210,   254,   255,   151,   152,   223,   279,   241,   490,    48,
-     151,   152,   311,   312,   126,   153,   354,   355,   132,   409,
-     379,   312,   235,   391,   392,   172,   393,   394,   437,   241,
-     133,   417,   481,   452,   312,   463,   241,   464,   241,   478,
-     312,   138,   407,   139,   194,   235,   479,   355,   235,   264,
-     265,   266,   267,   268,   269,   270,   271,   272,   273,   140,
-     151,   152,   497,   498,   520,   498,   530,   241,   537,   241,
-     539,   241,   141,   223,   142,   143,   144,    33,    68,    34,
-      69,   147,   148,   235,   519,   451,   150,   160,   172,   172,
-     169,   175,   180,   204,    70,    71,   232,   195,   214,   239,
-     325,   251,   461,   252,   531,   262,   285,   263,    73,    74,
-     276,   286,   536,   287,   289,   306,    75,    76,   274,   299,
-     304,   309,   477,   316,    77,    78,   319,    79,   317,   461,
-     326,   318,   233,    80,   349,   356,   361,   359,   367,    81,
-     369,   364,    82,   370,   372,   365,   366,   472,   373,   375,
-     376,   377,   385,   382,   476,   383,    83,    84,   163,   403,
-     400,   461,   401,   404,    85,   406,   414,    86,   410,   422,
-     416,   423,   429,   426,   441,   444,   448,   454,   502,   472,
-     445,   312,    87,    33,    68,    34,    69,   456,   462,   186,
-     508,   455,    88,    89,   457,   465,    90,    91,   473,   474,
-      70,    71,   517,   480,   472,    92,   483,   172,   488,   496,
-     511,    93,   484,   507,    73,    74,    94,   234,   513,   516,
-     514,   515,    75,    76,   521,   522,   529,   172,   532,   533,
-      77,    78,   534,    79,   535,   172,   540,   541,   542,    80,
-     408,   257,   503,   518,   187,    81,   117,   538,    82,   118,
-     313,   168,   421,   174,   509,   225,   482,   342,   162,   510,
-     343,   282,    83,    84,   227,    33,    68,    34,    69,   283,
-      85,   186,   228,    86,   453,   258,   202,     0,     0,     0,
-     212,   466,    70,    71,     0,     0,     0,     0,    87,     0,
-       0,     0,     0,     0,     0,    10,    73,    74,    88,    89,
-       0,     0,    90,    91,    75,    76,     0,     0,     0,     0,
-       0,    92,    77,    78,     0,    79,     0,    93,     0,     0,
-       0,    80,   188,     0,     0,     0,   187,    81,     0,     0,
-      82,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    83,    84,     0,    33,    68,    34,
-      69,     0,    85,     0,     0,    86,     0,     0,     0,     0,
-       0,     0,     0,     0,    70,    71,    72,     0,     0,     0,
-      87,     0,     0,     0,    33,    68,    34,    69,    73,    74,
-      88,    89,     0,     0,    90,    91,    75,    76,     0,     0,
-       0,    70,    71,    92,    77,    78,     0,    79,     0,    93,
-       0,     0,     0,    80,   188,    73,    74,     0,     0,    81,
-       0,     0,    82,    75,    76,     0,     0,     0,     0,     0,
-       0,    77,    78,     0,    79,     0,    83,    84,     0,     0,
-      80,     0,     0,     0,    85,     0,    81,    86,     0,    82,
+      35,   199,   197,   200,    47,   281,   332,    44,    48,   315,
+     197,   226,   266,   266,   166,   101,    33,   501,    34,   508,
+      33,    33,    34,    34,   173,    33,    54,    34,    61,   159,
+     160,   266,    63,    64,    65,    66,    67,   295,    54,   305,
+     509,    36,   266,   428,   247,   154,   155,   137,   154,   155,
+     154,   155,   122,   154,   155,   532,   204,    52,   264,   222,
+     534,   502,   133,   197,   299,   197,   204,   149,   154,   155,
+      41,   300,   123,   325,    27,   533,   228,   261,   306,   301,
+     326,   226,    37,   154,   155,   128,    33,     1,    34,     2,
+      32,    10,   242,   179,   180,   290,   241,   158,   263,   357,
+      53,   302,    42,   124,   535,   402,   293,    47,   403,   404,
+     170,    48,   207,   175,   176,   296,   429,   217,     3,    38,
+     375,   378,   185,   188,   335,   101,   420,   136,   175,   222,
+     328,   167,    49,    10,   369,     4,     5,   201,   503,   239,
+     197,   197,   219,     6,   336,   412,    55,   227,     7,   346,
+     443,   433,   230,   161,   437,   452,   510,   284,   201,   234,
+     221,   221,   235,   265,   226,   233,   307,   422,   484,     8,
+     363,   246,   220,   285,   427,   188,   288,   253,   254,   365,
+     337,   536,    40,   167,   358,   419,   355,   405,   388,   459,
+     338,    39,    10,   391,     9,   197,   439,   154,   155,   209,
+     181,   182,   440,    10,   513,   244,   245,   131,   248,   239,
+     258,   259,   222,   299,   393,   319,    11,   227,    48,   132,
+     300,    57,    48,    12,   210,  -133,    13,    60,   301,   339,
+     281,   349,   350,   351,   226,   249,   514,   170,   211,   212,
+     340,   320,    62,   394,   395,   341,    33,   434,    34,   353,
+     302,   356,   330,   458,   197,   188,   362,   115,   444,   445,
+     418,   213,   114,   342,   367,   154,   155,    33,   250,    34,
+     447,    58,   396,   116,    59,   449,   118,   448,   456,    50,
+     117,    51,   222,   214,   121,   197,   366,    10,   370,   387,
+     125,   251,    48,    33,   352,    34,   154,   155,    73,    74,
+     227,   283,   245,   126,    48,   154,   155,   127,    77,   498,
+     154,   155,   316,   317,   128,    78,    79,   134,    80,   135,
+     175,   142,   156,   416,    81,   140,   239,   359,   360,   386,
+     317,   490,   141,    83,   499,   424,   143,   414,   197,   398,
+     399,   400,   401,   446,   245,   461,   317,    84,   260,   472,
+     245,   239,   473,   245,   239,    86,   268,   269,   270,   271,
+     272,   273,   274,   275,   276,   277,   144,   154,   155,   145,
+     227,   487,   317,    88,   488,   360,   506,   507,   146,   528,
+     529,   507,   147,    89,   150,   175,   175,    91,    92,   239,
+     151,   460,   539,   245,   546,   245,    93,   330,   153,   540,
+     548,   245,    94,   163,   172,   178,   183,   545,   198,   208,
+     470,   218,   243,   266,   255,   256,    33,    68,    34,    69,
+     267,   289,   291,   280,   292,   294,   278,   304,   309,   311,
+     486,   314,   321,    70,    71,   236,   322,   470,   324,   323,
+     331,   364,   354,   361,   368,   371,   481,    73,    74,   372,
+     374,   376,   377,   485,   373,    75,    76,    77,   379,   380,
+     382,   384,   383,   389,    78,    79,   390,    80,   392,   470,
+     166,   407,   237,    81,   408,   410,   411,   511,   481,    82,
+     413,   421,    83,   417,   423,   453,   425,   426,   431,   517,
+     435,   220,   432,   438,   463,   454,    84,    85,   464,   450,
+     457,   526,   465,   481,    86,   317,   175,    87,   268,   269,
+     270,   271,   272,   273,   274,   275,   276,   277,   466,   154,
+     155,   471,    88,   474,   483,   493,   175,   482,   492,   520,
+     489,   505,    89,    90,   175,   516,    91,    92,   523,   522,
+     497,    33,    68,    34,    69,    93,   524,   189,   525,   531,
+     530,    94,   538,   541,   542,   543,    95,   238,    70,    71,
+     544,   549,   550,   551,   415,   512,   527,   318,   119,   171,
+     177,   120,    73,    74,   229,   430,   518,   547,   278,   491,
+      75,    76,    77,   231,   347,   519,   262,   348,   165,    78,
+      79,   232,    80,   205,   462,   475,   216,   286,    81,     0,
+       0,   287,     0,   190,    82,     0,     0,    83,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,    87,    83,    84,     0,     0,     0,     0,     0,
-       0,    85,    88,    89,    86,     0,    90,    91,     0,     0,
-       0,     0,     0,     0,     0,    92,     0,     0,     0,    87,
-       0,    93,     0,     0,     0,     0,    94,     0,     0,    88,
-      89,     0,     0,    90,    91,     0,     0,    33,    68,    34,
-      69,     0,    92,   134,     0,     0,     0,     0,    93,     0,
-       0,     0,     0,    94,    70,    71,     0,     0,     1,     0,
-       2,     0,     0,     0,    33,    68,    34,    69,    73,    74,
-       0,     0,     0,     0,     0,     0,    75,    76,     0,     0,
-       0,    70,    71,     0,    77,    78,     0,    79,     0,     3,
-       0,     0,     0,    80,     0,    73,    74,     0,     0,    81,
-       0,     0,    82,    75,    76,     4,     5,     0,     0,     0,
-       0,    77,    78,     6,    79,     0,    83,    84,     7,     0,
-      80,     0,     0,     0,    85,   187,    81,    86,     0,    82,
-       0,     0,     0,     0,     0,     0,     0,     0,     0,     8,
-       0,     0,    87,    83,    84,     0,     0,     0,     0,    10,
-       0,    85,    88,    89,    86,     0,    90,    91,     0,     0,
-       0,     0,     0,     0,     9,    92,     0,     0,     0,    87,
-       0,    93,     0,    10,     0,     0,    94,     0,     0,    88,
-      89,     0,     0,    90,    91,     0,    11,    33,    68,    34,
-      69,     0,    92,    12,     0,     0,    13,     0,    93,     0,
-       0,     0,     0,   188,    70,   161,     0,     0,     0,     0,
-       0,     0,     0,     0,    33,    68,    34,    69,    73,    74,
-       0,     0,     0,     0,     0,     0,    75,    76,     0,     0,
-       0,    70,    71,     0,    77,    78,     0,    79,     0,     0,
-       0,     0,     0,    80,     0,    73,    74,     0,     0,    81,
-       0,     0,    82,    75,    76,     0,     0,     0,     0,     0,
-       0,    77,    78,     0,    79,     0,    83,    84,     0,     0,
-      80,     0,     0,     0,    85,     0,    81,    86,     0,    82,
+       0,    84,    85,     0,     0,     0,     0,     0,     0,    86,
+       0,     0,    87,     0,    33,    68,    34,    69,     0,     0,
+     189,     0,     0,     0,     0,     0,     0,    88,     0,     0,
+       0,    70,    71,     0,    10,     0,     0,    89,    90,     0,
+       0,    91,    92,     0,     0,    73,    74,     0,     0,     0,
+      93,     0,     0,    75,    76,    77,    94,     0,     0,     0,
+       0,   191,    78,    79,     0,    80,     0,     0,     0,     0,
+       0,    81,     0,     0,     0,     0,   190,    82,     0,     0,
+      83,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,    84,    85,     0,     0,     0,     0,
+       0,     0,    86,     0,     0,    87,     0,    33,    68,    34,
+      69,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+      88,     0,     0,     0,    70,    71,    72,     0,     0,     0,
+      89,    90,     0,     0,    91,    92,     0,     0,    73,    74,
+       0,     0,     0,    93,     0,     0,    75,    76,    77,    94,
+       0,     0,     0,     0,   191,    78,    79,     0,    80,     0,
+       0,     0,     0,     0,    81,     0,     0,     0,     0,     0,
+      82,     0,     0,    83,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,    84,    85,     0,
+       0,     0,     0,     0,     0,    86,     0,     0,    87,     0,
+      33,    68,    34,    69,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,    88,     0,     0,     0,    70,    71,     0,
+       0,     0,     0,    89,    90,     0,     0,    91,    92,     0,
+       0,    73,    74,     0,     0,     0,    93,     0,     0,    75,
+      76,    77,    94,     0,     0,     0,     0,    95,    78,    79,
+       0,    80,     0,     0,     0,     0,     0,    81,     0,     0,
+       0,     0,     0,    82,     0,     0,    83,     0,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,    87,    83,    84,     0,     0,    68,     0,    69,
-       0,    85,    88,    89,    86,     0,    90,    91,     0,     0,
-       0,     0,     0,    70,   161,    92,     0,     0,     0,    87,
-       0,    93,     0,     0,     0,     0,    94,    73,    74,    88,
-      89,     0,     0,    90,    91,     0,    76,     0,     0,     0,
-       0,     0,    92,    77,    78,     0,    79,     0,    93,     0,
-       0,     0,    80,    94,    73,    74,     0,     0,     0,     0,
-       0,    82,     0,    76,     0,     0,     0,     0,     0,     0,
-      77,    78,     0,    79,     0,    83,    84,     0,     0,    80,
-       0,     0,     0,    85,     0,     0,    86,     0,    82,     0,
+      84,    85,     0,     0,     0,     0,     0,     0,    86,     0,
+       0,    87,     0,    33,    68,    34,    69,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,    88,     0,     0,     0,
+      70,    71,     0,     0,     0,     0,    89,    90,     0,     0,
+      91,    92,     0,     0,    73,    74,     0,     0,     0,    93,
+     136,     0,    75,    76,    77,    94,     0,     0,     0,     0,
+      95,    78,    79,     0,    80,     0,     0,     0,     0,     0,
+      81,     0,     0,     0,     0,     0,    82,     0,     0,    83,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,    87,    83,   256,     0,     0,     0,     0,     0,     0,
-      85,    88,     0,     0,     0,    90,    91,     0,     0,     0,
-       0,     0,     0,     0,    92,     0,     0,     0,    87,     0,
-      93,     0,     0,     0,     0,     0,     0,     0,    88,     0,
-       0,     0,    90,    91,     0,     0,     0,     0,     0,     0,
-       0,    92,     0,     0,     0,     0,     0,    93
+       0,     0,     0,    84,    85,     0,     0,     0,     0,     0,
+       0,    86,     0,     0,    87,     0,    33,    68,    34,    69,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,    88,
+       0,     0,     0,    70,    71,     0,    10,     0,     0,    89,
+      90,     0,     0,    91,    92,     0,     0,    73,    74,     0,
+       0,     0,    93,     0,     0,    75,    76,    77,    94,     0,
+       0,     0,     0,    95,    78,    79,     0,    80,     0,     0,
+       0,     0,     0,    81,     0,     0,     0,     0,   190,    82,
+       0,     0,    83,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,    84,    85,     0,     0,
+       0,     0,     0,     0,    86,     0,     0,    87,     0,    33,
+      68,    34,    69,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,    88,     0,     0,     0,    70,   164,     0,     0,
+       0,     0,    89,    90,     0,     0,    91,    92,     0,     0,
+      73,    74,     0,     0,     0,    93,     0,     0,    75,    76,
+      77,    94,     0,     0,     0,     0,   191,    78,    79,     0,
+      80,     0,     0,     0,     0,     0,    81,     0,     0,     0,
+       0,     0,    82,     0,     0,    83,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,    84,
+      85,     0,     0,     0,     0,     0,     0,    86,     0,     0,
+      87,     0,    33,    68,    34,    69,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,    88,     0,     0,     0,    70,
+      71,     0,     0,     0,     0,    89,    90,     0,     0,    91,
+      92,     0,     0,    73,    74,     0,     0,     0,    93,     0,
+       0,    75,    76,    77,    94,     0,     0,     0,     0,    95,
+      78,    79,     0,    80,     0,     0,    68,     0,    69,    81,
+       0,     0,     0,     0,     0,    82,     0,     0,    83,     0,
+       0,     0,    70,   164,     0,     0,     0,     0,     0,     0,
+       0,     0,    84,    85,     0,     0,    73,    74,     0,     0,
+      86,     0,     0,    87,     0,     0,    77,     0,     0,     0,
+       0,     0,     0,    78,    79,     0,    80,     0,    88,     0,
+       0,     0,    81,     0,     0,     0,     0,     0,    89,    90,
+       0,    83,    91,    92,     0,     0,     0,     0,     0,     0,
+       0,    93,     0,     0,     0,    84,    85,    94,    73,    74,
+       0,     0,    95,    86,     0,     0,    87,     0,    77,     0,
+       0,     0,     0,     0,     0,    78,    79,     0,    80,     0,
+       0,    88,     0,     0,    81,     0,     0,     0,     0,     0,
+       0,    89,     0,    83,     0,    91,    92,     0,     0,     0,
+       0,     0,     0,     0,    93,     0,     0,    84,   260,     0,
+      94,     0,     0,     0,     0,    86,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,    88,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,    89,     0,     0,     0,    91,    92,     0,
+       0,     0,     0,     0,     0,     0,    93,     0,     0,     0,
+       0,     0,    94
 };
 
 static const yytype_int16 yycheck[] =
 {
-       4,   134,    12,   130,   148,   126,    43,    11,    12,   195,
-       8,   254,     8,   134,     8,     4,   114,     6,     7,     7,
-       7,   233,    27,     4,    45,     6,    29,     4,    32,     6,
-      23,    24,    36,    37,    38,    39,    40,    29,    75,    28,
-       4,    10,     6,   222,   176,    21,    22,    21,    22,    57,
-     148,    72,   217,     8,    80,   188,    71,    94,    31,    67,
-      82,   139,    50,    50,   142,   186,   121,   188,    21,    22,
-       4,    73,     6,   217,    10,    11,    12,    13,    14,    15,
-      16,    17,    18,    19,   105,    21,    22,    10,    61,   187,
-      39,   116,    21,    22,   116,   121,   100,    37,    76,    33,
-      40,    41,    62,   113,     9,   142,   131,   128,   112,   113,
-      62,   115,   116,   150,    28,   121,    21,    22,   143,   217,
-     124,   125,    45,   255,   303,    94,   130,   164,    85,   144,
-     108,   252,    55,   136,   299,   116,     0,   349,   143,   127,
-     127,   262,   263,    79,   148,    59,   144,   333,   144,   153,
-     144,   188,   144,   146,   143,   299,   160,   400,    30,   163,
-     372,   198,   143,   375,   201,   173,   143,   122,   144,   355,
-     144,    94,   176,   133,   178,   179,   274,   126,   118,   143,
-     133,   133,   105,   316,    74,     3,    67,   110,   149,    21,
-      22,    81,   319,   122,   143,   316,   233,    68,   410,    89,
-       0,   299,   239,    73,   165,   128,    53,     4,   144,     6,
-      17,    18,    19,   217,   218,    96,    97,    88,   222,   108,
-      74,   111,    57,   367,   117,   114,    26,    81,   134,    64,
-     416,    78,    86,   237,   145,    89,   273,   241,   275,   145,
-     373,    73,    33,   280,   125,    92,    93,     9,   252,   347,
-     287,   255,   373,   116,   144,   382,   383,   111,    57,    21,
-      22,    21,    22,   143,   139,    64,   141,   106,   115,   367,
-       5,   404,    79,   394,    34,   312,     4,    97,     6,    46,
-      47,   291,    85,   404,   143,    46,    47,   291,   144,   145,
-     137,   144,   145,    21,    22,   299,   144,   145,    58,   303,
-      21,    22,   144,   145,   134,    33,   144,   145,     7,   346,
-     144,   145,   349,   144,   145,   319,   144,   145,   144,   145,
-       7,   358,   455,   144,   145,   144,   145,   144,   145,   144,
-     145,   143,   336,    84,   455,   372,   144,   145,   375,    10,
-      11,    12,    13,    14,    15,    16,    17,    18,    19,   143,
-      21,    22,   144,   145,   144,   145,   144,   145,   144,   145,
-     144,   145,     5,   367,   143,   143,    84,     4,     5,     6,
-       7,     5,    73,   410,   501,   412,   145,   100,   382,   383,
-     143,     7,   136,     7,    21,    22,    23,   143,     7,    11,
-     394,   143,   429,   143,   521,     8,    63,     9,    35,    36,
-      20,   144,   529,    73,   144,   101,    43,    44,    79,   145,
-      75,   144,   449,   143,    51,    52,   143,    54,    87,   456,
-     130,    87,    59,    60,   143,   131,     7,    63,    86,    66,
-       7,    99,    69,     7,    42,    99,    99,   441,    77,    42,
-     144,    98,     5,   143,   448,   143,    83,    84,    27,    94,
-     145,   488,   136,   143,    91,    87,    94,    94,   143,   103,
-     143,   103,    42,    98,   143,    38,   143,     7,   472,   473,
-     101,   145,   109,     4,     5,     6,     7,    42,    19,    10,
-     484,    97,   119,   120,    90,     5,   123,   124,   143,    42,
-      21,    22,   496,   144,   498,   132,     7,   501,   145,   110,
-      95,   138,   135,   144,    35,    36,   143,   144,   104,     9,
-     112,   104,    43,    44,   143,    33,   143,   521,   143,    70,
-      51,    52,   112,    54,    70,   529,   102,   144,     7,    60,
-     339,   185,   473,   498,    65,    66,    53,   532,    69,    54,
-     237,   113,   367,   118,   486,   150,   456,   262,   110,   488,
-     263,   199,    83,    84,   154,     4,     5,     6,     7,   199,
-      91,    10,   159,    94,   416,   186,   136,    -1,    -1,    -1,
-     141,   439,    21,    22,    -1,    -1,    -1,    -1,   109,    -1,
-      -1,    -1,    -1,    -1,    -1,   116,    35,    36,   119,   120,
-      -1,    -1,   123,   124,    43,    44,    -1,    -1,    -1,    -1,
-      -1,   132,    51,    52,    -1,    54,    -1,   138,    -1,    -1,
-      -1,    60,   143,    -1,    -1,    -1,    65,    66,    -1,    -1,
-      69,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    83,    84,    -1,     4,     5,     6,
-       7,    -1,    91,    -1,    -1,    94,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    21,    22,    23,    -1,    -1,    -1,
-     109,    -1,    -1,    -1,     4,     5,     6,     7,    35,    36,
-     119,   120,    -1,    -1,   123,   124,    43,    44,    -1,    -1,
-      -1,    21,    22,   132,    51,    52,    -1,    54,    -1,   138,
-      -1,    -1,    -1,    60,   143,    35,    36,    -1,    -1,    66,
-      -1,    -1,    69,    43,    44,    -1,    -1,    -1,    -1,    -1,
-      -1,    51,    52,    -1,    54,    -1,    83,    84,    -1,    -1,
-      60,    -1,    -1,    -1,    91,    -1,    66,    94,    -1,    69,
+       4,   132,   128,   136,    12,   198,   258,    11,    12,   237,
+     136,   151,     8,     8,    27,    43,     4,     7,     6,     7,
+       4,     4,     6,     6,   116,     4,    29,     6,    32,    23,
+      24,     8,    36,    37,    38,    39,    40,   221,    29,   226,
+      28,    81,     8,    72,   179,    21,    22,    75,    21,    22,
+      21,    22,    57,    21,    22,    69,    63,    83,   191,   151,
+       7,    51,    67,   189,    75,   191,    63,    95,    21,    22,
+      28,    82,    31,    58,     0,    89,   152,   188,    39,    90,
+      65,   221,   122,    21,    22,   135,     4,     1,     6,     3,
+     122,   117,   168,    47,    48,    33,   146,   101,   190,    10,
+      26,   112,    60,    62,    51,    37,    74,   115,    40,    41,
+     114,   115,   140,   117,   118,    33,   145,   145,    32,    74,
+     304,   308,   126,   127,   259,   153,   354,   134,   132,   221,
+     256,   144,     0,   117,   145,    49,    50,   134,   128,   167,
+     266,   267,   145,    57,    10,   338,   137,   151,    62,   145,
+     145,   379,   156,   147,   382,   407,   144,   123,   134,   163,
+     144,   144,   166,   191,   304,   144,   127,   360,   145,    83,
+     123,   176,   145,   201,   145,   179,   204,   181,   182,   290,
+      46,   128,    86,   144,    95,     9,   278,   119,   321,   417,
+      56,   122,   117,   324,   108,   321,   109,    21,    22,    54,
+      47,    48,   115,   117,    77,   145,   146,   132,    46,   237,
+     145,   146,   304,    75,    68,   243,   130,   221,   222,   144,
+      82,    30,   226,   137,    79,    87,   140,     3,    90,    95,
+     423,    17,    18,    19,   374,    73,   109,   241,    93,    94,
+     106,   245,    74,    97,    98,   111,     4,   380,     6,   277,
+     112,   279,   256,     9,   380,   259,   284,   146,   389,   390,
+     352,   116,   118,   129,   292,    21,    22,     4,   106,     6,
+      58,   140,   126,    33,   143,   401,    86,    65,   411,   140,
+     144,   142,   374,   138,   107,   411,   290,   117,   296,   317,
+       5,   129,   296,     4,    80,     6,    21,    22,    35,    36,
+     304,   145,   146,    98,   308,    21,    22,   144,    45,    34,
+      21,    22,   145,   146,   135,    52,    53,     7,    55,     7,
+     324,    85,    33,   351,    61,   144,   354,   145,   146,   145,
+     146,   464,   144,    70,    59,   363,   144,   341,   464,   145,
+     146,   145,   146,   145,   146,   145,   146,    84,    85,   145,
+     146,   379,   145,   146,   382,    92,    10,    11,    12,    13,
+      14,    15,    16,    17,    18,    19,     5,    21,    22,   144,
+     374,   145,   146,   110,   145,   146,   145,   146,   144,   510,
+     145,   146,    85,   120,     5,   389,   390,   124,   125,   417,
+      74,   419,   145,   146,   145,   146,   133,   401,   146,   530,
+     145,   146,   139,   101,   144,     7,   137,   538,   144,     7,
+     438,     7,    11,     8,   144,   144,     4,     5,     6,     7,
+       9,    64,   145,    20,    74,   145,    80,   146,    76,   102,
+     458,   145,   144,    21,    22,    23,    88,   465,   144,    88,
+     131,    64,   144,   132,     7,   100,   450,    35,    36,   100,
+      87,     7,     7,   457,   100,    43,    44,    45,    42,    78,
+      42,    99,   145,   144,    52,    53,   144,    55,     5,   497,
+      27,   146,    60,    61,   137,    95,   144,   481,   482,    67,
+      88,    95,    70,   144,   144,    38,   145,   145,   104,   493,
+      99,   145,   104,    42,     7,   102,    84,    85,    98,   144,
+     144,   505,    42,   507,    92,   146,   510,    95,    10,    11,
+      12,    13,    14,    15,    16,    17,    18,    19,    91,    21,
+      22,    19,   110,     5,    42,   136,   530,   144,     7,    96,
+     145,   111,   120,   121,   538,   145,   124,   125,   113,   105,
+     146,     4,     5,     6,     7,   133,   105,    10,     9,    33,
+     144,   139,   144,   144,    71,   113,   144,   145,    21,    22,
+      71,   103,   145,     7,   344,   482,   507,   241,    53,   115,
+     120,    54,    35,    36,   153,   374,   495,   541,    80,   465,
+      43,    44,    45,   157,   266,   497,   189,   267,   112,    52,
+      53,   162,    55,   138,   423,   448,   144,   202,    61,    -1,
+      -1,   202,    -1,    66,    67,    -1,    -1,    70,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    84,    85,    -1,    -1,    -1,    -1,    -1,    -1,    92,
+      -1,    -1,    95,    -1,     4,     5,     6,     7,    -1,    -1,
+      10,    -1,    -1,    -1,    -1,    -1,    -1,   110,    -1,    -1,
+      -1,    21,    22,    -1,   117,    -1,    -1,   120,   121,    -1,
+      -1,   124,   125,    -1,    -1,    35,    36,    -1,    -1,    -1,
+     133,    -1,    -1,    43,    44,    45,   139,    -1,    -1,    -1,
+      -1,   144,    52,    53,    -1,    55,    -1,    -1,    -1,    -1,
+      -1,    61,    -1,    -1,    -1,    -1,    66,    67,    -1,    -1,
+      70,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    84,    85,    -1,    -1,    -1,    -1,
+      -1,    -1,    92,    -1,    -1,    95,    -1,     4,     5,     6,
+       7,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+     110,    -1,    -1,    -1,    21,    22,    23,    -1,    -1,    -1,
+     120,   121,    -1,    -1,   124,   125,    -1,    -1,    35,    36,
+      -1,    -1,    -1,   133,    -1,    -1,    43,    44,    45,   139,
+      -1,    -1,    -1,    -1,   144,    52,    53,    -1,    55,    -1,
+      -1,    -1,    -1,    -1,    61,    -1,    -1,    -1,    -1,    -1,
+      67,    -1,    -1,    70,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    84,    85,    -1,
+      -1,    -1,    -1,    -1,    -1,    92,    -1,    -1,    95,    -1,
+       4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,   110,    -1,    -1,    -1,    21,    22,    -1,
+      -1,    -1,    -1,   120,   121,    -1,    -1,   124,   125,    -1,
+      -1,    35,    36,    -1,    -1,    -1,   133,    -1,    -1,    43,
+      44,    45,   139,    -1,    -1,    -1,    -1,   144,    52,    53,
+      -1,    55,    -1,    -1,    -1,    -1,    -1,    61,    -1,    -1,
+      -1,    -1,    -1,    67,    -1,    -1,    70,    -1,    -1,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,   109,    83,    84,    -1,    -1,    -1,    -1,    -1,
-      -1,    91,   119,   120,    94,    -1,   123,   124,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,   132,    -1,    -1,    -1,   109,
-      -1,   138,    -1,    -1,    -1,    -1,   143,    -1,    -1,   119,
-     120,    -1,    -1,   123,   124,    -1,    -1,     4,     5,     6,
-       7,    -1,   132,   133,    -1,    -1,    -1,    -1,   138,    -1,
-      -1,    -1,    -1,   143,    21,    22,    -1,    -1,     1,    -1,
-       3,    -1,    -1,    -1,     4,     5,     6,     7,    35,    36,
-      -1,    -1,    -1,    -1,    -1,    -1,    43,    44,    -1,    -1,
-      -1,    21,    22,    -1,    51,    52,    -1,    54,    -1,    32,
-      -1,    -1,    -1,    60,    -1,    35,    36,    -1,    -1,    66,
-      -1,    -1,    69,    43,    44,    48,    49,    -1,    -1,    -1,
-      -1,    51,    52,    56,    54,    -1,    83,    84,    61,    -1,
-      60,    -1,    -1,    -1,    91,    65,    66,    94,    -1,    69,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    82,
-      -1,    -1,   109,    83,    84,    -1,    -1,    -1,    -1,   116,
-      -1,    91,   119,   120,    94,    -1,   123,   124,    -1,    -1,
-      -1,    -1,    -1,    -1,   107,   132,    -1,    -1,    -1,   109,
-      -1,   138,    -1,   116,    -1,    -1,   143,    -1,    -1,   119,
-     120,    -1,    -1,   123,   124,    -1,   129,     4,     5,     6,
-       7,    -1,   132,   136,    -1,    -1,   139,    -1,   138,    -1,
-      -1,    -1,    -1,   143,    21,    22,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,     4,     5,     6,     7,    35,    36,
-      -1,    -1,    -1,    -1,    -1,    -1,    43,    44,    -1,    -1,
-      -1,    21,    22,    -1,    51,    52,    -1,    54,    -1,    -1,
-      -1,    -1,    -1,    60,    -1,    35,    36,    -1,    -1,    66,
-      -1,    -1,    69,    43,    44,    -1,    -1,    -1,    -1,    -1,
-      -1,    51,    52,    -1,    54,    -1,    83,    84,    -1,    -1,
-      60,    -1,    -1,    -1,    91,    -1,    66,    94,    -1,    69,
+      84,    85,    -1,    -1,    -1,    -1,    -1,    -1,    92,    -1,
+      -1,    95,    -1,     4,     5,     6,     7,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,   110,    -1,    -1,    -1,
+      21,    22,    -1,    -1,    -1,    -1,   120,   121,    -1,    -1,
+     124,   125,    -1,    -1,    35,    36,    -1,    -1,    -1,   133,
+     134,    -1,    43,    44,    45,   139,    -1,    -1,    -1,    -1,
+     144,    52,    53,    -1,    55,    -1,    -1,    -1,    -1,    -1,
+      61,    -1,    -1,    -1,    -1,    -1,    67,    -1,    -1,    70,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,   109,    83,    84,    -1,    -1,     5,    -1,     7,
-      -1,    91,   119,   120,    94,    -1,   123,   124,    -1,    -1,
-      -1,    -1,    -1,    21,    22,   132,    -1,    -1,    -1,   109,
-      -1,   138,    -1,    -1,    -1,    -1,   143,    35,    36,   119,
-     120,    -1,    -1,   123,   124,    -1,    44,    -1,    -1,    -1,
-      -1,    -1,   132,    51,    52,    -1,    54,    -1,   138,    -1,
-      -1,    -1,    60,   143,    35,    36,    -1,    -1,    -1,    -1,
-      -1,    69,    -1,    44,    -1,    -1,    -1,    -1,    -1,    -1,
-      51,    52,    -1,    54,    -1,    83,    84,    -1,    -1,    60,
-      -1,    -1,    -1,    91,    -1,    -1,    94,    -1,    69,    -1,
+      -1,    -1,    -1,    84,    85,    -1,    -1,    -1,    -1,    -1,
+      -1,    92,    -1,    -1,    95,    -1,     4,     5,     6,     7,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   110,
+      -1,    -1,    -1,    21,    22,    -1,   117,    -1,    -1,   120,
+     121,    -1,    -1,   124,   125,    -1,    -1,    35,    36,    -1,
+      -1,    -1,   133,    -1,    -1,    43,    44,    45,   139,    -1,
+      -1,    -1,    -1,   144,    52,    53,    -1,    55,    -1,    -1,
+      -1,    -1,    -1,    61,    -1,    -1,    -1,    -1,    66,    67,
+      -1,    -1,    70,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    84,    85,    -1,    -1,
+      -1,    -1,    -1,    -1,    92,    -1,    -1,    95,    -1,     4,
+       5,     6,     7,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,   110,    -1,    -1,    -1,    21,    22,    -1,    -1,
+      -1,    -1,   120,   121,    -1,    -1,   124,   125,    -1,    -1,
+      35,    36,    -1,    -1,    -1,   133,    -1,    -1,    43,    44,
+      45,   139,    -1,    -1,    -1,    -1,   144,    52,    53,    -1,
+      55,    -1,    -1,    -1,    -1,    -1,    61,    -1,    -1,    -1,
+      -1,    -1,    67,    -1,    -1,    70,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    84,
+      85,    -1,    -1,    -1,    -1,    -1,    -1,    92,    -1,    -1,
+      95,    -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,   110,    -1,    -1,    -1,    21,
+      22,    -1,    -1,    -1,    -1,   120,   121,    -1,    -1,   124,
+     125,    -1,    -1,    35,    36,    -1,    -1,    -1,   133,    -1,
+      -1,    43,    44,    45,   139,    -1,    -1,    -1,    -1,   144,
+      52,    53,    -1,    55,    -1,    -1,     5,    -1,     7,    61,
+      -1,    -1,    -1,    -1,    -1,    67,    -1,    -1,    70,    -1,
+      -1,    -1,    21,    22,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    84,    85,    -1,    -1,    35,    36,    -1,    -1,
+      92,    -1,    -1,    95,    -1,    -1,    45,    -1,    -1,    -1,
+      -1,    -1,    -1,    52,    53,    -1,    55,    -1,   110,    -1,
+      -1,    -1,    61,    -1,    -1,    -1,    -1,    -1,   120,   121,
+      -1,    70,   124,   125,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,   133,    -1,    -1,    -1,    84,    85,   139,    35,    36,
+      -1,    -1,   144,    92,    -1,    -1,    95,    -1,    45,    -1,
+      -1,    -1,    -1,    -1,    -1,    52,    53,    -1,    55,    -1,
+      -1,   110,    -1,    -1,    61,    -1,    -1,    -1,    -1,    -1,
+      -1,   120,    -1,    70,    -1,   124,   125,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,   133,    -1,    -1,    84,    85,    -1,
+     139,    -1,    -1,    -1,    -1,    92,    -1,    -1,    -1,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,   109,    83,    84,    -1,    -1,    -1,    -1,    -1,    -1,
-      91,   119,    -1,    -1,    -1,   123,   124,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,   132,    -1,    -1,    -1,   109,    -1,
-     138,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   119,    -1,
-      -1,    -1,   123,   124,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,   132,    -1,    -1,    -1,    -1,    -1,   138
+      -1,    -1,    -1,   110,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,   120,    -1,    -1,    -1,   124,   125,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,   133,    -1,    -1,    -1,
+      -1,    -1,   139
 };
 
   /* YYSTOS[STATE-NUM] -- The (internal number of the) accessing
      symbol of state STATE-NUM.  */
-static const yytype_uint8 yystos[] =
+static const yytype_uint16 yystos[] =
 {
-       0,     1,     3,    32,    48,    49,    56,    61,    82,   107,
-     116,   129,   136,   139,   148,   149,   150,   151,   152,   153,
-     154,   175,   176,   179,   180,   183,   185,   188,   189,   190,
-     254,   255,   121,     4,     6,   252,    80,   121,    73,   121,
-      85,    28,    59,   191,   252,   186,   187,   202,   252,     0,
-     139,   141,    82,   188,    29,   136,   184,    30,   139,   142,
-       3,   252,    73,   252,   252,   252,   252,   252,     5,     7,
-      21,    22,    23,    35,    36,    43,    44,    51,    52,    54,
-      60,    66,    69,    83,    84,    91,    94,   109,   119,   120,
-     123,   124,   132,   138,   143,   157,   192,   193,   194,   196,
-     228,   229,   230,   231,   232,   233,   234,   235,   242,   245,
-     248,   252,   117,   145,    33,   143,    85,   184,   191,   106,
-     190,    31,    61,     5,    97,   143,   134,   222,   223,   131,
-     143,   190,     7,     7,   133,   228,   238,   239,   143,    84,
-     143,     5,   143,   143,    84,   188,   228,     5,    73,   195,
-     145,    21,    22,    33,   249,   252,    23,    24,   146,   250,
-     100,    22,   231,    27,   143,   181,   182,   252,   187,   143,
-     196,   251,   252,   252,   189,     7,    46,    47,    46,    47,
-     136,   177,   252,   155,   156,   252,    10,    65,   143,   224,
-     225,   226,   227,   228,   245,   143,   251,   224,   133,   236,
-     237,    62,   239,   240,     7,    53,    78,    92,    93,   115,
-     137,   243,   243,   228,     7,   144,   144,   143,   196,   199,
-     200,   203,   232,   252,   222,   194,   252,   229,   230,   143,
-     252,   252,    23,    59,   144,   228,   241,   145,   222,    11,
-     144,   145,   190,   155,    45,    72,   105,   128,   161,   252,
-     252,   143,   143,   164,   144,   145,    84,   157,   227,   196,
-     224,   228,     8,     9,    10,    11,    12,    13,    14,    15,
-      16,    17,    18,    19,    79,   247,    20,   242,   244,   144,
-     122,   228,   237,   240,   228,    63,   144,    73,    73,   144,
-     199,    33,   201,   202,    74,    81,    89,   111,   198,   145,
-     201,    39,   126,   197,    75,   204,   101,   211,   212,   144,
-     241,   144,   145,   182,   228,   252,   143,    87,    87,   143,
-      57,    64,   178,   245,   246,   252,   130,   161,   162,   163,
-     155,    10,    45,    55,    94,   105,   110,   128,   158,   159,
-     160,   144,   225,   226,    17,    18,    19,    79,   228,   143,
-     196,   228,    10,    94,   144,   145,   131,   228,   122,    63,
-     228,     7,   144,   202,    99,    99,    99,    86,   199,     7,
-       7,   201,    42,    77,   205,    42,   144,    98,   213,   144,
-     228,   224,   143,   143,   251,     5,    67,    96,    97,   125,
-     253,   144,   145,   144,   145,    37,    40,    41,   118,   173,
-     145,   136,   165,    94,   143,   242,    87,   252,   158,   228,
-     143,   196,     9,   241,    94,   242,   143,   228,   144,    71,
-     144,   200,   103,   103,   241,   224,    98,   206,   241,    42,
-     108,   114,   214,   215,   144,   251,   251,   144,    57,    64,
-     245,   143,   174,   161,    38,   101,   166,   224,   143,     9,
-     241,   228,   144,   244,     7,    97,    42,    90,   207,   218,
-     219,   228,    19,   144,   144,     5,   253,   168,   169,   170,
-     171,   172,   252,   143,    42,   144,   252,   228,   144,   144,
-     144,   224,   218,     7,   135,   208,   209,   210,   145,    34,
-      58,   220,     7,    50,   127,   216,   110,   144,   145,     7,
-      28,   143,   252,   168,    76,   108,   167,   144,   252,   210,
-     219,    95,   221,   104,   112,   104,     9,   252,   169,   251,
-     144,   143,    33,    68,    88,     7,    50,   127,   217,   143,
-     144,   251,   143,    70,   112,    70,   251,   144,   211,   144,
-     102,   144,     7
+       0,     1,     3,    32,    49,    50,    57,    62,    83,   108,
+     117,   130,   137,   140,   149,   150,   151,   152,   153,   154,
+     155,   176,   177,   180,   181,   184,   186,   189,   190,   191,
+     256,   257,   122,     4,     6,   254,    81,   122,    74,   122,
+      86,    28,    60,   192,   254,   187,   188,   203,   254,     0,
+     140,   142,    83,   189,    29,   137,   185,    30,   140,   143,
+       3,   254,    74,   254,   254,   254,   254,   254,     5,     7,
+      21,    22,    23,    35,    36,    43,    44,    45,    52,    53,
+      55,    61,    67,    70,    84,    85,    92,    95,   110,   120,
+     121,   124,   125,   133,   139,   144,   158,   193,   194,   195,
+     197,   229,   230,   231,   232,   233,   234,   235,   236,   237,
+     244,   247,   250,   254,   118,   146,    33,   144,    86,   185,
+     192,   107,   191,    31,    62,     5,    98,   144,   135,   223,
+     224,   132,   144,   191,     7,     7,   134,   229,   240,   241,
+     144,   144,    85,   144,     5,   144,   144,    85,   189,   229,
+       5,    74,   196,   146,    21,    22,    33,   251,   254,    23,
+      24,   147,   252,   101,    22,   232,    27,   144,   182,   183,
+     254,   188,   144,   197,   253,   254,   254,   190,     7,    47,
+      48,    47,    48,   137,   178,   254,   156,   157,   254,    10,
+      66,   144,   225,   226,   227,   228,   229,   247,   144,   253,
+     225,   134,   238,   239,    63,   241,   242,   229,     7,    54,
+      79,    93,    94,   116,   138,   245,   245,   229,     7,   145,
+     145,   144,   197,   200,   201,   204,   233,   254,   223,   195,
+     254,   230,   231,   144,   254,   254,    23,    60,   145,   229,
+     243,   146,   223,    11,   145,   146,   191,   156,    46,    73,
+     106,   129,   162,   254,   254,   144,   144,   165,   145,   146,
+      85,   158,   228,   197,   225,   229,     8,     9,    10,    11,
+      12,    13,    14,    15,    16,    17,    18,    19,    80,   249,
+      20,   244,   246,   145,   123,   229,   239,   242,   229,    64,
+      33,   145,    74,    74,   145,   200,    33,   202,   203,    75,
+      82,    90,   112,   199,   146,   202,    39,   127,   198,    76,
+     205,   102,   212,   213,   145,   243,   145,   146,   183,   229,
+     254,   144,    88,    88,   144,    58,    65,   179,   247,   248,
+     254,   131,   162,   163,   164,   156,    10,    46,    56,    95,
+     106,   111,   129,   159,   160,   161,   145,   226,   227,    17,
+      18,    19,    80,   229,   144,   197,   229,    10,    95,   145,
+     146,   132,   229,   123,    64,   158,   254,   229,     7,   145,
+     203,   100,   100,   100,    87,   200,     7,     7,   202,    42,
+      78,   206,    42,   145,    99,   214,   145,   229,   225,   144,
+     144,   253,     5,    68,    97,    98,   126,   255,   145,   146,
+     145,   146,    37,    40,    41,   119,   174,   146,   137,   166,
+      95,   144,   244,    88,   254,   159,   229,   144,   197,     9,
+     243,    95,   244,   144,   229,   145,   145,   145,    72,   145,
+     201,   104,   104,   243,   225,    99,   207,   243,    42,   109,
+     115,   215,   216,   145,   253,   253,   145,    58,    65,   247,
+     144,   175,   162,    38,   102,   167,   225,   144,     9,   243,
+     229,   145,   246,     7,    98,    42,    91,   208,   219,   220,
+     229,    19,   145,   145,     5,   255,   169,   170,   171,   172,
+     173,   254,   144,    42,   145,   254,   229,   145,   145,   145,
+     225,   219,     7,   136,   209,   210,   211,   146,    34,    59,
+     221,     7,    51,   128,   217,   111,   145,   146,     7,    28,
+     144,   254,   169,    77,   109,   168,   145,   254,   211,   220,
+      96,   222,   105,   113,   105,     9,   254,   170,   253,   145,
+     144,    33,    69,    89,     7,    51,   128,   218,   144,   145,
+     253,   144,    71,   113,    71,   253,   145,   212,   145,   103,
+     145,     7
 };
 
   /* YYR1[YYN] -- Symbol number of symbol that rule YYN derives.  */
-static const yytype_uint8 yyr1[] =
+static const yytype_uint16 yyr1[] =
 {
-       0,   147,   148,   148,   148,   148,   148,   148,   149,   149,
-     149,   149,   149,   149,   149,   149,   149,   149,   150,   151,
-     151,   151,   151,   152,   153,   154,   155,   156,   156,   157,
-     157,   157,   157,   157,   157,   157,   157,   157,   157,   157,
-     157,   157,   157,   157,   157,   157,   157,   158,   158,   158,
-     158,   158,   158,   158,   159,   159,   160,   160,   161,   161,
-     161,   161,   162,   162,   163,   163,   164,   164,   165,   165,
-     166,   166,   167,   167,   168,   168,   169,   169,   169,   170,
-     170,   171,   172,   173,   173,   173,   173,   174,   174,   175,
-     175,   175,   175,   176,   177,   177,   178,   178,   178,   178,
-     179,   180,   181,   181,   182,   183,   183,   184,   184,   185,
-     186,   186,   187,   188,   188,   189,   189,   190,   191,   191,
-     191,   192,   192,   193,   193,   194,   194,   194,   195,   196,
-     197,   197,   197,   198,   198,   198,   198,   198,   198,   198,
-     198,   199,   199,   200,   200,   200,   200,   200,   200,   201,
-     201,   202,   202,   203,   203,   204,   204,   205,   205,   206,
-     206,   207,   207,   208,   208,   209,   209,   210,   211,   212,
-     212,   213,   213,   214,   214,   215,   215,   216,   216,   216,
-     217,   217,   217,   218,   218,   219,   220,   220,   220,   221,
-     221,   221,   222,   222,   223,   224,   224,   225,   225,   226,
-     226,   227,   227,   227,   227,   227,   227,   227,   227,   227,
-     227,   227,   228,   228,   229,   229,   230,   230,   231,   231,
-     231,   231,   231,   231,   231,   231,   231,   231,   232,   232,
-     232,   232,   233,   234,   234,   235,   235,   236,   236,   237,
-     238,   238,   239,   240,   240,   241,   241,   242,   242,   242,
-     242,   242,   242,   242,   242,   243,   243,   243,   243,   243,
-     243,   244,   244,   245,   245,   246,   246,   247,   247,   247,
-     247,   247,   247,   247,   247,   247,   247,   248,   249,   249,
-     250,   250,   250,   251,   251,   252,   252,   253,   253,   253,
-     253,   254,   255,   255
+       0,   148,   149,   149,   149,   149,   149,   149,   150,   150,
+     150,   150,   150,   150,   150,   150,   150,   150,   151,   152,
+     152,   152,   152,   153,   154,   155,   156,   157,   157,   158,
+     158,   158,   158,   158,   158,   158,   158,   158,   158,   158,
+     158,   158,   158,   158,   158,   158,   158,   159,   159,   159,
+     159,   159,   159,   159,   160,   160,   161,   161,   162,   162,
+     162,   162,   163,   163,   164,   164,   165,   165,   166,   166,
+     167,   167,   168,   168,   169,   169,   170,   170,   170,   171,
+     171,   172,   173,   174,   174,   174,   174,   175,   175,   176,
+     176,   176,   176,   177,   178,   178,   179,   179,   179,   179,
+     180,   181,   182,   182,   183,   184,   184,   185,   185,   186,
+     187,   187,   188,   189,   189,   190,   190,   191,   192,   192,
+     192,   193,   193,   194,   194,   195,   195,   195,   196,   197,
+     198,   198,   198,   199,   199,   199,   199,   199,   199,   199,
+     199,   200,   200,   201,   201,   201,   201,   201,   201,   202,
+     202,   203,   203,   204,   204,   205,   205,   206,   206,   207,
+     207,   208,   208,   209,   209,   210,   210,   211,   212,   213,
+     213,   214,   214,   215,   215,   216,   216,   217,   217,   217,
+     218,   218,   218,   219,   219,   220,   221,   221,   221,   222,
+     222,   222,   223,   223,   224,   225,   225,   226,   226,   227,
+     227,   228,   228,   228,   228,   228,   228,   228,   228,   228,
+     228,   228,   229,   229,   230,   230,   231,   231,   232,   232,
+     232,   232,   232,   232,   232,   232,   232,   232,   232,   233,
+     233,   233,   233,   234,   234,   235,   236,   236,   237,   237,
+     238,   238,   239,   240,   240,   241,   242,   242,   243,   243,
+     244,   244,   244,

<TRUNCATED>


[25/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/TypeIDSelectors.hpp
----------------------------------------------------------------------
diff --git a/types/TypeIDSelectors.hpp b/types/TypeIDSelectors.hpp
new file mode 100644
index 0000000..d75a887
--- /dev/null
+++ b/types/TypeIDSelectors.hpp
@@ -0,0 +1,152 @@
+/**
+ * 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_TYPES_TYPE_ID_SELECTORS_HPP_
+#define QUICKSTEP_TYPES_TYPE_ID_SELECTORS_HPP_
+
+#include <type_traits>
+
+#include "types/TypeID.hpp"
+#include "utility/meta/Common.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+struct TypeIDSelectorAll;
+
+struct TypeIDSelectorNumeric;
+
+struct TypeIDSelectorParameterized;
+
+struct TypeIDSelectorNonParameterized;
+
+template <TypeID ...candidates>
+struct TypeIDSelectorEqualsAny;
+
+
+// Forward declaration
+template <TypeID type_id>
+struct TypeIDTrait;
+
+struct TypeIDSelectorAll {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+    inline static auto Invoke(const FunctorT &functor) {
+      return functor(TypeIDConstant());
+    }
+  };
+};
+
+struct TypeIDSelectorNumeric {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wreturn-type"
+    inline static auto Invoke(const FunctorT &functor)
+        -> decltype(functor(TypeIDConstant())) {
+      DLOG(FATAL) << "Unexpected TypeID: "
+                  << kTypeNames[static_cast<int>(TypeIDConstant::value)];
+    }
+#pragma GCC diagnostic pop
+  };
+};
+
+template <typename TypeIDConstant, typename FunctorT>
+struct TypeIDSelectorNumeric::Implementation<
+    TypeIDConstant, FunctorT,
+    std::enable_if_t<TypeIDTrait<TypeIDConstant::value>
+                         ::kStaticSuperTypeID == Type::kNumeric>> {
+  inline static auto Invoke(const FunctorT &functor) {
+    return functor(TypeIDConstant());
+  }
+};
+
+template <TypeID ...candidates>
+struct TypeIDSelectorEqualsAny {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wreturn-type"
+    inline static auto Invoke(const FunctorT &functor)
+        -> decltype(functor(TypeIDConstant())) {
+      DLOG(FATAL) << "Unexpected TypeID: "
+                  << kTypeNames[static_cast<int>(TypeIDConstant::value)];
+    }
+#pragma GCC diagnostic pop
+  };
+};
+
+template <TypeID ...candidates>
+template <typename TypeIDConstant, typename FunctorT>
+struct TypeIDSelectorEqualsAny<candidates...>::Implementation<
+    TypeIDConstant, FunctorT,
+    std::enable_if_t<
+        meta::EqualsAny<TypeIDConstant,
+                        std::integral_constant<TypeID, candidates>...>::value>> {
+  inline static auto Invoke(const FunctorT &functor) {
+    return functor(TypeIDConstant());
+  }
+};
+
+namespace internal {
+
+template <bool require_parameterized>
+struct TypeIDSelectorParameterizedHelper {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wreturn-type"
+    inline static auto Invoke(const FunctorT &functor)
+        -> decltype(functor(TypeIDConstant())) {
+      DLOG(FATAL) << "Unexpected TypeID: "
+                  << kTypeNames[static_cast<int>(TypeIDConstant::value)];
+    }
+#pragma GCC diagnostic pop
+  };
+};
+
+template <bool require_non_parameterized>
+template <typename TypeIDConstant, typename FunctorT>
+struct TypeIDSelectorParameterizedHelper<require_non_parameterized>::Implementation<
+    TypeIDConstant, FunctorT,
+    std::enable_if_t<TypeIDTrait<TypeIDConstant::value>::kParameterized
+                         ^ require_non_parameterized>> {
+  inline static auto Invoke(const FunctorT &functor) {
+    return functor(TypeIDConstant());
+  }
+};
+
+}  // namespace internal
+
+struct TypeIDSelectorNonParameterized
+    : internal::TypeIDSelectorParameterizedHelper<true> {};
+
+struct TypeIDSelectorParameterized
+    : internal::TypeIDSelectorParameterizedHelper<false> {};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_TYPE_ID_SELECTORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/TypeRegistrar.hpp
----------------------------------------------------------------------
diff --git a/types/TypeRegistrar.hpp b/types/TypeRegistrar.hpp
new file mode 100644
index 0000000..f4c9fb9
--- /dev/null
+++ b/types/TypeRegistrar.hpp
@@ -0,0 +1,122 @@
+/**
+ * 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_TYPES_TYPE_REGISTRAR_HPP_
+#define QUICKSTEP_TYPES_TYPE_REGISTRAR_HPP_
+
+#include <cstdint>
+#include <type_traits>
+
+#include "types/DatetimeLit.hpp"
+#include "types/IntervalLit.hpp"
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypeIDSelectors.hpp"
+#include "utility/meta/Common.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <TypeID type_id>
+struct TypeIDTrait;
+
+#define REGISTER_TYPE(T, type_id, super_type_id, parameterized, layout, CppType) \
+  class T; \
+  template <> struct TypeIDTrait<type_id> { \
+    typedef T TypeClass; \
+    typedef CppType cpptype; \
+    static constexpr TypeID kStaticTypeID = type_id; \
+    static constexpr Type::SuperTypeID kStaticSuperTypeID = super_type_id; \
+    static constexpr bool kParameterized = parameterized; \
+    static constexpr TypeStorageLayout kLayout = layout; \
+  };
+
+REGISTER_TYPE(BoolType, kBool, \
+              Type::kNumeric, false, kNativeEmbedded, bool);
+REGISTER_TYPE(IntType, kInt, \
+              Type::kNumeric, false, kNativeEmbedded, int);
+REGISTER_TYPE(LongType, kLong, \
+              Type::kNumeric, false, kNativeEmbedded, std::int64_t);
+REGISTER_TYPE(FloatType, kFloat, \
+              Type::kNumeric, false, kNativeEmbedded, float);
+REGISTER_TYPE(DoubleType, kDouble, \
+              Type::kNumeric, false, kNativeEmbedded, double);
+REGISTER_TYPE(DateType, kDate, \
+              Type::kOther, false, kNativeEmbedded, DateLit);
+REGISTER_TYPE(DatetimeType, kDatetime, \
+              Type::kOther, false, kNativeEmbedded, DatetimeLit);
+REGISTER_TYPE(DatetimeIntervalType, kDatetimeInterval, \
+              Type::kOther, false, kNativeEmbedded, DatetimeIntervalLit);
+REGISTER_TYPE(YearMonthIntervalType, kYearMonthInterval, \
+              Type::kOther, false, kNativeEmbedded, YearMonthIntervalLit);
+REGISTER_TYPE(CharType, kChar, \
+              Type::kAsciiString, true, kNonNativeInline, void);
+REGISTER_TYPE(VarCharType, kVarChar, \
+              Type::kAsciiString, true, kOutOfLine, void);
+REGISTER_TYPE(NullType, kNullType, \
+              Type::kOther, false, kNonNativeInline, void);
+
+#undef REGISTER_TYPE
+
+using TypeIDSequenceAll =
+    meta::MakeSequence<static_cast<std::size_t>(kNumTypeIDs)>
+        ::type::template cast_to<TypeID>;
+
+template <typename Selector = TypeIDSelectorAll, typename FunctorT>
+auto InvokeOnTypeID(const TypeID type_id, const FunctorT &functor);
+
+namespace internal {
+
+template <int l, int r, typename Selector, typename FunctorT>
+inline auto InvokeOnTypeIDInner(const int value,
+                                const FunctorT &functor) {
+  DCHECK_LE(l, r);
+  if (l == r) {
+    constexpr TypeID type_id = static_cast<TypeID>(r);
+    return Selector::template Implementation<
+        std::integral_constant<TypeID, type_id>, FunctorT>::Invoke(functor);
+  }
+  constexpr int m = (l + r) >> 1;
+  if (value <= m) {
+    return InvokeOnTypeIDInner<l, m, Selector, FunctorT>(value, functor);
+  } else {
+    return InvokeOnTypeIDInner<m+1, r, Selector, FunctorT>(value, functor);
+  }
+}
+
+}  // namespace internal
+
+template <typename Selector, typename FunctorT>
+auto InvokeOnTypeID(const TypeID type_id,
+                    const FunctorT &functor) {
+  return internal::InvokeOnTypeIDInner<0, static_cast<int>(kNumTypeIDs)-1,
+                                       Selector, FunctorT>(
+      static_cast<int>(type_id), functor);
+}
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_TYPE_REGISTRAR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/TypeSynthesizer.hpp
----------------------------------------------------------------------
diff --git a/types/TypeSynthesizer.hpp b/types/TypeSynthesizer.hpp
new file mode 100644
index 0000000..27ba02a
--- /dev/null
+++ b/types/TypeSynthesizer.hpp
@@ -0,0 +1,210 @@
+/**
+ * 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_TYPES_TYPE_SYNTHESIZER_HPP_
+#define QUICKSTEP_TYPES_TYPE_SYNTHESIZER_HPP_
+
+#include <cstddef>
+#include <type_traits>
+
+#include "types/Type.hpp"
+#include "types/Type.pb.h"
+#include "types/TypeID.hpp"
+#include "types/TypeRegistrar.hpp"
+#include "utility/Macros.hpp"
+#include "utility/PtrMap.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename TypeClass, bool parameterized>
+class TypeInstance;
+
+
+template <TypeID type_id>
+class TypeSynthesizer
+    : public Type,
+      public TypeInstance<typename TypeIDTrait<type_id>::TypeClass,
+                          TypeIDTrait<type_id>::kParameterized> {
+ public:
+  using Trait = TypeIDTrait<type_id>;
+  using TypeClass = typename Trait::TypeClass;
+
+  static constexpr Type::SuperTypeID kStaticSuperTypeID = Trait::kStaticSuperTypeID;
+  static constexpr TypeID kStaticTypeID = Trait::kStaticTypeID;
+  static constexpr bool kParameterized = Trait::kParameterized;
+  static constexpr TypeStorageLayout kLayout = Trait::kLayout;
+
+  typedef typename Trait::cpptype cpptype;
+
+  serialization::Type getProto() const override {
+    serialization::Type proto;
+
+    proto.mutable_type_id()->CopyFrom(TypeIDFactory::GetProto(type_id_));
+    proto.set_nullable(nullable_);
+
+    if (kParameterized) {
+      proto.set_length(parameter_);
+    }
+
+    return proto;
+  }
+
+  const Type& getNullableVersion() const override {
+    return getInstance<kParameterized>(true);
+  }
+
+  const Type& getNonNullableVersion() const override {
+    return getInstance<kParameterized>(false);
+  }
+
+ protected:
+  template <TypeStorageLayout layout = kLayout, bool parameterized = kParameterized>
+  explicit TypeSynthesizer(const bool nullable,
+                           std::enable_if_t<layout == kNativeEmbedded ||
+                                            layout == kNativeInline>* = 0)
+      : Type(kStaticSuperTypeID, kStaticTypeID, nullable,
+             sizeof(cpptype), sizeof(cpptype)) {
+    DCHECK(!kParameterized);
+  }
+
+  template <TypeStorageLayout layout = kLayout, bool parameterized = kParameterized>
+  TypeSynthesizer(const bool nullable,
+                  const std::size_t minimum_byte_length,
+                  const std::size_t maximum_byte_length,
+                  const std::size_t parameter,
+                  std::enable_if_t<parameterized &&
+                                   (layout == kNonNativeInline ||
+                                    layout == kOutOfLine)>* = 0)
+      : Type(kStaticSuperTypeID, kStaticTypeID, nullable,
+             minimum_byte_length, maximum_byte_length, parameter) {
+    DCHECK(kLayout != kNonNativeInline || minimum_byte_length == maximum_byte_length);
+  }
+
+  template <TypeStorageLayout layout = kLayout, bool parameterized = kParameterized>
+  TypeSynthesizer(const bool nullable,
+                  const std::size_t minimum_byte_length,
+                  const std::size_t maximum_byte_length,
+                  std::enable_if_t<!parameterized &&
+                                   (layout == kNonNativeInline ||
+                                    layout == kOutOfLine)>* = 0)
+      : Type(kStaticSuperTypeID, kStaticTypeID, nullable,
+             minimum_byte_length, maximum_byte_length) {
+    DCHECK(kLayout != kNonNativeInline || minimum_byte_length == maximum_byte_length);
+  }
+
+ private:
+  template <bool has_param>
+  inline const Type& getInstance(const bool nullable,
+                                 std::enable_if_t<has_param>* = 0) const {
+    return TypeInstance<TypeClass, kParameterized>::Instance(parameter_, nullable);
+  }
+
+  template <bool has_param>
+  inline const Type& getInstance(const bool nullable,
+                                 std::enable_if_t<!has_param>* = 0) const {
+    return TypeInstance<TypeClass, kParameterized>::Instance(nullable);
+  }
+
+  friend class TypeInstance<TypeClass, kParameterized>;
+
+  DISALLOW_COPY_AND_ASSIGN(TypeSynthesizer);
+};
+
+template <TypeID type_id>
+constexpr Type::SuperTypeID TypeSynthesizer<type_id>::kStaticSuperTypeID;
+
+template <TypeID type_id>
+constexpr TypeID TypeSynthesizer<type_id>::kStaticTypeID;
+
+template <TypeID type_id>
+constexpr bool TypeSynthesizer<type_id>::kParameterized;
+
+template <TypeID type_id>
+constexpr TypeStorageLayout TypeSynthesizer<type_id>::kLayout;
+
+
+template <typename TypeClass>
+class TypeInstance<TypeClass, false> {
+ public:
+  static const TypeClass& InstanceNonNullable() {
+    return InstanceInternal<false>();
+  }
+
+  static const TypeClass& InstanceNullable() {
+    return InstanceInternal<true>();
+  }
+
+  static const TypeClass& Instance(const bool nullable) {
+    if (nullable) {
+      return InstanceNullable();
+    } else {
+      return InstanceNonNullable();
+    }
+  }
+
+ private:
+  template <bool nullable>
+  inline static const TypeClass& InstanceInternal() {
+    static TypeClass instance(nullable);
+    return instance;
+  }
+};
+
+template <typename TypeClass>
+class TypeInstance<TypeClass, true> {
+ public:
+  static const TypeClass& InstanceNonNullable(const std::size_t length) {
+    return InstanceInternal<false>(length);
+  }
+
+  static const TypeClass& InstanceNullable(const std::size_t length) {
+    return InstanceInternal<true>(length);
+  }
+
+  static const TypeClass& Instance(const bool nullable, const std::size_t length) {
+    if (nullable) {
+      return InstanceNullable(length);
+    } else {
+      return InstanceNonNullable(length);
+    }
+  }
+
+ private:
+  template <bool nullable>
+  inline static const TypeClass& InstanceInternal(const std::size_t length) {
+    static PtrMap<size_t, TypeClass> instance_map;
+    auto imit = instance_map.find(length);
+    if (imit == instance_map.end()) {
+      imit = instance_map.insert(length, new TypeClass(length, nullable)).first;
+    }
+    return *(imit->second);
+  }
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_TYPE_SYNTHESIZER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/TypeUtil.hpp
----------------------------------------------------------------------
diff --git a/types/TypeUtil.hpp b/types/TypeUtil.hpp
new file mode 100644
index 0000000..b146f02
--- /dev/null
+++ b/types/TypeUtil.hpp
@@ -0,0 +1,70 @@
+/**
+ * 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_TYPES_TYPE_UTIL_HPP_
+#define QUICKSTEP_TYPES_TYPE_UTIL_HPP_
+
+#include <type_traits>
+
+#include "types/BoolType.hpp"
+#include "types/CharType.hpp"
+#include "types/DateType.hpp"
+#include "types/DatetimeIntervalType.hpp"
+#include "types/DatetimeType.hpp"
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/NullType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypeRegistrar.hpp"
+#include "types/VarCharType.hpp"
+#include "types/YearMonthIntervalType.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+class TypeUtil {
+ public:
+  static bool IsParameterized(const TypeID type_id) {
+    return InvokeOnTypeID(
+        type_id,
+        [&](auto tid) -> bool {  // NOLINT(build/c++11)
+      return TypeIDTrait<decltype(tid)::value>::kParameterized;
+    });
+  }
+
+ private:
+  TypeUtil() {}
+
+  DISALLOW_COPY_AND_ASSIGN(TypeUtil);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_TYPE_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/TypedValue.cpp
----------------------------------------------------------------------
diff --git a/types/TypedValue.cpp b/types/TypedValue.cpp
index 8dd8b60..ad1eb0f 100644
--- a/types/TypedValue.cpp
+++ b/types/TypedValue.cpp
@@ -47,6 +47,7 @@ bool TypedValue::isPlausibleInstanceOf(const TypeSignature type) const {
   }
 
   switch (type_id) {
+    case kBool:
     case kInt:
     case kLong:
     case kFloat:
@@ -82,33 +83,34 @@ serialization::TypedValue TypedValue::getProto() const {
 
   // NOTE(chasseur): To represent a NULL value, only the 'type_id' field of the
   // proto is filled in, and all the optional value fields are omitted.
+  proto.mutable_type_id()->CopyFrom(TypeIDFactory::GetProto(getTypeID()));
   switch (getTypeID()) {
+    case kBool:
+      if (!isNull()) {
+        proto.set_int_value(getLiteral<bool>());
+      }
+      break;
     case kInt:
-      proto.set_type_id(serialization::Type::INT);
       if (!isNull()) {
         proto.set_int_value(getLiteral<int>());
       }
       break;
     case kLong:
-      proto.set_type_id(serialization::Type::LONG);
       if (!isNull()) {
         proto.set_long_value(getLiteral<std::int64_t>());
       }
       break;
     case kFloat:
-      proto.set_type_id(serialization::Type::FLOAT);
       if (!isNull()) {
         proto.set_float_value(getLiteral<float>());
       }
       break;
     case kDouble:
-      proto.set_type_id(serialization::Type::DOUBLE);
       if (!isNull()) {
         proto.set_double_value(getLiteral<double>());
       }
       break;
     case kDate:
-      proto.set_type_id(serialization::Type::DATE);
       if (!isNull()) {
         serialization::TypedValue::DateLit *literal_date_proto = proto.mutable_date_value();
         literal_date_proto->set_year(value_union_.date_value.year);
@@ -117,37 +119,31 @@ serialization::TypedValue TypedValue::getProto() const {
       }
       break;
     case kDatetime:
-      proto.set_type_id(serialization::Type::DATETIME);
       if (!isNull()) {
         proto.set_datetime_value(value_union_.datetime_value.ticks);
       }
       break;
     case kDatetimeInterval:
-      proto.set_type_id(serialization::Type::DATETIME_INTERVAL);
       if (!isNull()) {
         proto.set_datetime_interval_value(value_union_.datetime_interval_value.interval_ticks);
       }
       break;
     case kYearMonthInterval:
-      proto.set_type_id(serialization::Type::YEAR_MONTH_INTERVAL);
       if (!isNull()) {
         proto.set_year_month_interval_value(value_union_.year_month_interval_value.months);
       }
       break;
     case kChar:
-      proto.set_type_id(serialization::Type::CHAR);
       if (!isNull()) {
         proto.set_out_of_line_data(static_cast<const char*>(getOutOfLineData()), getDataSize());
       }
       break;
     case kVarChar:
-      proto.set_type_id(serialization::Type::VAR_CHAR);
       if (!isNull()) {
         proto.set_out_of_line_data(static_cast<const char*>(getOutOfLineData()), getDataSize());
       }
       break;
     case kNullType:
-      proto.set_type_id(serialization::Type::NULL_TYPE);
       DCHECK(isNull());
       break;
     default:
@@ -166,24 +162,29 @@ TypedValue TypedValue::ReconstructFromProto(const serialization::TypedValue &pro
       << "Attempted to create TypedValue from an invalid proto description:\n"
       << proto.DebugString();
 
-  switch (proto.type_id()) {
-    case serialization::Type::INT:
+  const TypeID type_id = TypeIDFactory::ReconstructFromProto(proto.type_id());
+  switch (type_id) {
+    case kBool:
+      return proto.has_bool_value() ?
+          TypedValue(static_cast<bool>(proto.bool_value())) :
+          TypedValue(kBool);
+    case kInt:
       return proto.has_int_value() ?
           TypedValue(static_cast<int>(proto.int_value())) :
           TypedValue(kInt);
-    case serialization::Type::LONG:
+    case kLong:
       return proto.has_long_value() ?
           TypedValue(static_cast<std::int64_t>(proto.long_value())) :
           TypedValue(kLong);
-    case serialization::Type::FLOAT:
+    case kFloat:
       return proto.has_float_value() ?
           TypedValue(static_cast<float>(proto.float_value())) :
           TypedValue(kFloat);
-    case serialization::Type::DOUBLE:
+    case kDouble:
       return proto.has_double_value() ?
           TypedValue(static_cast<double>(proto.double_value())) :
           TypedValue(kDouble);
-    case serialization::Type::DATE:
+    case kDate:
       if (proto.has_date_value()) {
         return TypedValue(DateLit::Create(proto.date_value().year(),
                                           proto.date_value().month(),
@@ -191,7 +192,7 @@ TypedValue TypedValue::ReconstructFromProto(const serialization::TypedValue &pro
       } else {
         return TypedValue(kDate);
       }
-    case serialization::Type::DATETIME:
+    case kDatetime:
       if (proto.has_datetime_value()) {
         DatetimeLit datetime;
         datetime.ticks = proto.datetime_value();
@@ -199,7 +200,7 @@ TypedValue TypedValue::ReconstructFromProto(const serialization::TypedValue &pro
       } else {
         return TypedValue(kDatetime);
       }
-    case serialization::Type::DATETIME_INTERVAL:
+    case kDatetimeInterval:
       if (proto.has_datetime_interval_value()) {
         DatetimeIntervalLit interval;
         interval.interval_ticks = proto.datetime_interval_value();
@@ -207,7 +208,7 @@ TypedValue TypedValue::ReconstructFromProto(const serialization::TypedValue &pro
       } else {
         return TypedValue(kDatetimeInterval);
       }
-    case serialization::Type::YEAR_MONTH_INTERVAL:
+    case kYearMonthInterval:
       if (proto.has_year_month_interval_value()) {
         YearMonthIntervalLit interval;
         interval.months = proto.year_month_interval_value();
@@ -215,19 +216,19 @@ TypedValue TypedValue::ReconstructFromProto(const serialization::TypedValue &pro
       } else {
         return TypedValue(kYearMonthInterval);
       }
-    case serialization::Type::CHAR:
+    case kChar:
       return proto.has_out_of_line_data() ?
           TypedValue(kChar,
                      static_cast<const void*>(proto.out_of_line_data().c_str()),
                      proto.out_of_line_data().size()).ensureNotReference() :
           TypedValue(kChar);
-    case serialization::Type::VAR_CHAR:
+    case kVarChar:
       return proto.has_out_of_line_data() ?
           TypedValue(kVarChar,
                      static_cast<const void*>(proto.out_of_line_data().c_str()),
                      proto.out_of_line_data().size()).ensureNotReference() :
           TypedValue(kVarChar);
-    case serialization::Type::NULL_TYPE:
+    case kNullType:
       return TypedValue(kNullType);
     default:
       FATAL_ERROR("Unrecognized TypeID in TypedValue::ReconstructFromProto");

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/TypedValue.hpp
----------------------------------------------------------------------
diff --git a/types/TypedValue.hpp b/types/TypedValue.hpp
index 0ba3d53..196e8ec 100644
--- a/types/TypedValue.hpp
+++ b/types/TypedValue.hpp
@@ -90,6 +90,16 @@ class TypedValue {
   }
 
   /**
+   * @brief Constructor for a literal value of BoolType.
+   **/
+  explicit TypedValue(const bool literal_bool)
+      : value_info_(static_cast<std::uint64_t>(kBool)) {
+    // Zero-out all bytes in the union for getHash() and fastEqualCheck().
+    value_union_.hash64 = 0;
+    value_union_.bool_value = literal_bool;
+  }
+
+  /**
    * @brief Constructor for a literal value of IntType.
    **/
   explicit TypedValue(const int literal_int)
@@ -264,9 +274,9 @@ class TypedValue {
    *        TypedValue will take ownership of this memory.
    * @param value_size The number of bytes of data at value_ptr.
    **/
-  static TypedValue CreateWithOwnedData(const TypeID type_id,
-                                        void *value_ptr,
-                                        const std::size_t value_size) {
+  inline static TypedValue CreateWithOwnedData(const TypeID type_id,
+                                               void *value_ptr,
+                                               const std::size_t value_size) {
     TypedValue val(type_id, value_ptr, value_size);
     val.value_info_ |= kOwnershipMask;
     return val;
@@ -282,6 +292,7 @@ class TypedValue {
    **/
   static bool RepresentedInline(const TypeID type_id) {
     switch (type_id) {
+      case kBool:
       case kInt:
       case kLong:
       case kFloat:
@@ -313,6 +324,8 @@ class TypedValue {
    **/
   static bool HashIsReversible(const TypeID type_id) {
     switch (type_id) {
+      case kBool:
+        return true;
       case kInt:
       case kFloat:
         return sizeof(value_union_.int_value) <= sizeof(std::size_t);
@@ -391,6 +404,8 @@ class TypedValue {
   inline std::size_t getDataSize() const {
     DCHECK(!isNull());
     switch (getTypeID()) {
+      case kBool:
+        return sizeof(bool);
       case kInt:
       case kFloat:
         return sizeof(int);
@@ -478,7 +493,8 @@ class TypedValue {
    * @return The out-of-line data this TypedValue points to.
    **/
   inline const void* getOutOfLineData() const {
-    DCHECK(!(getTypeID() == kInt
+    DCHECK(!(getTypeID() == kBool
+                   || getTypeID() == kInt
                    || getTypeID() == kLong
                    || getTypeID() == kFloat
                    || getTypeID() == kDouble
@@ -547,6 +563,10 @@ class TypedValue {
                   value_info_ >> kSizeShift);
     } else {
       switch (getTypeID()) {
+        case kBool:
+          // 1 byte copy.
+          *static_cast<bool*>(destination) = value_union_.bool_value;
+          break;
         case kInt:
         case kFloat:
           // 4 bytes byte-for-byte copy.
@@ -574,6 +594,7 @@ class TypedValue {
    **/
   inline std::size_t getHash() const {
     switch (getTypeID()) {
+      case kBool:
       case kInt:
       case kLong:
       case kFloat:
@@ -670,6 +691,7 @@ class TypedValue {
     DCHECK(!other.isNull());
     DCHECK_EQ(getTypeID(), other.getTypeID());
     switch (getTypeID()) {
+      case kBool:
       case kInt:
       case kLong:
       case kFloat:
@@ -710,6 +732,16 @@ class TypedValue {
     }
   }
 
+  bool operator==(const TypedValue &other) const {
+    if (getTypeID() != other.getTypeID()) {
+      return false;
+    }
+    if (isNull() || other.isNull()) {
+      return isNull() == other.isNull();
+    }
+    return fastEqualCheck(other);
+  }
+
   /**
    * @brief Generate a serialized Protocol Buffer representation
    *        of this TypedValue.
@@ -789,6 +821,7 @@ class TypedValue {
   inline void reverseHash(const std::size_t hash);
 
   union ValueUnion {
+    bool bool_value;
     int int_value;
     std::int64_t long_value;
     float float_value;
@@ -842,6 +875,13 @@ class TypedValue {
 
 // Explicit specializations of getLiteral().
 template <>
+inline bool TypedValue::getLiteral<bool>() const {
+  DCHECK_EQ(kBool, getTypeID());
+  DCHECK(!isNull());
+  return value_union_.bool_value;
+}
+
+template <>
 inline int TypedValue::getLiteral<int>() const {
   DCHECK_EQ(kInt, getTypeID());
   DCHECK(!isNull());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/TypedValue.proto
----------------------------------------------------------------------
diff --git a/types/TypedValue.proto b/types/TypedValue.proto
index 7f3ab7a..7cf3eca 100644
--- a/types/TypedValue.proto
+++ b/types/TypedValue.proto
@@ -22,17 +22,18 @@ package quickstep.serialization;
 import "types/Type.proto";
 
 message TypedValue {
-  required Type.TypeID type_id = 1;
+  required TypeID type_id = 1;
 
   // NOTE(zuyu): For a NULL value, none of the optional fields are filled in.
-  optional int32 int_value = 2;
-  optional int64 long_value = 3;
-  optional float float_value = 4;
-  optional double double_value = 5;
-  optional bytes out_of_line_data = 6;
-  optional int64 datetime_value = 7;
-  optional int64 datetime_interval_value = 8;
-  optional int64 year_month_interval_value = 9;
+  optional bool bool_value = 2;
+  optional int32 int_value = 3;
+  optional int64 long_value = 4;
+  optional float float_value = 5;
+  optional double double_value = 6;
+  optional bytes out_of_line_data = 7;
+  optional int64 datetime_value = 8;
+  optional int64 datetime_interval_value = 9;
+  optional int64 year_month_interval_value = 10;
 
   message DateLit {
     required int32 year = 1;
@@ -40,5 +41,5 @@ message TypedValue {
     required uint32 day = 3;
   }
 
-  optional DateLit date_value = 10;
+  optional DateLit date_value = 11;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/VarCharType.cpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.cpp b/types/VarCharType.cpp
index 02845b1..7eeb04c 100644
--- a/types/VarCharType.cpp
+++ b/types/VarCharType.cpp
@@ -42,38 +42,6 @@ using std::string;
 
 namespace quickstep {
 
-template <bool nullable_internal>
-const VarCharType& VarCharType::InstanceInternal(const std::size_t length) {
-  static PtrMap<size_t, VarCharType> instance_map;
-  PtrMap<size_t, VarCharType>::iterator imit = instance_map.find(length);
-  if (imit == instance_map.end()) {
-    imit = instance_map.insert(length, new VarCharType(length, nullable_internal)).first;
-  }
-  return *(imit->second);
-}
-
-const VarCharType& VarCharType::InstanceNonNullable(const std::size_t length) {
-  return InstanceInternal<false>(length);
-}
-
-const VarCharType& VarCharType::InstanceNullable(const std::size_t length) {
-  return InstanceInternal<true>(length);
-}
-
-const VarCharType& VarCharType::InstanceFromProto(const serialization::Type &proto) {
-  return Instance(proto.GetExtension(serialization::VarCharType::length), proto.nullable());
-}
-
-serialization::Type VarCharType::getProto() const {
-  serialization::Type proto;
-  proto.set_type_id(serialization::Type::VAR_CHAR);
-
-  proto.set_nullable(nullable_);
-
-  proto.SetExtension(serialization::VarCharType::length, length_);
-  return proto;
-}
-
 size_t VarCharType::estimateAverageByteLength() const {
   if (length_ > 160) {
     return 80;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/VarCharType.hpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.hpp b/types/VarCharType.hpp
index bb50e92..05b2aae 100644
--- a/types/VarCharType.hpp
+++ b/types/VarCharType.hpp
@@ -24,8 +24,8 @@
 #include <cstdio>
 #include <string>
 
+#include "types/AsciiStringSuperType.hpp"
 #include "types/Type.hpp"
-#include "types/Type.pb.h"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
@@ -43,72 +43,9 @@ namespace quickstep {
  *       character. This means that the VARCHAR(X) type requires from 1 to X+1
  *       bytes of storage, depending on string length.
  **/
-class VarCharType : public AsciiStringSuperType {
+class VarCharType : public AsciiStringSuperType<kVarChar> {
  public:
   /**
-   * @brief Get a reference to the non-nullable singleton instance of this Type
-   *        for the specified length.
-   *
-   * @param length The length parameter of the VarCharType.
-   * @return A reference to the non-nullable singleton instance of this Type
-   *         for the specified length.
-   **/
-  static const VarCharType& InstanceNonNullable(const std::size_t length);
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type for
-   *        the specified length.
-   *
-   * @param length The length parameter of the VarCharType.
-   * @return A reference to the nullable singleton instance of this Type for
-   *         the specified length.
-   **/
-  static const VarCharType& InstanceNullable(const std::size_t length);
-
-  /**
-   * @brief Get a reference to the singleton instance of this Type for the
-   *        specified length and nullability.
-   *
-   * @param length The length parameter of the VarCharType.
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the singleton instance of this Type for the
-   *         specified length and nullability.
-   **/
-  static const VarCharType& Instance(const std::size_t length, const bool nullable) {
-    if (nullable) {
-      return InstanceNullable(length);
-    } else {
-      return InstanceNonNullable(length);
-    }
-  }
-
-  /**
-   * @brief Get a reference to the singleton instance of this Type described
-   *        by the given Protocol Buffer serialization.
-   *
-   * @param type The serialized Protocol Buffer representation of the desired
-   *        VarCharType.
-   * @return A reference to the singleton instance of this Type for the given
-   *         Protocol Buffer.
-   **/
-  static const VarCharType& InstanceFromProto(const serialization::Type &type);
-
-  /**
-   * @brief Generate a serialized Protocol Buffer representation of this Type.
-   *
-   * @return The serialized Protocol Buffer representation of this Type.
-   **/
-  serialization::Type getProto() const override;
-
-  const Type& getNullableVersion() const override {
-    return InstanceNullable(length_);
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable(length_);
-  }
-
-  /**
    * @note Includes an extra byte for a terminating null character.
    **/
   std::size_t estimateAverageByteLength() const override;
@@ -137,11 +74,9 @@ class VarCharType : public AsciiStringSuperType {
 
  private:
   VarCharType(const std::size_t length, const bool nullable)
-      : AsciiStringSuperType(kVarChar, nullable, 1, length + 1, length) {
-  }
+      : AsciiStringSuperType<kVarChar>(nullable, 1, length + 1, length) {}
 
-  template <bool nullable_internal>
-  static const VarCharType& InstanceInternal(const std::size_t length);
+  template <typename, bool> friend class TypeInstance;
 
   DISALLOW_COPY_AND_ASSIGN(VarCharType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/YearMonthIntervalType.cpp
----------------------------------------------------------------------
diff --git a/types/YearMonthIntervalType.cpp b/types/YearMonthIntervalType.cpp
index 3c15a91..d656fca 100644
--- a/types/YearMonthIntervalType.cpp
+++ b/types/YearMonthIntervalType.cpp
@@ -30,7 +30,6 @@
 
 #include "types/IntervalLit.hpp"
 #include "types/IntervalParser.hpp"
-#include "types/NullCoercibilityCheckMacro.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
@@ -46,16 +45,6 @@ using std::snprintf;
 
 namespace quickstep {
 
-bool YearMonthIntervalType::isCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kYearMonthInterval);
-}
-
-bool YearMonthIntervalType::isSafelyCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kYearMonthInterval);
-}
-
 std::string YearMonthIntervalType::printValueToString(const TypedValue &value) const {
   DCHECK(!value.isNull());
 
@@ -127,14 +116,6 @@ std::string YearMonthIntervalType::printValueToString(const TypedValue &value) c
   return std::string(interval_buf);
 }
 
-void YearMonthIntervalType::printValueToFile(const TypedValue &value,
-                                             FILE *file,
-                                             const int padding) const {
-  // We simply re-use the logic from printValueToString(), as trying to do
-  // padding on-the fly with so many different fields is too much of a hassle.
-  std::fprintf(file, "%*s", static_cast<int>(padding), printValueToString(value).c_str());
-}
-
 bool YearMonthIntervalType::parseValueFromString(const std::string &value_string,
                                                  TypedValue *value) const {
   // Try simple-format parse first.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/YearMonthIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/YearMonthIntervalType.hpp b/types/YearMonthIntervalType.hpp
index a2ba175..e890ea9 100644
--- a/types/YearMonthIntervalType.hpp
+++ b/types/YearMonthIntervalType.hpp
@@ -27,6 +27,7 @@
 #include "types/IntervalLit.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
+#include "types/TypeSynthesizer.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
@@ -39,73 +40,14 @@ namespace quickstep {
 /**
  * @brief A type representing the year-month interval.
  **/
-class YearMonthIntervalType : public Type {
+class YearMonthIntervalType : public TypeSynthesizer<kYearMonthInterval> {
  public:
-  typedef YearMonthIntervalLit cpptype;
-
-  static const TypeID kStaticTypeID = kYearMonthInterval;
-
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const YearMonthIntervalType& InstanceNonNullable() {
-    static YearMonthIntervalType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const YearMonthIntervalType& InstanceNullable() {
-    static YearMonthIntervalType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const YearMonthIntervalType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
-  std::size_t estimateAverageByteLength() const override {
-    return sizeof(YearMonthIntervalLit);
-  }
-
-  bool isCoercibleFrom(const Type &original_type) const override;
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override;
-
   int getPrintWidth() const override {
     return YearMonthIntervalLit::kPrintingChars;
   }
 
   std::string printValueToString(const TypedValue &value) const override;
 
-  void printValueToFile(const TypedValue &value,
-                        FILE *file,
-                        const int padding = 0) const override;
-
   TypedValue makeZeroValue() const override {
     return TypedValue(YearMonthIntervalLit{0});
   }
@@ -115,8 +57,9 @@ class YearMonthIntervalType : public Type {
 
  private:
   explicit YearMonthIntervalType(const bool nullable)
-      : Type(Type::kOther, kYearMonthInterval, nullable, sizeof(YearMonthIntervalLit), sizeof(YearMonthIntervalLit)) {
-  }
+      : TypeSynthesizer<kYearMonthInterval>(nullable) {}
+
+  template <typename, bool> friend class TypeInstance;
 
   DISALLOW_COPY_AND_ASSIGN(YearMonthIntervalType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/containers/ColumnVector.cpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.cpp b/types/containers/ColumnVector.cpp
index dfc0fae..ef3587e 100644
--- a/types/containers/ColumnVector.cpp
+++ b/types/containers/ColumnVector.cpp
@@ -41,4 +41,8 @@ ColumnVector* ColumnVector::MakeVectorOfValue(
   }
 }
 
+constexpr bool NativeColumnVector::kNative;
+
+constexpr bool IndirectColumnVector::kNative;
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/containers/ColumnVector.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.hpp b/types/containers/ColumnVector.hpp
index 5ef9871..0d6447d 100644
--- a/types/containers/ColumnVector.hpp
+++ b/types/containers/ColumnVector.hpp
@@ -129,6 +129,8 @@ class ColumnVector {
  **/
 class NativeColumnVector : public ColumnVector {
  public:
+  static constexpr bool kNative = true;
+
   /**
    * @brief Constructor for a NativeColumnVector which owns its own array of
    *        values.
@@ -140,8 +142,8 @@ class NativeColumnVector : public ColumnVector {
   NativeColumnVector(const Type &type, const std::size_t reserved_length)
       : ColumnVector(type),
         type_length_(type.maximumByteLength()),
-        values_(std::malloc(type.maximumByteLength() * reserved_length)),
         reserved_length_(reserved_length),
+        values_(std::malloc(type.maximumByteLength() * reserved_length)),
         actual_length_(0u),
         null_bitmap_(type.isNullable() ? new BitVector<false>(reserved_length) : nullptr) {
     DCHECK(UsableForType(type_));
@@ -395,8 +397,9 @@ class NativeColumnVector : public ColumnVector {
 
  private:
   const std::size_t type_length_;
-  void *values_;
   const std::size_t reserved_length_;
+
+  void *values_;
   std::size_t actual_length_;
   std::unique_ptr<BitVector<false>> null_bitmap_;
 
@@ -409,6 +412,8 @@ class NativeColumnVector : public ColumnVector {
  **/
 class IndirectColumnVector : public ColumnVector {
  public:
+  static constexpr bool kNative = false;
+
   /**
    * @brief Constructor.
    *
@@ -503,11 +508,21 @@ class IndirectColumnVector : public ColumnVector {
    * @param value A value to append to this NativeColumnVector.
    **/
   inline void appendTypedValue(TypedValue &&value) {
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+    DCHECK(value.isPlausibleInstanceOf(type_.getSignature())) << type_.getName();
     DCHECK_LT(values_.size(), reserved_length_);
     values_.emplace_back(std::move(value));
   }
 
+  inline void appendNullValue() {
+    DCHECK(type_.isNullable());
+    DCHECK_LT(values_.size(), reserved_length_);
+    values_.emplace_back(type_.makeNullValue());
+  }
+
+  inline void fillWithNulls() {
+    fillWithValue(type_.makeNullValue());
+  }
+
   /**
    * @brief Fill this entire ColumnVector with copies of value.
    *
@@ -569,6 +584,7 @@ class IndirectColumnVector : public ColumnVector {
  private:
   const bool type_is_nullable_;
   const std::size_t reserved_length_;
+
   std::vector<TypedValue> values_;
 
   DISALLOW_COPY_AND_ASSIGN(IndirectColumnVector);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/CMakeLists.txt b/types/operations/CMakeLists.txt
index c5dad0f..948d013 100644
--- a/types/operations/CMakeLists.txt
+++ b/types/operations/CMakeLists.txt
@@ -25,19 +25,62 @@ QS_PROTOBUF_GENERATE_CPP(types_operations_Operation_proto_srcs
 
 # Declare micro-libs:
 add_library(quickstep_types_operations_Operation Operation.cpp Operation.hpp)
+add_library(quickstep_types_operations_OperationFactory OperationFactory.cpp OperationFactory.hpp)
+add_library(quickstep_types_operations_OperationUtil ../../empty_src.cpp OperationUtil.hpp)
+add_library(quickstep_types_operations_OperationSignature OperationSignature.cpp OperationSignature.hpp)
 add_library(quickstep_types_operations_Operation_proto ${types_operations_Operation_proto_srcs})
 
 # Link dependencies:
 target_link_libraries(quickstep_types_operations_Operation
+                      quickstep_types_operations_OperationSignature
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_types_operations_OperationFactory
+                      quickstep_types_Type
+                      quickstep_types_TypeFactory
+                      quickstep_types_TypeID
+                      quickstep_types_TypeUtil
+                      quickstep_types_TypedValue
+                      quickstep_types_operations_Operation
+                      quickstep_types_operations_OperationSignature
+                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperations
+                      quickstep_types_operations_binaryoperations_AsciiStringBinaryOperations
+                      quickstep_types_operations_binaryoperations_BinaryOperation
+                      quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+                      quickstep_types_operations_binaryoperations_CMathBinaryOperations
+                      quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
+                      quickstep_types_operations_unaryoperations_AsciiStringUnaryOperations
+                      quickstep_types_operations_unaryoperations_CMathUnaryOperations
+                      quickstep_types_operations_unaryoperations_CastOperation
+                      quickstep_types_operations_unaryoperations_DateExtractOperation
+                      quickstep_types_operations_unaryoperations_SubstringOperation
+                      quickstep_types_operations_unaryoperations_UnaryOperation
+                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper
+                      quickstep_utility_HashPair
+                      quickstep_utility_Macros
+                      quickstep_utility_StringUtil)
+target_link_libraries(quickstep_types_operations_OperationUtil
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_types_Type
+                      quickstep_types_TypedValue
+                      quickstep_types_containers_ColumnVector)
+target_link_libraries(quickstep_types_operations_OperationSignature
+                      quickstep_types_TypeID
+                      quickstep_types_Type_proto
+                      quickstep_types_operations_Operation_proto
+                      quickstep_utility_HashPair
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_types_operations_Operation_proto
                       quickstep_types_Type_proto
+                      quickstep_types_TypedValue_proto
                       ${PROTOBUF_LIBRARY})
 
 # Module all-in-one library:
 add_library(quickstep_types_operations ../../empty_src.cpp)
 target_link_libraries(quickstep_types_operations
                       quickstep_types_operations_Operation
+                      quickstep_types_operations_OperationFactory
+                      quickstep_types_operations_OperationUtil
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_binaryoperations
                       quickstep_types_operations_comparisons

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/Operation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/Operation.hpp b/types/operations/Operation.hpp
index 51178b5..6da0f4c 100644
--- a/types/operations/Operation.hpp
+++ b/types/operations/Operation.hpp
@@ -20,6 +20,10 @@
 #ifndef QUICKSTEP_TYPES_OPERATIONS_OPERATION_HPP_
 #define QUICKSTEP_TYPES_OPERATIONS_OPERATION_HPP_
 
+#include <string>
+#include <vector>
+
+#include "types/operations/OperationSignature.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -28,6 +32,9 @@ namespace quickstep {
  *  @{
  */
 
+class Operation;
+typedef std::shared_ptr<const Operation> OperationPtr;
+
 /**
  * @brief An operation which can be applied to typed values. Each exact
  *        concrete Operation is a singleton.
@@ -72,7 +79,7 @@ class Operation {
    * @return The human-readable name of this Operation.
    **/
   inline const char* getName() const {
-    return name_;
+    return "NoName";
   }
 
   /**
@@ -81,7 +88,11 @@ class Operation {
    * @return The short name of this Operation.
    **/
   inline const char* getShortName() const {
-    return short_name_;
+    return "NoShortName";
+  }
+
+  virtual std::vector<OperationSignaturePtr> getSignatures() const {
+    return {};
   }
 
   /**
@@ -98,19 +109,12 @@ class Operation {
   }
 
  protected:
-  Operation(const OperationSuperTypeID super_type_id,
-            const char *name,
-            const char *short_name)
-      : super_type_id_(super_type_id),
-        name_(name),
-        short_name_(short_name) {
+  explicit Operation(const OperationSuperTypeID super_type_id)
+      : super_type_id_(super_type_id) {
   }
 
  private:
   const OperationSuperTypeID super_type_id_;
-  const char *name_;
-  const char *short_name_;
-
 
   DISALLOW_COPY_AND_ASSIGN(Operation);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/Operation.proto
----------------------------------------------------------------------
diff --git a/types/operations/Operation.proto b/types/operations/Operation.proto
index d6391f0..da2a282 100644
--- a/types/operations/Operation.proto
+++ b/types/operations/Operation.proto
@@ -20,6 +20,7 @@ syntax = "proto2";
 package quickstep.serialization;
 
 import "types/Type.proto";
+import "types/TypedValue.proto";
 
 message Comparison {
   enum ComparisonID {
@@ -38,58 +39,8 @@ message Comparison {
   required ComparisonID comparison_id = 1;
 }
 
-message UnaryOperation {
-  enum UnaryOperationID {
-    NEGATE = 0;
-    CAST = 1;
-    DATE_EXTRACT = 2;
-    SUBSTRING = 3;
-  }
-
-  required UnaryOperationID operation_id = 1;
-
-  extensions 32 to max;
-}
-
-message CastOperation {
-  extend UnaryOperation {
-    // Required when operation_id = CAST.
-    optional Type target_type = 64;
-  }
-}
-
-message DateExtractOperation {
-  enum Unit {
-    YEAR = 0;
-    MONTH = 1;
-    DAY = 2;
-    HOUR = 3;
-    MINUTE = 4;
-    SECOND = 5;
-  }
-
-  extend UnaryOperation {
-    // Required when operation_id = DATE_EXTRACT.
-    optional Unit unit = 96;
-  }
-}
-
-message SubstringOperation {
-  extend UnaryOperation {
-    // Required when operation_id = SUBSTRING.
-    optional int64 start_position = 100;
-    optional int64 substring_length = 101;
-  }
-}
-
-message BinaryOperation {
-  enum BinaryOperationID {
-    ADD = 0;
-    SUBTRACT = 1;
-    MULTIPLY = 2;
-    DIVIDE = 3;
-    MODULO = 4;
-  }
-
-  required BinaryOperationID operation_id = 1;
+message OperationSignature {
+  required string operation_name = 1;
+  repeated TypeID argument_type_ids = 2;
+  required uint32 num_static_arguments = 3;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/OperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationFactory.cpp b/types/operations/OperationFactory.cpp
new file mode 100644
index 0000000..531318b
--- /dev/null
+++ b/types/operations/OperationFactory.cpp
@@ -0,0 +1,357 @@
+/**
+ * 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 "types/operations/OperationFactory.hpp"
+
+#include <list>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypeUtil.hpp"
+#include "types/TypedValue.hpp"
+#include "types/operations/Operation.hpp"
+#include "types/operations/OperationSignature.hpp"
+#include "types/operations/binary_operations/ArithmeticBinaryOperations.hpp"
+#include "types/operations/binary_operations/AsciiStringBinaryOperations.hpp"
+#include "types/operations/binary_operations/BinaryOperationWrapper.hpp"
+#include "types/operations/binary_operations/CMathBinaryOperations.hpp"
+#include "types/operations/unary_operations/ArithmeticUnaryOperations.hpp"
+#include "types/operations/unary_operations/AsciiStringUnaryOperations.hpp"
+#include "types/operations/unary_operations/CMathUnaryOperations.hpp"
+#include "types/operations/unary_operations/CastOperation.hpp"
+#include "types/operations/unary_operations/DateExtractOperation.hpp"
+#include "types/operations/unary_operations/SubstringOperation.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
+#include "utility/StringUtil.hpp"
+
+namespace quickstep {
+
+namespace {
+
+struct FunctorPackDispatcher {
+  template <typename FunctorT>
+  inline static std::list<OperationPtr> Generate(
+      std::enable_if_t<FunctorT::kOperationSuperTypeID == Operation::kUnaryOperation>* = 0) {
+    return { std::make_shared<const UnaryOperationWrapper<FunctorT>>() };
+  }
+
+  template <typename FunctorT>
+  inline static std::list<OperationPtr> Generate(
+      std::enable_if_t<FunctorT::kOperationSuperTypeID == Operation::kBinaryOperation>* = 0) {
+    return { std::make_shared<const BinaryOperationWrapper<FunctorT>>() };
+  }
+
+  template <typename FunctorT>
+  inline static std::list<OperationPtr> Generate(
+      decltype(FunctorT::template GenerateOperations<FunctorPackDispatcher>())* = 0) {
+    return FunctorT::template GenerateOperations<FunctorPackDispatcher>();
+  }
+};
+
+}  // namespace
+
+OperationFactory::OperationFactory() {
+  registerOperation<CastOperation>();
+  registerOperation<DateExtractOperation>();
+  registerOperation<SubstringOperation>();
+
+  registerFunctorPack<ArithmeticUnaryFunctorPack>();
+  registerFunctorPack<AsciiStringUnaryFunctorPack>();
+  registerFunctorPack<CMathUnaryFunctorPack>();
+
+  registerFunctorPack<ArithmeticBinaryFunctorPack>();
+  registerFunctorPack<AsciiStringBinaryFunctorPack>();
+  registerFunctorPack<CMathBinaryFunctorPack>();
+}
+
+OperationSignaturePtr OperationFactory::resolveOperation(
+    const std::string &operation_name,
+    const std::shared_ptr<const std::vector<const Type*>> &argument_types,
+    const std::shared_ptr<const std::vector<TypedValue>> &static_arguments,
+    std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
+    std::shared_ptr<const std::vector<TypedValue>> *coerced_static_arguments,
+    std::string *message) const {
+  const std::string lower_case_name = ToLower(operation_name);
+  const std::size_t arity = argument_types->size();
+  const auto &indices_it =
+      primary_index_.find(std::make_pair(lower_case_name, arity));
+
+  if (indices_it == primary_index_.end()) {
+    *message = "Unrecognized function " + operation_name
+                   + " with " + std::to_string(arity) + " arguments";
+  }
+
+  ResolveStatus status;
+  OperationSignaturePtr op_signature = nullptr;
+  const auto &secondary_index = indices_it->second;
+
+  std::vector<TypeID> argument_type_ids;
+  for (const auto *type : *argument_types) {
+    argument_type_ids.emplace_back(type->getTypeID());
+  }
+
+  // First, try full exact matching.
+  status = resolveOperationWithFullTypeMatch(secondary_index,
+                                             argument_type_ids,
+                                             *argument_types,
+                                             *static_arguments,
+                                             coerced_static_arguments,
+                                             &op_signature,
+                                             message);
+  if (status == ResolveStatus::kSuccess) {
+    DCHECK(op_signature != nullptr);
+    *coerced_argument_types = argument_types;
+    return op_signature;
+  } else if (status == ResolveStatus::kError) {
+    return nullptr;
+  }
+
+  // Otherwise, try partial (non-static arguments) exact matching.
+  status = resolveOperationWithPartialTypeMatch(secondary_index,
+                                                argument_type_ids,
+                                                *argument_types,
+                                                *static_arguments,
+                                                coerced_argument_types,
+                                                coerced_static_arguments,
+                                                &op_signature,
+                                                message);
+  if (status == ResolveStatus::kSuccess) {
+    DCHECK(op_signature != nullptr);
+    return op_signature;
+  } else if (status == ResolveStatus::kError) {
+    return nullptr;
+  }
+
+  // TODO
+  *message = "Unexpected argument types for function " + operation_name;
+  return nullptr;
+}
+
+OperationFactory::ResolveStatus OperationFactory::resolveOperationWithFullTypeMatch(
+    const PartialSignatureIndex &secondary_index,
+    const std::vector<TypeID> &argument_type_ids,
+    const std::vector<const Type*> &argument_types,
+    const std::vector<TypedValue> &static_arguments,
+    std::shared_ptr<const std::vector<TypedValue>> *partial_static_arguments,
+    OperationSignaturePtr *resolved_op_signature,
+    std::string *message) const {
+  const std::size_t max_num_static_arguments = static_arguments.size();
+  auto it = secondary_index.lower_bound(
+      std::make_pair(&argument_type_ids, max_num_static_arguments));
+
+  if (it != secondary_index.end() && *it->first.first == argument_type_ids) {
+    const OperationSignaturePtr op_signature = it->second;
+    const OperationPtr operation = getOperation(op_signature);
+
+    *partial_static_arguments =
+        std::make_shared<const std::vector<TypedValue>>(
+            static_arguments.begin()
+                + (max_num_static_arguments - op_signature->getNumStaticArguments()),
+            static_arguments.end());
+
+    if (canApplyOperationTo(operation,
+                            argument_types,
+                            **partial_static_arguments,
+                            message)) {
+      *resolved_op_signature = op_signature;
+      return ResolveStatus::kSuccess;
+    } else {
+      return ResolveStatus::kError;
+    }
+  }
+
+  return ResolveStatus::kNotFound;
+}
+
+OperationFactory::ResolveStatus OperationFactory::resolveOperationWithPartialTypeMatch(
+    const PartialSignatureIndex &secondary_index,
+    const std::vector<TypeID> &argument_type_ids,
+    const std::vector<const Type*> &argument_types,
+    const std::vector<TypedValue> &static_arguments,
+    std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
+    std::shared_ptr<const std::vector<TypedValue>> *coerced_static_arguments,
+    OperationSignaturePtr *resolved_op_signature,
+    std::string *message) const {
+  const std::size_t arity = argument_types.size();
+  const std::size_t max_num_static_arguments = static_arguments.size();
+  const std::size_t first_static_argument_position = arity - max_num_static_arguments;
+
+  auto it = secondary_index.lower_bound(
+      std::make_pair(&argument_type_ids, max_num_static_arguments));
+  while (it != secondary_index.end()) {
+    const std::vector<TypeID> &expected_type_ids = *it->first.first;
+    DCHECK_GE(expected_type_ids.size(), it->first.second);
+    const std::size_t num_non_static_arguments =
+        expected_type_ids.size() - it->first.second;
+
+    if (!std::equal(expected_type_ids.begin(),
+                    expected_type_ids.begin() + num_non_static_arguments,
+                    argument_type_ids.begin())) {
+      break;
+    }
+
+    // Coerce static arguments
+    std::vector<const Type*> coerced_static_arg_types;
+    std::vector<TypedValue> coerced_static_args;
+
+    bool is_coercible = true;
+    for (std::size_t i = num_non_static_arguments; i < arity; ++i) {
+      const Type &arg_type = *argument_types.at(i);
+      const TypedValue &arg_value =
+          static_arguments.at(i - first_static_argument_position);
+      const TypeID &expected_type_id = expected_type_ids.at(i);
+
+      if (arg_type.getTypeID() == expected_type_id) {
+        coerced_static_arg_types.emplace_back(&arg_type);
+        coerced_static_args.emplace_back(arg_value);
+      } else {
+        const Type *expected_type = nullptr;
+        if (TypeFactory::TypeRequiresLengthParameter(expected_type_id)) {
+          // TODO: refactor type system to make this coercion extensible.
+          if (expected_type_id == kChar && arg_type.getTypeID() == kVarChar) {
+            expected_type = &TypeFactory::GetType(
+                expected_type_id, arg_type.maximumByteLength() - 1);
+          } else if (expected_type_id == kVarChar && arg_type.getTypeID() == kChar) {
+            expected_type = &TypeFactory::GetType(
+                expected_type_id, arg_type.maximumByteLength() + 1);
+          }
+        } else {
+          expected_type = &TypeFactory::GetType(expected_type_id);
+        }
+
+        if (expected_type != nullptr && expected_type->isSafelyCoercibleFrom(arg_type)) {
+          coerced_static_arg_types.emplace_back(expected_type);
+          coerced_static_args.emplace_back(
+              expected_type->coerceValue(arg_value, arg_type));
+        } else {
+          is_coercible = false;
+          break;
+        }
+      }
+    }
+
+    if (is_coercible) {
+      std::vector<const Type*> coerced_arg_types(
+          argument_types.begin(),
+          argument_types.begin() + num_non_static_arguments);
+      for (const Type *type : coerced_static_arg_types) {
+        coerced_arg_types.emplace_back(type);
+      }
+
+      const OperationPtr operation = getOperation(it->second);
+      if (canApplyOperationTo(operation,
+                              coerced_arg_types,
+                              coerced_static_args,
+                              message)) {
+        *coerced_argument_types =
+            std::make_shared<const std::vector<const Type*>>(std::move(coerced_arg_types));
+        *coerced_static_arguments =
+            std::make_shared<const std::vector<TypedValue>>(std::move(coerced_static_args));
+        *resolved_op_signature = it->second;
+        return ResolveStatus::kSuccess;
+      }
+    }
+
+    ++it;
+  }
+
+  return ResolveStatus::kNotFound;
+}
+
+bool OperationFactory::canApplyOperationTo(
+    const OperationPtr operation,
+    const std::vector<const Type*> &argument_types,
+    const std::vector<TypedValue> &static_arguments,
+    std::string *message) const {
+  switch (operation->getOperationSuperTypeID()) {
+    case Operation::kUnaryOperation: {
+      const UnaryOperationPtr unary_operation =
+          std::static_pointer_cast<const UnaryOperation>(operation);
+      return unary_operation->canApplyTo(*argument_types[0],
+                                         static_arguments,
+                                         message);
+    }
+    case Operation::kBinaryOperation: {
+      const BinaryOperationPtr binary_operation =
+          std::static_pointer_cast<const BinaryOperation>(operation);
+      return binary_operation->canApplyTo(*argument_types[0],
+                                          *argument_types[1],
+                                          static_arguments,
+                                          message);
+    }
+    default: {
+      const auto operation_id =
+         static_cast<std::underlying_type_t<Operation::OperationSuperTypeID>>(
+             operation->getOperationSuperTypeID());
+      LOG(FATAL) << "Unknown opeation super type id: " << operation_id;
+    }
+  }
+}
+
+
+const OperationFactory& OperationFactory::Instance() {
+  static OperationFactory instance;
+  return instance;
+}
+
+template <typename OperationT>
+void OperationFactory::registerOperation() {
+  registerOperationInternal(std::make_shared<const OperationT>());
+}
+
+template <typename FunctorPackT>
+void OperationFactory::registerFunctorPack() {
+  for (const OperationPtr &operation :
+           FunctorPackT::template GenerateOperations<FunctorPackDispatcher>()) {
+    registerOperationInternal(operation);
+  }
+}
+
+void OperationFactory::registerOperationInternal(const OperationPtr &operation) {
+  DCHECK(operation->getOperationSuperTypeID() == Operation::kUnaryOperation ||
+         operation->getOperationSuperTypeID() == Operation::kBinaryOperation);
+
+  for (const OperationSignaturePtr op_sig_orig : operation->getSignatures()) {
+    DCHECK(operation->getOperationSuperTypeID() != Operation::kUnaryOperation ||
+           op_sig_orig->getNonStaticArity() == 1u);
+    DCHECK(operation->getOperationSuperTypeID() != Operation::kBinaryOperation ||
+           op_sig_orig->getNonStaticArity() == 2u);
+
+    const OperationSignaturePtr op_sig =
+        OperationSignature::Create(ToLower(op_sig_orig->getName()),
+                                   op_sig_orig->getArgumentTypeIDs(),
+                                   op_sig_orig->getNumStaticArguments());
+
+    // TODO: print error message for collision
+    operations_.emplace(op_sig, operation);
+
+    const PartialSignature sig_ref =
+        std::make_pair(&op_sig->getArgumentTypeIDs(),
+                       op_sig->getNumStaticArguments());
+    primary_index_[std::make_pair(op_sig->getName(),
+                                  op_sig->getArity())].emplace(sig_ref, op_sig);
+  }
+}
+
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/OperationFactory.hpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationFactory.hpp b/types/operations/OperationFactory.hpp
new file mode 100644
index 0000000..3e90b6d
--- /dev/null
+++ b/types/operations/OperationFactory.hpp
@@ -0,0 +1,203 @@
+/**
+ * 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_TYPES_OPERATIONS_OPERATION_FACTORY_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_OPERATION_FACTORY_HPP_
+
+#include <memory>
+#include <set>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/operations/Operation.hpp"
+#include "types/operations/OperationSignature.hpp"
+#include "types/operations/binary_operations/BinaryOperation.hpp"
+#include "types/operations/unary_operations/UnaryOperation.hpp"
+#include "utility/HashPair.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class Type;
+
+/** \addtogroup Types
+ *  @{
+ */
+
+class OperationFactory {
+ public:
+  static const OperationFactory& Instance();
+
+  inline bool hasOperation(const std::string &operation_name,
+                           const std::size_t arity) const {
+    const auto indices_it =
+        primary_index_.find(std::make_pair(operation_name, arity));
+    return indices_it != primary_index_.end();
+  }
+
+  inline OperationPtr getOperation(const OperationSignaturePtr &op_signature) const {
+    DCHECK(operations_.find(op_signature) != operations_.end());
+    return operations_.at(op_signature);
+  }
+
+  inline OperationPtr getOperation(const std::string &operation_name,
+                                   const std::vector<TypeID> &argument_type_ids,
+                                   const std::size_t num_static_arguments = 0) const {
+    return getOperation(
+        OperationSignature::Create(
+            operation_name, argument_type_ids, num_static_arguments));
+  }
+
+  inline UnaryOperationPtr getUnaryOperation(
+      const OperationSignaturePtr &op_signature) const {
+    const OperationPtr operation = getOperation(op_signature);
+    DCHECK(operation->getOperationSuperTypeID() == Operation::kUnaryOperation);
+    return std::static_pointer_cast<const UnaryOperation>(operation);
+  }
+
+  inline UnaryOperationPtr getUnaryOperation(
+      const std::string &operation_name,
+      const std::vector<TypeID> &argument_type_ids,
+      const std::size_t num_static_arguments = 0) const {
+    return getUnaryOperation(
+        OperationSignature::Create(
+            operation_name, argument_type_ids, num_static_arguments));
+  }
+
+  inline BinaryOperationPtr getBinaryOperation(
+      const OperationSignaturePtr &op_signature) const {
+    const OperationPtr operation = getOperation(op_signature);
+    DCHECK(operation->getOperationSuperTypeID() == Operation::kBinaryOperation);
+    return std::static_pointer_cast<const BinaryOperation>(operation);
+  }
+
+  inline BinaryOperationPtr getBinaryOperation(
+      const std::string &operation_name,
+      const std::vector<TypeID> &argument_type_ids,
+      const std::size_t num_static_arguments = 0) const {
+    return getBinaryOperation(
+        OperationSignature::Create(
+            operation_name, argument_type_ids, num_static_arguments));
+  }
+
+  OperationSignaturePtr resolveOperation(
+      const std::string &operation_name,
+      const std::shared_ptr<const std::vector<const Type*>> &argument_types,
+      const std::shared_ptr<const std::vector<TypedValue>> &static_arguments,
+      std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
+      std::shared_ptr<const std::vector<TypedValue>> *coerced_static_arguments,
+      std::string *message) const;
+
+ private:
+  OperationFactory();
+
+  template <typename OperationT>
+  void registerOperation();
+
+  template <typename FunctorPackT>
+  void registerFunctorPack();
+
+  void registerOperationInternal(const OperationPtr &operation);
+
+  using PartialSignature = std::pair<const std::vector<TypeID>*, std::size_t>;
+
+  struct PartialSignatureLess {
+    inline bool operator()(const PartialSignature &lhs,
+                           const PartialSignature &rhs) const {
+      int cmp_code = static_cast<int>(lhs.first->size())
+                         - static_cast<int>(lhs.first->size());
+      if (cmp_code != 0) {
+        return cmp_code < 0;
+      }
+      for (std::size_t i = 0; i < lhs.first->size(); ++i) {
+        cmp_code = static_cast<int>(lhs.first->at(i))
+                       - static_cast<int>(rhs.first->at(i));
+        if (cmp_code != 0) {
+          return cmp_code < 0;
+        }
+      }
+      return lhs.second > rhs.second;
+    }
+  };
+
+  using PartialSignatureIndex = std::map<PartialSignature,
+                                         OperationSignaturePtr,
+                                         PartialSignatureLess>;
+
+  enum class ResolveStatus {
+    kSuccess = 0,
+    kError,
+    kNotFound
+  };
+
+  ResolveStatus resolveOperationWithFullTypeMatch(
+      const PartialSignatureIndex &secondary_index,
+      const std::vector<TypeID> &argument_type_ids,
+      const std::vector<const Type*> &argument_types,
+      const std::vector<TypedValue> &static_arguments,
+      std::shared_ptr<const std::vector<TypedValue>> *trimmed_static_arguments,
+      OperationSignaturePtr *resolved_op_signature,
+      std::string *message) const;
+
+  ResolveStatus resolveOperationWithPartialTypeMatch(
+      const PartialSignatureIndex &secondary_index,
+      const std::vector<TypeID> &argument_type_ids,
+      const std::vector<const Type*> &argument_types,
+      const std::vector<TypedValue> &static_arguments,
+      std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
+      std::shared_ptr<const std::vector<TypedValue>> *coerced_static_arguments,
+      OperationSignaturePtr *resolved_op_signature,
+      std::string *message) const;
+
+//  ResolveStatus resolveOperationGeneric(
+//      const std::set<OperationSignaturePtr> signatures,
+//      const std::vector<TypeID> &argument_type_ids,
+//      const std::vector<const Type*> &argument_types,
+//      const std::vector<TypedValue> &static_arguments,
+//      std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
+//      std::shared_ptr<const std::vector<TypedValue>> *coerced_static_arguments,
+//      OperationSignaturePtr *op_signature,
+//      std::string *message) const;
+
+  bool canApplyOperationTo(const OperationPtr operation,
+                           const std::vector<const Type*> &argument_types,
+                           const std::vector<TypedValue> &static_arguments,
+                           std::string *message) const;
+
+  std::unordered_map<OperationSignaturePtr,
+                     OperationPtr,
+                     OperationSignatureHash,
+                     OperationSignatureEqual> operations_;
+
+  std::unordered_map<std::pair<std::string, std::size_t>,
+                     PartialSignatureIndex> primary_index_;
+
+  DISALLOW_COPY_AND_ASSIGN(OperationFactory);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_OPERATION_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/OperationSignature.cpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationSignature.cpp b/types/operations/OperationSignature.cpp
new file mode 100644
index 0000000..6b6c4a6
--- /dev/null
+++ b/types/operations/OperationSignature.cpp
@@ -0,0 +1,91 @@
+/**
+ * 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 "types/operations/OperationSignature.hpp"
+
+#include <cstdint>
+#include <string>
+#include <type_traits>
+#include <vector>
+
+#include "types/TypeID.hpp"
+#include "types/Type.pb.h"
+#include "types/operations/Operation.pb.h"
+
+namespace quickstep {
+
+serialization::OperationSignature OperationSignature::getProto() const {
+  serialization::OperationSignature op_signature;
+
+  op_signature.set_operation_name(operation_name_);
+  for (const TypeID tid : argument_type_ids_) {
+    op_signature.add_argument_type_ids()->CopyFrom(TypeIDFactory::GetProto(tid));
+  }
+  op_signature.set_num_static_arguments(
+      static_cast<std::uint32_t>(num_static_arguments_));
+
+  return op_signature;
+}
+
+OperationSignaturePtr OperationSignature::ReconstructFromProto(
+    const serialization::OperationSignature &proto) {
+  std::vector<TypeID> argument_type_ids;
+  for (int i = 0; i < proto.argument_type_ids_size(); ++i) {
+    argument_type_ids.emplace_back(
+        TypeIDFactory::ReconstructFromProto(proto.argument_type_ids(i)));
+  }
+
+  return Create(proto.operation_name(),
+                argument_type_ids,
+                proto.num_static_arguments());
+}
+
+std::string OperationSignature::toString() const {
+  const std::size_t num_regular_arguments =
+      argument_type_ids_.size() - num_static_arguments_;
+
+  std::string str;
+  str.append(operation_name_);
+  str.push_back('(');
+  for (std::size_t i = 0; i < num_regular_arguments; ++i) {
+    if (i != 0) {
+      str.append(", ");
+    }
+    str.append(
+        kTypeNames[static_cast<std::underlying_type_t<TypeID>>(
+            argument_type_ids_[i])]);
+  }
+  if (num_static_arguments_ > 0) {
+    str.append(", static(");
+    for (std::size_t i = 0; i < num_static_arguments_; ++i) {
+      if (i != 0) {
+        str.append(", ");
+      }
+      str.append(
+          kTypeNames[static_cast<std::underlying_type_t<TypeID>>(
+              argument_type_ids_[i + num_regular_arguments])]);
+    }
+    str.push_back(')');
+  }
+  str.push_back(')');
+
+  return str;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/OperationSignature.hpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationSignature.hpp b/types/operations/OperationSignature.hpp
new file mode 100644
index 0000000..6659a85
--- /dev/null
+++ b/types/operations/OperationSignature.hpp
@@ -0,0 +1,182 @@
+/**
+ * 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_TYPES_OPERATIONS_OPERATION_SIGNATURE_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_OPERATION_SIGNATURE_HPP_
+
+#include <memory>
+#include <string>
+#include <type_traits>
+#include <vector>
+
+#include "types/TypeID.hpp"
+#include "types/operations/Operation.pb.h"
+
+#include "utility/HashPair.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+class OperationSignature;
+typedef std::shared_ptr<const OperationSignature> OperationSignaturePtr;
+
+class OperationSignature {
+ public:
+  serialization::OperationSignature getProto() const;
+
+  static OperationSignaturePtr ReconstructFromProto(
+      const serialization::OperationSignature &proto);
+
+  inline const std::string& getName() const {
+    return operation_name_;
+  }
+
+  inline std::size_t getArity() const {
+    return argument_type_ids_.size();
+  }
+
+  inline std::size_t getNonStaticArity() const {
+    return argument_type_ids_.size() - num_static_arguments_;
+  }
+
+  inline const std::vector<TypeID>& getArgumentTypeIDs() const {
+    return argument_type_ids_;
+  }
+
+  inline std::size_t getNumStaticArguments() const {
+    return num_static_arguments_;
+  }
+
+  inline bool operator==(const OperationSignature &r) const {
+    return operation_name_ == r.operation_name_
+        && argument_type_ids_ == r.argument_type_ids_
+        && num_static_arguments_ == r.num_static_arguments_;
+  }
+
+  inline bool operator!=(const OperationSignature &r) const {
+    return !(*this == r);
+  }
+
+  inline bool operator<(const OperationSignature &r) const {
+    int cmp_code = operation_name_.compare(r.operation_name_);
+    if (cmp_code != 0) {
+      return cmp_code < 0;
+    }
+    cmp_code = static_cast<int>(getArity() - r.getArity());
+    if (cmp_code != 0) {
+      return cmp_code < 0;
+    }
+    cmp_code = static_cast<int>(num_static_arguments_ - r.num_static_arguments_);
+    if (cmp_code != 0) {
+      return cmp_code > 0;
+    }
+    for (std::size_t i = 0; i < getArity(); ++i) {
+      const auto l_tid =
+          static_cast<std::underlying_type_t<TypeID>>(argument_type_ids_.at(i));
+      const auto r_tid =
+          static_cast<std::underlying_type_t<TypeID>>(r.argument_type_ids_.at(i));
+      if (l_tid != r_tid) {
+        return l_tid < r_tid;
+      }
+    }
+    return false;
+  }
+
+  inline std::size_t hash() const {
+    std::size_t hash_code = std::hash<std::string>()(operation_name_);
+    for (const TypeID tid : argument_type_ids_) {
+      hash_code = CombineHashes(hash_code, static_cast<std::size_t>(tid));
+    }
+    hash_code = CombineHashes(hash_code, num_static_arguments_);
+    return hash_code;
+  }
+
+  std::string toString() const;
+
+  static OperationSignaturePtr Create(
+      const std::string &operation_name,
+      const std::vector<TypeID> &argument_type_ids,
+      const std::size_t num_static_arguments) {
+    return OperationSignaturePtr(
+        new OperationSignature(operation_name,
+                               argument_type_ids,
+                               num_static_arguments));
+  }
+
+  static OperationSignaturePtr Create(
+      const std::string &operation_name,
+      const std::vector<TypeID> &regular_argument_type_ids,
+      const std::vector<TypeID> &static_argument_type_ids) {
+    std::vector<TypeID> argument_type_ids = regular_argument_type_ids;
+    argument_type_ids.insert(argument_type_ids.end(),
+                             static_argument_type_ids.begin(),
+                             static_argument_type_ids.end());
+    return OperationSignaturePtr(
+        new OperationSignature(operation_name,
+                               argument_type_ids,
+                               static_argument_type_ids.size()));
+  }
+
+ private:
+  OperationSignature(const std::string &operation_name,
+                     const std::vector<TypeID> &argument_type_ids,
+                     const std::size_t num_static_arguments)
+      : operation_name_(operation_name),
+        argument_type_ids_(argument_type_ids),
+        num_static_arguments_(num_static_arguments) {
+    DCHECK_GE(argument_type_ids_.size(), num_static_arguments_);
+  }
+
+  const std::string operation_name_;
+  const std::vector<TypeID> argument_type_ids_;
+  const std::size_t num_static_arguments_;
+
+  DISALLOW_COPY_AND_ASSIGN(OperationSignature);
+};
+
+/**
+ * @brief Implements the equal function for operation signatures.
+ */
+struct OperationSignatureEqual {
+  inline bool operator()(const OperationSignaturePtr &lhs,
+                         const OperationSignaturePtr &rhs) const {
+    return *lhs == *rhs;
+  }
+};
+
+/**
+ * @brief Implements the hash function for operation signatures.
+ */
+struct OperationSignatureHash {
+  inline std::size_t operator()(const OperationSignaturePtr &op_sig) const {
+    return op_sig->hash();
+  }
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_OPERATION_SIGNATURE_HPP_


[22/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/BinaryOperationWrapper.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationWrapper.hpp b/types/operations/binary_operations/BinaryOperationWrapper.hpp
new file mode 100644
index 0000000..98c2e8d
--- /dev/null
+++ b/types/operations/binary_operations/BinaryOperationWrapper.hpp
@@ -0,0 +1,629 @@
+/**
+ * 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_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_WRAPPER_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_WRAPPER_HPP_
+
+#include <cstddef>
+#include <list>
+#include <string>
+#include <tuple>
+#include <type_traits>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/operations/OperationSignature.hpp"
+#include "types/operations/OperationUtil.hpp"
+#include "types/operations/binary_operations/BinaryOperation.hpp"
+#include "utility/Macros.hpp"
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename LeftArgumentT, typename RightArgumentT, typename ResultT>
+struct BinaryFunctor {
+  typedef LeftArgumentT LeftArgumentType;
+  typedef RightArgumentT RightArgumentType;
+  typedef ResultT ResultType;
+
+  static constexpr Operation
+      ::OperationSuperTypeID kOperationSuperTypeID = Operation::kBinaryOperation;
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+class UncheckedBinaryOperatorWrapperCodegen : public UncheckedBinaryOperator {
+ public:
+  template <typename ...ConstructorArgs>
+  UncheckedBinaryOperatorWrapperCodegen(const Type &left_type,
+                                        const Type &right_type,
+                                        const Type &result_type,
+                                        ConstructorArgs &&...args)
+      : functor_(std::forward<ConstructorArgs>(args)...),
+        impl_(functor_, left_type, right_type, result_type) {}
+
+  TypedValue applyToTypedValues(const TypedValue &left,
+                                const TypedValue &right) const override {
+    return impl_.applyToTypedValues(left, right);
+  }
+
+  ColumnVector* applyToColumnVectors(const ColumnVector &left,
+                                     const ColumnVector &right) const override {
+    using LeftCVT = typename LeftGen::ColumnVectorType;
+    DCHECK_EQ(left.isNative(), LeftCVT::kNative);
+    using LeftAccessorT = ColumnVectorValueAccessor<LeftCVT>;
+    LeftAccessorT left_accessor(static_cast<const LeftCVT&>(left));
+
+    using RightCVT = typename RightGen::ColumnVectorType;
+    DCHECK_EQ(right.isNative(), RightCVT::kNative);
+    using RightAccessorT = ColumnVectorValueAccessor<RightCVT>;
+    RightAccessorT right_accessor(static_cast<const RightCVT&>(right));
+
+    const std::size_t num_tuples = left_accessor.getNumTuples();
+    DCHECK_EQ(num_tuples, right_accessor.getNumTuples());
+
+    return impl_.applyToValueAccessors(num_tuples,
+                                       &left_accessor, kInvalidAttributeID,
+                                       &right_accessor, kInvalidAttributeID);
+  }
+
+  ColumnVector* applyToColumnVectorAndStaticValue(
+      const ColumnVector &left,
+      const TypedValue &right) const override {
+    using LeftCVT = typename LeftGen::ColumnVectorType;
+    DCHECK_EQ(left.isNative(), LeftCVT::kNative);
+
+    using LeftAccessorT = ColumnVectorValueAccessor<LeftCVT>;
+    LeftAccessorT accessor(static_cast<const LeftCVT&>(left));
+    return impl_.applyToValueAccessorAndStaticValue(&accessor, 0, right);
+  }
+
+  ColumnVector* applyToStaticValueAndColumnVector(
+      const TypedValue &left,
+      const ColumnVector &right) const override {
+    using RightCVT = typename RightGen::ColumnVectorType;
+    DCHECK_EQ(right.isNative(), RightCVT::kNative);
+
+    using RightAccessorT = ColumnVectorValueAccessor<RightCVT>;
+    RightAccessorT accessor(static_cast<const RightCVT&>(right));
+    return impl_.applyToStaticValueAndValueAccessor(left, &accessor, 0);
+  }
+
+  ColumnVector* applyToSingleValueAccessor(
+      ValueAccessor *accessor,
+      const attribute_id left_id,
+      const attribute_id right_id) const override {
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        accessor,
+        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      return impl_.applyToSingleValueAccessor(accessor, left_id, right_id);
+    });
+  }
+
+  ColumnVector* applyToValueAccessorAndStaticValue(
+      ValueAccessor *left_accessor,
+      const attribute_id left_id,
+      const TypedValue &right) const override {
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        left_accessor,
+        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      return impl_.applyToValueAccessorAndStaticValue(accessor, left_id, right);
+    });
+  }
+
+  ColumnVector* applyToStaticValueAndValueAccessor(
+      const TypedValue &left,
+      ValueAccessor *right_accessor,
+      const attribute_id right_id) const override {
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        right_accessor,
+        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      return impl_.applyToStaticValueAndValueAccessor(left, accessor, right_id);
+    });
+  }
+
+  ColumnVector* applyToColumnVectorAndValueAccessor(
+      const ColumnVector &left,
+      ValueAccessor *right_accessor,
+      const attribute_id right_id) const override {
+    using LeftCVT = typename LeftGen::ColumnVectorType;
+    DCHECK_EQ(left.isNative(), LeftCVT::kNative);
+    using LeftAccessorT = ColumnVectorValueAccessor<LeftCVT>;
+    LeftAccessorT left_accessor(static_cast<const LeftCVT&>(left));
+
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        right_accessor,
+        [&](auto *right_accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+    const std::size_t num_tuples = left_accessor.getNumTuples();
+    DCHECK_EQ(num_tuples, right_accessor->getNumTuples());
+
+    return impl_.applyToValueAccessors(num_tuples,
+                                       &left_accessor, kInvalidAttributeID,
+                                       right_accessor, right_id);
+    });
+  }
+
+  ColumnVector* applyToValueAccessorAndColumnVector(
+      ValueAccessor *left_accessor,
+      const attribute_id left_id,
+      const ColumnVector &right) const override {
+    using RightCVT = typename RightGen::ColumnVectorType;
+    DCHECK_EQ(right.isNative(), RightCVT::kNative);
+    using RightAccessorT = ColumnVectorValueAccessor<RightCVT>;
+    RightAccessorT right_accessor(static_cast<const RightCVT&>(right));
+
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        left_accessor,
+        [&](auto *left_accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      const std::size_t num_tuples = right_accessor.getNumTuples();
+      DCHECK_EQ(num_tuples, left_accessor->getNumTuples());
+
+      return impl_.applyToValueAccessors(num_tuples,
+                                         left_accessor, left_id,
+                                         &right_accessor, kInvalidAttributeID);
+    });
+  }
+
+  TypedValue accumulateColumnVector(
+      const TypedValue &current,
+      const ColumnVector &column_vector,
+      std::size_t *num_tuples_applied) const override {
+    constexpr bool is_supported =
+        LeftType::kStaticTypeID == ResultType::kStaticTypeID &&
+        (LeftType::kLayout == kNativeEmbedded || LeftType::kLayout == kNativeInline) &&
+        std::is_copy_assignable<typename LeftType::cpptype>::value;
+
+    using RightCVT = typename RightGen::ColumnVectorType;
+    DCHECK_EQ(column_vector.isNative(), RightCVT::kNative);
+    using RightAccessorT = ColumnVectorValueAccessor<RightCVT>;
+    RightAccessorT accessor(static_cast<const RightCVT&>(column_vector));
+
+    return impl_.template accumulateValueAccessor<is_supported>(
+        current,
+        &accessor,
+        kInvalidAttributeID,
+        num_tuples_applied);
+  }
+
+  TypedValue accumulateValueAccessor(
+      const TypedValue &current,
+      ValueAccessor *accessor,
+      const attribute_id value_accessor_id,
+      std::size_t *num_tuples_applied) const override {
+    constexpr bool is_supported =
+        LeftType::kStaticTypeID == ResultType::kStaticTypeID &&
+        (LeftType::kLayout == kNativeEmbedded || LeftType::kLayout == kNativeInline) &&
+        std::is_copy_assignable<typename LeftType::cpptype>::value;
+
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        accessor,
+        [&](auto *accessor) -> TypedValue {  // NOLINT(build/c++11)
+      return impl_.template accumulateValueAccessor<is_supported>(
+          current,
+          accessor,
+          value_accessor_id,
+          num_tuples_applied);
+    });
+  }
+
+ private:
+  using LeftType = typename FunctorT::LeftArgumentType;
+  using RightType = typename FunctorT::RightArgumentType;
+  using ResultType = typename FunctorT::ResultType;
+
+  using FuncSpec = typename FunctorSpecializer<FunctorT, SpecArgs...>::type;
+  using LeftGen = Codegen<FuncSpec, LeftType>;
+  using RightGen = Codegen<FuncSpec, RightType>;
+  using ResultGen = Codegen<FuncSpec, ResultType>;
+
+  template <bool left_nullable, bool right_nullable>
+  struct Implementation;
+
+  const FunctorT functor_;
+  const Implementation<false, false> impl_;
+
+  DISALLOW_COPY_AND_ASSIGN(UncheckedBinaryOperatorWrapperCodegen);
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <bool left_nullable, bool right_nullable>
+struct UncheckedBinaryOperatorWrapperCodegen<FunctorT, SpecArgs...>
+    ::Implementation {
+  Implementation(const FunctorT &functor_in,
+                 const Type &left_type_in,
+                 const Type &right_type_in,
+                 const Type &result_type_in)
+      : functor(functor_in),
+        left_type(left_type_in),
+        right_type(right_type_in),
+        result_type(result_type_in) {}
+
+  inline TypedValue applyToTypedValues(const TypedValue &left,
+                                       const TypedValue &right) const {
+    if ((left_nullable && left.isNull()) || (right_nullable && right.isNull())) {
+      return TypedValue(ResultType::kStaticTypeID);
+    }
+
+    return ResultGen::template ApplyBinaryTypedValue<LeftGen, RightGen>(
+        LeftGen::ToNativeValueConst(left),
+        RightGen::ToNativeValueConst(right),
+        result_type,
+        functor);
+  }
+
+  template <typename AccessorT>
+  inline ColumnVector* applyToValueAccessorAndStaticValue(
+      AccessorT *accessor,
+      const attribute_id attr_id,
+      const TypedValue &static_value) const {
+    using AccessorGen = LeftGen;
+    using StaticValueGen = RightGen;
+
+    constexpr bool accessor_nullable = left_nullable;
+    constexpr bool static_value_nullable = right_nullable;
+
+    using ResultCVT = typename ResultGen::ColumnVectorType;
+    ResultCVT *result_cv = new ResultCVT(result_type, accessor->getNumTuples());
+
+    if (static_value_nullable && static_value.isNull()) {
+      result_cv->fillWithNulls();
+      return result_cv;
+    }
+
+    typename StaticValueGen::NativeTypeConst literal =
+        StaticValueGen::ToNativeValueConst(static_value);
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename AccessorGen::NativeTypeConstPtr arg_value =
+          AccessorGen::template GetValuePtr<
+              accessor_nullable, AccessorT>(accessor, attr_id);
+      if (accessor_nullable && AccessorGen::IsNull(arg_value)) {
+        result_cv->appendNullValue();
+      } else {
+        ResultGen::template ApplyBinaryColumnVector<LeftGen, RightGen>(
+            AccessorGen::Dereference(arg_value), literal, functor, result_cv);
+      }
+    }
+    return result_cv;
+  }
+
+  template <typename AccessorT>
+  inline ColumnVector* applyToStaticValueAndValueAccessor(
+      const TypedValue &static_value,
+      AccessorT *accessor,
+      const attribute_id attr_id) const {
+    using AccessorGen = RightGen;
+    using StaticValueGen = LeftGen;
+
+    constexpr bool accessor_nullable = right_nullable;
+    constexpr bool static_value_nullable = left_nullable;
+
+    using ResultCVT = typename ResultGen::ColumnVectorType;
+    ResultCVT *result_cv = new ResultCVT(result_type, accessor->getNumTuples());
+
+    if (static_value_nullable && static_value.isNull()) {
+      result_cv->fillWithNulls();
+      return result_cv;
+    }
+
+    typename StaticValueGen::NativeTypeConst literal =
+        StaticValueGen::ToNativeValueConst(static_value);
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename AccessorGen::NativeTypeConstPtr arg_value =
+          AccessorGen::template GetValuePtr<
+              accessor_nullable, AccessorT>(accessor, attr_id);
+      if (accessor_nullable && AccessorGen::IsNull(arg_value)) {
+        result_cv->appendNullValue();
+      } else {
+        ResultGen::template ApplyBinaryColumnVector<LeftGen, RightGen>(
+            literal, AccessorGen::Dereference(arg_value), functor, result_cv);
+      }
+    }
+    return result_cv;
+  }
+
+  template <typename AccessorT>
+  inline ColumnVector* applyToSingleValueAccessor(
+      AccessorT *accessor,
+      const attribute_id left_id,
+      const attribute_id right_id) const {
+    using ResultCVT = typename ResultGen::ColumnVectorType;
+    ResultCVT *result_cv = new ResultCVT(result_type, accessor->getNumTuples());
+
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename LeftGen::NativeTypeConstPtr left_value =
+          LeftGen::template GetValuePtr<
+              left_nullable, AccessorT>(accessor, left_id);
+      if (left_nullable && LeftGen::IsNull(left_value)) {
+        result_cv->appendNullValue();
+        continue;
+      }
+      typename RightGen::NativeTypeConstPtr right_value =
+          RightGen::template GetValuePtr<
+              right_nullable, AccessorT>(accessor, right_id);
+      if (right_nullable && RightGen::IsNull(right_value)) {
+        result_cv->appendNullValue();
+        continue;
+      }
+      ResultGen::template ApplyBinaryColumnVector<LeftGen, RightGen>(
+          LeftGen::Dereference(left_value),
+          RightGen::Dereference(right_value),
+          functor,
+          result_cv);
+    }
+    return result_cv;
+  }
+
+  template <typename LeftAccessorT, typename RightAccessorT>
+  inline ColumnVector* applyToValueAccessors(const std::size_t num_tuples,
+                                             LeftAccessorT *left_accessor,
+                                             const attribute_id left_id,
+                                             RightAccessorT *right_accessor,
+                                             const attribute_id right_id) const {
+    DCHECK_EQ(num_tuples, left_accessor->getNumTuples());
+    DCHECK_EQ(num_tuples, right_accessor->getNumTuples());
+
+    using ResultCVT = typename ResultGen::ColumnVectorType;
+    ResultCVT *result_cv = new ResultCVT(result_type, num_tuples);
+
+    left_accessor->beginIteration();
+    right_accessor->beginIteration();
+    while (left_accessor->next()) {
+      right_accessor->next();
+      typename LeftGen::NativeTypeConstPtr left_value =
+          LeftGen::template GetValuePtr<
+              left_nullable, LeftAccessorT>(left_accessor, left_id);
+      if (left_nullable && LeftGen::IsNull(left_value)) {
+        result_cv->appendNullValue();
+        continue;
+      }
+      typename RightGen::NativeTypeConstPtr right_value =
+          RightGen::template GetValuePtr<
+              right_nullable, RightAccessorT>(right_accessor, right_id);
+      if (right_nullable && RightGen::IsNull(right_value)) {
+        result_cv->appendNullValue();
+        continue;
+      }
+      ResultGen::template ApplyBinaryColumnVector<LeftGen, RightGen>(
+          LeftGen::Dereference(left_value),
+          RightGen::Dereference(right_value),
+          functor,
+          result_cv);
+    }
+    return result_cv;
+  }
+
+  template <bool supported, typename AccessorT>
+  inline TypedValue accumulateValueAccessor(const TypedValue &current,
+                                            AccessorT *accessor,
+                                            const attribute_id attr_id,
+                                            std::size_t *num_tuples_applied,
+                                            std::enable_if_t<supported>* = 0) const {
+    DCHECK(num_tuples_applied);
+
+    *num_tuples_applied = 0;
+    if (left_nullable && current.isNull()) {
+      return result_type.makeNullValue();
+    }
+
+    using LeftCppType = typename LeftType::cpptype;
+    using ResultCppType = typename ResultType::cpptype;
+    ResultCppType accumulated = current.getLiteral<LeftCppType>();
+
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename RightGen::NativeTypeConstPtr right_value =
+          RightGen::template GetValuePtr<
+              right_nullable, AccessorT>(accessor, attr_id);
+      if (right_nullable && RightGen::IsNull(right_value)) {
+        continue;
+      }
+
+      accumulated =
+          ResultGen::FunctorSpecializer::Invoke(
+              functor, accumulated, RightGen::Dereference(right_value));
+      ++(*num_tuples_applied);
+    }
+
+    return TypedValue(accumulated);
+  }
+
+  template <bool supported, typename AccessorT>
+  inline TypedValue accumulateValueAccessor(const TypedValue &current,
+                                            AccessorT *accessor,
+                                            const attribute_id attr_id,
+                                            std::size_t *num_tuples_applied,
+                                            std::enable_if_t<!supported>* = 0) const {
+    LOG(FATAL) << "Unimplemented method UncheckedBinaryOperatorWrapperCodegen"
+               << "::accumulateValueAccessor() because ResultType and LeftType "
+               << "are not same or not native types.";
+  }
+
+  const FunctorT &functor;
+  const Type &left_type;
+  const Type &right_type;
+  const Type &result_type;
+};
+
+template <typename FunctorT>
+class BinaryOperationWrapper : public BinaryOperation {
+ public:
+  BinaryOperationWrapper()
+      : BinaryOperation(),
+        operation_name_(FunctorT::GetName()) {}
+
+  std::string getName() const override {
+    return operation_name_;
+  }
+
+  std::string getShortName() const override {
+    return getName();
+  }
+
+  std::vector<OperationSignaturePtr> getSignatures() const override {
+    return {
+        OperationSignature::Create(
+            getName(), {LeftType::kStaticTypeID, RightType::kStaticTypeID}, 0)
+    };
+  }
+
+  bool canApplyTo(const Type &left,
+                  const Type &right,
+                  const std::vector<TypedValue> &static_arguments,
+                  std::string *message) const override {
+    DCHECK(left.getTypeID() == LeftType::kStaticTypeID);
+    DCHECK(right.getTypeID() == RightType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return true;
+  }
+
+  const Type* getResultType(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(left.getTypeID() == LeftType::kStaticTypeID);
+    DCHECK(right.getTypeID() == RightType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return getResultTypeImpl<ResultType::kParameterized>(
+        left, right, static_arguments);
+  }
+
+  UncheckedBinaryOperator* makeUncheckedBinaryOperator(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(left.getTypeID() == LeftType::kStaticTypeID);
+    DCHECK(right.getTypeID() == RightType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return makeUncheckedBinaryOperatorImpl<
+        std::is_default_constructible<FunctorT>::value>(
+            left, right, static_arguments);
+  }
+
+ private:
+  using LeftType = typename FunctorT::LeftArgumentType;
+  using RightType = typename FunctorT::RightArgumentType;
+  using ResultType = typename FunctorT::ResultType;
+
+  template <bool functor_use_default_constructor>
+  inline UncheckedBinaryOperator* makeUncheckedBinaryOperatorImpl(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<functor_use_default_constructor>* = 0) const {
+    return new UncheckedBinaryOperatorWrapperCodegen<FunctorT>(
+        left, right, *getResultType(left, right, static_arguments));
+  }
+
+  template <bool functor_use_default_constructor>
+  inline UncheckedBinaryOperator* makeUncheckedBinaryOperatorImpl(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<!functor_use_default_constructor>* = 0) const {
+    return new UncheckedBinaryOperatorWrapperCodegen<FunctorT>(
+        left, right, *getResultType(left, right, static_arguments),
+        static_cast<const LeftType&>(left),
+        static_cast<const RightType&>(right));
+  }
+
+  template <bool result_type_has_parameter>
+  inline const Type* getResultTypeImpl(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<!result_type_has_parameter>* = 0) const {
+    return &TypeFactory::GetType(
+        ResultType::kStaticTypeID,
+        left.isNullable() || right.isNullable());
+  }
+
+  template <bool result_type_has_parameter>
+  inline const Type* getResultTypeImpl(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<result_type_has_parameter>* = 0) const {
+    return FunctorT::GetResultType(left, right);
+  }
+
+  const std::string operation_name_;
+
+  DISALLOW_COPY_AND_ASSIGN(BinaryOperationWrapper);
+};
+
+template <typename LeftPack, typename RightPack,
+          template <typename LeftT,
+                    typename RightT,
+                    typename ResultT> class FunctorT,
+          template <typename LeftT,
+                    typename RightT> class ResultGenerator>
+struct BinaryFunctorCrossProductPack {
+  template <std::size_t l, std::size_t r>
+  inline static OperationPtr GenerateInner() {
+    using LeftType = std::tuple_element_t<l, LeftPack>;
+    using RightType = std::tuple_element_t<r, RightPack>;
+    using ResultType = typename ResultGenerator<LeftType, RightType>::type;
+
+    return std::make_shared<
+        const BinaryOperationWrapper<
+            FunctorT<LeftType, RightType, ResultType>>>();
+  }
+
+  template <std::size_t l, std::size_t ...Rs>
+  inline static std::list<OperationPtr> GenerateRightHelper() {
+    return { GenerateInner<l, Rs>()... };
+  }
+
+  template <std::size_t ...Ls, std::size_t ...Rs>
+  inline static std::vector<std::list<OperationPtr>> GenerateLeftHelper(
+      meta::IntegerSequence<Ls...> &&l_seq, meta::IntegerSequence<Rs...> &&r_seq) {
+    return { GenerateRightHelper<Ls, Rs...>()... };
+  }
+
+  template <typename Dispatcher>
+  inline static std::list<OperationPtr> GenerateOperations() {
+    std::vector<std::list<OperationPtr>> op_list_groups =
+        GenerateLeftHelper(typename meta::MakeSequence<std::tuple_size<LeftPack>::value>::type(),
+                           typename meta::MakeSequence<std::tuple_size<RightPack>::value>::type());
+
+    std::list<OperationPtr> operations;
+    for (std::list<OperationPtr> &op_list : op_list_groups) {
+      operations.splice(operations.end(), std::move(op_list));
+    }
+    return operations;
+  }
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_WRAPPER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/CMakeLists.txt b/types/operations/binary_operations/CMakeLists.txt
index 4d26b75..09566aa 100644
--- a/types/operations/binary_operations/CMakeLists.txt
+++ b/types/operations/binary_operations/CMakeLists.txt
@@ -16,185 +16,97 @@
 # under the License.
 
 # Declare micro-libs:
-add_library(quickstep_types_operations_binaryoperations_AddBinaryOperation
-            AddBinaryOperation.cpp
-            AddBinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
+add_library(quickstep_types_operations_binaryoperations_ArithmeticBinaryOperations
             ../../../empty_src.cpp
-            ArithmeticBinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
+            ArithmeticBinaryOperations.hpp)
+add_library(quickstep_types_operations_binaryoperations_ArithmeticBinaryFunctorOverloads
             ../../../empty_src.cpp
-            ArithmeticBinaryOperators.hpp)
+            ArithmeticBinaryFunctorOverloads.hpp)
+add_library(quickstep_types_operations_binaryoperations_AsciiStringBinaryOperations
+            ../../../empty_src.cpp
+            AsciiStringBinaryOperations.hpp)
 add_library(quickstep_types_operations_binaryoperations_BinaryOperation
             BinaryOperation.cpp
             BinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_BinaryOperationFactory
-            BinaryOperationFactory.cpp
-            BinaryOperationFactory.hpp)
-add_library(quickstep_types_operations_binaryoperations_BinaryOperationID
-            BinaryOperationID.cpp
-            BinaryOperationID.hpp)
-add_library(quickstep_types_operations_binaryoperations_DivideBinaryOperation
-            DivideBinaryOperation.cpp
-            DivideBinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_ModuloBinaryOperation
-            ModuloBinaryOperation.cpp
-            ModuloBinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_MultiplyBinaryOperation
-            MultiplyBinaryOperation.cpp
-            MultiplyBinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_SubtractBinaryOperation
-            SubtractBinaryOperation.cpp
-            SubtractBinaryOperation.hpp)
+add_library(quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+            ../../../empty_src.cpp
+            BinaryOperationWrapper.hpp)
+add_library(quickstep_types_operations_binaryoperations_CMathBinaryOperations
+            ../../../empty_src.cpp
+            CMathBinaryOperations.hpp)
 
 # Link dependencies:
-target_link_libraries(quickstep_types_operations_binaryoperations_AddBinaryOperation
-                      glog
+target_link_libraries(quickstep_types_operations_binaryoperations_ArithmeticBinaryFunctorOverloads
                       quickstep_types_DateOperatorOverloads
+                      quickstep_utility_meta_Common)
+target_link_libraries(quickstep_types_operations_binaryoperations_ArithmeticBinaryOperations
                       quickstep_types_DateType
                       quickstep_types_DatetimeIntervalType
                       quickstep_types_DatetimeLit
                       quickstep_types_DatetimeType
                       quickstep_types_IntervalLit
+                      quickstep_types_NumericTypeUnifier
                       quickstep_types_Type
                       quickstep_types_TypeErrors
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_types_YearMonthIntervalType
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_EqualsAnyConstant
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
+                      quickstep_types_operations_binaryoperations_ArithmeticBinaryFunctorOverloads
+                      quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+                      quickstep_utility_meta_Common)
+target_link_libraries(quickstep_types_operations_binaryoperations_AsciiStringBinaryOperations
                       glog
-                      quickstep_types_DoubleType
-                      quickstep_types_FloatType
+                      quickstep_types_CharType
                       quickstep_types_IntType
-                      quickstep_types_LongType
-                      quickstep_types_NumericTypeUnifier
                       quickstep_types_Type
-                      quickstep_types_TypeErrors
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
-                      quickstep_types_TypedValue
-                      quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
-                      glog
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_ValueAccessor
-                      quickstep_storage_ValueAccessorUtil
-                      quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
-                      quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_utility_Macros)
+                      quickstep_types_VarCharType
+                      quickstep_types_operations_OperationUtil
+                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper
+                      quickstep_types_port_strnlen)
 target_link_libraries(quickstep_types_operations_binaryoperations_BinaryOperation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_StorageBlockInfo
                       quickstep_types_TypedValue
                       quickstep_types_operations_Operation
-                      quickstep_types_operations_Operation_proto
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      glog
-                      quickstep_types_operations_Operation_proto
-                      quickstep_types_operations_binaryoperations_AddBinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_types_operations_binaryoperations_DivideBinaryOperation
-                      quickstep_types_operations_binaryoperations_ModuloBinaryOperation
-                      quickstep_types_operations_binaryoperations_MultiplyBinaryOperation
-                      quickstep_types_operations_binaryoperations_SubtractBinaryOperation
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_DivideBinaryOperation
-                      glog
-                      quickstep_types_DateOperatorOverloads
-                      quickstep_types_DatetimeIntervalType
-                      quickstep_types_DoubleType
-                      quickstep_types_FloatType
-                      quickstep_types_IntType
-                      quickstep_types_IntervalLit
-                      quickstep_types_LongType
-                      quickstep_types_Type
-                      quickstep_types_TypeErrors
-                      quickstep_types_TypeFactory
-                      quickstep_types_TypeID
-                      quickstep_types_TypedValue
-                      quickstep_types_YearMonthIntervalType
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_EqualsAnyConstant
+                      quickstep_types_operations_OperationSignature
                       quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_ModuloBinaryOperation
+target_link_libraries(quickstep_types_operations_binaryoperations_BinaryOperationWrapper
                       glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
                       quickstep_types_Type
-                      quickstep_types_TypeErrors
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_EqualsAnyConstant
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_MultiplyBinaryOperation
-                      glog
-                      quickstep_types_DateOperatorOverloads
-                      quickstep_types_DatetimeIntervalType
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_operations_OperationSignature
+                      quickstep_types_operations_OperationUtil
+                      quickstep_types_operations_binaryoperations_BinaryOperation
+                      quickstep_utility_Macros
+                      quickstep_utility_meta_Common)
+target_link_libraries(quickstep_types_operations_binaryoperations_CMathBinaryOperations
                       quickstep_types_DoubleType
                       quickstep_types_FloatType
                       quickstep_types_IntType
-                      quickstep_types_IntervalLit
                       quickstep_types_LongType
-                      quickstep_types_Type
-                      quickstep_types_TypeErrors
-                      quickstep_types_TypeFactory
-                      quickstep_types_TypeID
-                      quickstep_types_TypedValue
-                      quickstep_types_YearMonthIntervalType
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_EqualsAnyConstant
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_SubtractBinaryOperation
-                      glog
-                      quickstep_types_DateOperatorOverloads
-                      quickstep_types_DateType
-                      quickstep_types_DatetimeIntervalType
-                      quickstep_types_DatetimeLit
-                      quickstep_types_DatetimeType
-                      quickstep_types_IntervalLit
-                      quickstep_types_Type
-                      quickstep_types_TypeErrors
-                      quickstep_types_TypeFactory
-                      quickstep_types_TypeID
-                      quickstep_types_TypedValue
-                      quickstep_types_YearMonthIntervalType
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_EqualsAnyConstant
-                      quickstep_utility_Macros)
+                      quickstep_types_operations_OperationUtil
+                      quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+                      quickstep_utility_meta_Common)
 
 # Module all-in-one library:
 add_library(quickstep_types_operations_binaryoperations ../../../empty_src.cpp)
 target_link_libraries(quickstep_types_operations_binaryoperations
                       quickstep_types_operations_binaryoperations_AddBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
+                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperations
+                      quickstep_types_operations_binaryoperations_ArithmeticBinaryFunctorOverloads
+                      quickstep_types_operations_binaryoperations_AsciiStringBinaryOperations
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_types_operations_binaryoperations_DivideBinaryOperation
-                      quickstep_types_operations_binaryoperations_ModuloBinaryOperation
-                      quickstep_types_operations_binaryoperations_MultiplyBinaryOperation
-                      quickstep_types_operations_binaryoperations_SubtractBinaryOperation)
+                      quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+                      quickstep_types_operations_binaryoperations_CMathBinaryOperations)
 
 # Tests:
 add_library(quickstep_types_operations_binaryoperations_tests_BinaryOperationTestUtil
@@ -228,14 +140,7 @@ target_link_libraries(BinaryOperation_tests
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_types_operations_Operation_proto
-                      quickstep_types_operations_binaryoperations_AddBinaryOperation
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_types_operations_binaryoperations_DivideBinaryOperation
-                      quickstep_types_operations_binaryoperations_ModuloBinaryOperation
-                      quickstep_types_operations_binaryoperations_MultiplyBinaryOperation
-                      quickstep_types_operations_binaryoperations_SubtractBinaryOperation
                       quickstep_types_operations_binaryoperations_tests_BinaryOperationTestUtil
                       quickstep_utility_Macros)
 add_test(BinaryOperation_tests BinaryOperation_tests)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/CMathBinaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/CMathBinaryOperations.hpp b/types/operations/binary_operations/CMathBinaryOperations.hpp
new file mode 100644
index 0000000..9a90a12
--- /dev/null
+++ b/types/operations/binary_operations/CMathBinaryOperations.hpp
@@ -0,0 +1,78 @@
+/**
+ * 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_TYPES_OPERATIONS_BINARY_OPERATIONS_CMATH_BINARY_OPERATIONS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_CMATH_BINARY_OPERATIONS_HPP_
+
+#include <cmath>
+#include <string>
+
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/operations/OperationUtil.hpp"
+#include "types/operations/binary_operations/BinaryOperationWrapper.hpp"
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename LeftT, typename RightT, typename ResultT,
+          typename ResultT::cpptype f(typename LeftT::cpptype,
+                                      typename RightT::cpptype),
+          typename FunctorNameT>
+struct CMathBinaryFunctorWrapper {
+  struct Implemenation : public BinaryFunctor<LeftT, RightT, ResultT> {
+    inline typename ResultT::cpptype apply(
+        const typename LeftT::cpptype &left,
+        const typename RightT::cpptype &right) const {
+      return f(left, right);
+    }
+    inline static std::string GetName() {
+      return FunctorNameT::ToString();
+    }
+  };
+
+  typedef Implemenation type;
+};
+
+template <typename LeftT, typename RightT, typename ResultT,
+          typename ResultT::cpptype f(typename LeftT::cpptype,
+                                      typename RightT::cpptype),
+          typename FunctorNameT>
+using CMathBinaryFunctor =
+    typename CMathBinaryFunctorWrapper<LeftT, RightT, ResultT, f, FunctorNameT>::type;
+
+using CMathBinaryFunctorPack = FunctorPack<
+// pow
+    CMathBinaryFunctor<FloatType, FloatType, FloatType,
+                       std::pow, meta::StringLiteral<'p','o','w'>>,
+    CMathBinaryFunctor<DoubleType, DoubleType, DoubleType,
+                       std::pow, meta::StringLiteral<'p','o','w'>>
+>;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_CMATH_UNARY_OPERATIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/DivideBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/DivideBinaryOperation.cpp b/types/operations/binary_operations/DivideBinaryOperation.cpp
deleted file mode 100644
index 59dcb3e..0000000
--- a/types/operations/binary_operations/DivideBinaryOperation.cpp
+++ /dev/null
@@ -1,391 +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 "types/operations/binary_operations/DivideBinaryOperation.hpp"
-
-#include <string>
-#include <utility>
-
-#include "types/DateOperatorOverloads.hpp"
-#include "types/DatetimeIntervalType.hpp"
-#include "types/DoubleType.hpp"
-#include "types/FloatType.hpp"
-#include "types/IntType.hpp"
-#include "types/IntervalLit.hpp"
-#include "types/LongType.hpp"
-#include "types/Type.hpp"
-#include "types/TypeErrors.hpp"
-#include "types/TypeFactory.hpp"
-#include "types/TypeID.hpp"
-#include "types/YearMonthIntervalType.hpp"
-#include "types/operations/binary_operations/ArithmeticBinaryOperators.hpp"
-#include "utility/EqualsAnyConstant.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-bool DivideBinaryOperation::canApplyToTypes(const Type &left, const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble:
-    case kDatetimeInterval:
-    case kYearMonthInterval: {
-      return (right.getSuperTypeID() == Type::kNumeric);
-    }
-    default:
-      return false;
-  }
-}
-
-const Type* DivideBinaryOperation::resultTypeForArgumentTypes(const Type &left, const Type &right) const {
-  if (left.getSuperTypeID() == Type::kNumeric && right.getSuperTypeID() == Type::kNumeric) {
-    return TypeFactory::GetUnifyingType(left, right);
-  } else if (left.getTypeID() == kDatetimeInterval && right.getSuperTypeID() == Type::kNumeric) {
-    return &(DatetimeIntervalType::Instance(left.isNullable() || right.isNullable()));
-  } else if (left.getTypeID() == kYearMonthInterval && right.getSuperTypeID() == Type::kNumeric) {
-    return &(YearMonthIntervalType::Instance(left.isNullable() || right.isNullable()));
-  } else {
-    return nullptr;
-  }
-}
-
-const Type* DivideBinaryOperation::resultTypeForPartialArgumentTypes(
-    const Type *left,
-    const Type *right) const {
-  if (left == nullptr) {
-    // Can't determine result type just based on right (divisor) type.
-    return nullptr;
-  } else if (right == nullptr) {
-    switch (left->getTypeID()) {
-      case kDouble:
-        // Double has highest precedence of numeric types.
-        return &TypeFactory::GetType(kDouble, true);
-      case kDatetimeInterval:
-        return &TypeFactory::GetType(kDatetimeInterval, true);
-      case kYearMonthInterval:
-        return &TypeFactory::GetType(kYearMonthInterval, true);
-      default:
-        // Ambiguous or inapplicable.
-        return nullptr;
-    }
-  } else {
-    return resultTypeForArgumentTypes(*left, *right);
-  }
-}
-
-bool DivideBinaryOperation::partialTypeSignatureIsPlausible(
-    const Type *result_type,
-    const Type *left_argument_type,
-    const Type *right_argument_type) const {
-  // Early check: if either argument type is nullable or unknown, result type
-  // must also be nullable.
-  if ((left_argument_type == nullptr)
-      || left_argument_type->isNullable()
-      || (right_argument_type == nullptr)
-      || right_argument_type->isNullable()) {
-    if ((result_type != nullptr) && (!result_type->isNullable())) {
-      return false;
-    }
-  }
-
-  if (left_argument_type == nullptr) {
-    if (right_argument_type == nullptr) {
-      if (result_type == nullptr) {
-        // All types unknown.
-        return true;
-      } else {
-        // Only result type is known, just check that it is one of the types
-        // that can possibly be returned.
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-      }
-    }
-
-    if (result_type == nullptr) {
-      // Right (divisor) argument type is known, left (dividend) argument and
-      // result types are unknown. Just check that it is possible to divide by
-      // the right (divisor) type.
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          right_argument_type->getTypeID(),
-          kInt, kLong, kFloat, kDouble);
-    }
-
-    // Return type and right (divisor) argument type are known, left (dividend)
-    // argument type is unknown. Check that result and divisor are compatible.
-    switch (right_argument_type->getTypeID()) {
-      case kInt:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-      case kLong:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kLong, kDouble, kDatetimeInterval, kYearMonthInterval);
-      case kFloat:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-      case kDouble:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kDouble, kDatetimeInterval, kYearMonthInterval);
-      default:
-        return false;
-    }
-  } else {  // left_argument_type != nullptr
-    if (right_argument_type == nullptr) {
-      if (result_type == nullptr) {
-        // Left (dividend) argument type is known, right (divisor) argument
-        // type and result type are unknown. Just check that the left
-        // (dividend) type can be divided.
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            left_argument_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-      }
-
-      // Result type and left (dividend) argument type are known, but right
-      // (divisor) argument type is unknown. Check that result and dividend are
-      // compatible.
-      switch (left_argument_type->getTypeID()) {
-        case kInt:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kInt, kLong, kFloat, kDouble);
-        case kLong:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kLong, kDouble);
-        case kFloat:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kFloat, kDouble);
-        case kDouble:
-          return (result_type->getTypeID() == kDouble);
-        case kDatetimeInterval:
-          return (result_type->getTypeID() == kDatetimeInterval);
-        case kYearMonthInterval:
-          return (result_type->getTypeID() == kYearMonthInterval);
-        default:
-          return false;
-      }
-    }
-
-    // Left and right (dividend and divisor) argument types are both known.
-    const Type *actual_result_type = resultTypeForArgumentTypes(*left_argument_type,
-                                                                *right_argument_type);
-    if (actual_result_type == nullptr) {
-      // Both argument Types are known, but this operation is NOT applicable to
-      // them. No matter what the result_type is, the signature is not
-      // plausible.
-      return false;
-    } else if (result_type == nullptr) {
-      return true;
-    } else {
-      // Check if result type matches.
-      return result_type->equals(*actual_result_type);
-    }
-  }
-}
-
-std::pair<const Type*, const Type*> DivideBinaryOperation::pushDownTypeHint(
-    const Type *result_type_hint) const {
-  if (result_type_hint == nullptr) {
-    return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-
-  switch (result_type_hint->getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble:
-      return std::pair<const Type*, const Type*>(result_type_hint, result_type_hint);
-    case kDatetimeInterval:
-    case kYearMonthInterval:
-      // Left (dividend) should be the same kind of interval as '*type_hint',
-      // right (divisor) can be any numeric type.
-      return std::pair<const Type*, const Type*>(result_type_hint, nullptr);
-    default:
-      // Inapplicable.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-}
-
-TypedValue DivideBinaryOperation::applyToChecked(const TypedValue &left,
-                                                 const Type &left_type,
-                                                 const TypedValue &right,
-                                                 const Type &right_type) const {
-  switch (left_type.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      if (right_type.getSuperTypeID() == Type::kNumeric) {
-        return applyToCheckedNumericHelper<DivideFunctor>(left, left_type,
-                                                          right, right_type);
-      }
-      break;
-    }
-    case kDatetimeInterval: {
-      switch (right_type.getTypeID()) {
-        case kInt:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kDatetimeInterval);
-          }
-
-          return TypedValue(left.getLiteral<DatetimeIntervalLit>() / right.getLiteral<IntType::cpptype>());
-        case kLong:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kDatetimeInterval);
-          }
-
-          return TypedValue(left.getLiteral<DatetimeIntervalLit>() / right.getLiteral<LongType::cpptype>());
-        case kFloat:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kDatetimeInterval);
-          }
-
-          return TypedValue(left.getLiteral<DatetimeIntervalLit>() / right.getLiteral<FloatType::cpptype>());
-        case kDouble:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kDatetimeInterval);
-          }
-
-          return TypedValue(left.getLiteral<DatetimeIntervalLit>() / right.getLiteral<DoubleType::cpptype>());
-        default:
-          break;
-      }
-      break;
-    }
-    case kYearMonthInterval: {
-      switch (right_type.getTypeID()) {
-        case kInt:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kYearMonthInterval);
-          }
-
-          return TypedValue(left.getLiteral<YearMonthIntervalLit>() / right.getLiteral<IntType::cpptype>());
-        case kLong:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kYearMonthInterval);
-          }
-
-          return TypedValue(left.getLiteral<YearMonthIntervalLit>() / right.getLiteral<LongType::cpptype>());
-        case kFloat:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kYearMonthInterval);
-          }
-
-          return TypedValue(left.getLiteral<YearMonthIntervalLit>() / right.getLiteral<FloatType::cpptype>());
-        case kDouble:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kYearMonthInterval);
-          }
-
-          return TypedValue(left.getLiteral<YearMonthIntervalLit>() / right.getLiteral<DoubleType::cpptype>());
-        default:
-          break;
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-             << left_type.getName() << " and " << right_type.getName();
-}
-
-UncheckedBinaryOperator* DivideBinaryOperation::makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                                                    const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      if (right.getSuperTypeID() == Type::kNumeric) {
-        return makeNumericBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator>(left, right);
-      }
-      break;
-    }
-    case kDatetimeInterval: {
-      switch (right.getTypeID()) {
-        case kInt: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   DatetimeIntervalType,
-                                                   DatetimeIntervalLit, IntType::cpptype>(left, right);
-        }
-        case kLong: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   DatetimeIntervalType,
-                                                   DatetimeIntervalLit, LongType::cpptype>(left, right);
-        }
-        case kFloat: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   DatetimeIntervalType,
-                                                   DatetimeIntervalLit, FloatType::cpptype>(left, right);
-        }
-        case kDouble: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   DatetimeIntervalType,
-                                                   DatetimeIntervalLit, DoubleType::cpptype>(left, right);
-        }
-        default:
-          break;
-      }
-      break;
-    }
-    case kYearMonthInterval: {
-      switch (right.getTypeID()) {
-        case kInt: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   YearMonthIntervalType,
-                                                   YearMonthIntervalLit, IntType::cpptype>(left, right);
-        }
-        case kLong: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   YearMonthIntervalType,
-                                                   YearMonthIntervalLit, LongType::cpptype>(left, right);
-        }
-        case kFloat: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   YearMonthIntervalType,
-                                                   YearMonthIntervalLit, FloatType::cpptype>(left, right);
-        }
-        case kDouble: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   YearMonthIntervalType,
-                                                   YearMonthIntervalLit, DoubleType::cpptype>(left, right);
-        }
-        default:
-          break;
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/DivideBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/DivideBinaryOperation.hpp b/types/operations/binary_operations/DivideBinaryOperation.hpp
deleted file mode 100644
index 3ab7aa1..0000000
--- a/types/operations/binary_operations/DivideBinaryOperation.hpp
+++ /dev/null
@@ -1,93 +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_TYPES_OPERATIONS_BINARY_OPERATIONS_DIVIDE_BINARY_OPERATION_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_DIVIDE_BINARY_OPERATION_HPP_
-
-#include <utility>
-
-#include "types/TypedValue.hpp"
-#include "types/operations/binary_operations/ArithmeticBinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-class Type;
-class UncheckedBinaryOperator;
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief The BinaryOperation for division.
- *
- * @note DivideBinaryOperation is not commutative: the left argument is the
- *       dividend and the right argument is the divisor.
- **/
-class DivideBinaryOperation : public ArithmeticBinaryOperation {
- public:
-  /**
-   * @brief Get a reference to the singleton instance of this Operation.
-   *
-   * @return A reference to the singleton instance of this Operation.
-   **/
-  static const DivideBinaryOperation& Instance() {
-    static DivideBinaryOperation instance;
-    return instance;
-  }
-
-  bool canApplyToTypes(const Type &left,
-                       const Type &right) const override;
-
-  const Type* resultTypeForArgumentTypes(const Type &left,
-                                         const Type &right) const override;
-
-  const Type* resultTypeForPartialArgumentTypes(const Type *left,
-                                                const Type *right) const override;
-
-  bool partialTypeSignatureIsPlausible(const Type *result_type,
-                                       const Type *left_argument_type,
-                                       const Type *right_argument_type) const override;
-
-  std::pair<const Type*, const Type*> pushDownTypeHint(
-      const Type *result_type_hint) const override;
-
-  TypedValue applyToChecked(const TypedValue &left,
-                            const Type &left_type,
-                            const TypedValue &right,
-                            const Type &right_type) const override;
-
-  UncheckedBinaryOperator* makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                               const Type &right) const override;
-
- private:
-  DivideBinaryOperation()
-      : ArithmeticBinaryOperation(BinaryOperationID::kDivide) {
-  }
-
-  DISALLOW_COPY_AND_ASSIGN(DivideBinaryOperation);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_DIVIDE_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/ModuloBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ModuloBinaryOperation.cpp b/types/operations/binary_operations/ModuloBinaryOperation.cpp
deleted file mode 100644
index 917c6c9..0000000
--- a/types/operations/binary_operations/ModuloBinaryOperation.cpp
+++ /dev/null
@@ -1,259 +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 "types/operations/binary_operations/ModuloBinaryOperation.hpp"
-
-#include <string>
-#include <utility>
-
-#include "types/Type.hpp"
-#include "types/TypeErrors.hpp"
-#include "types/TypeFactory.hpp"
-#include "types/TypeID.hpp"
-#include "types/operations/binary_operations/ArithmeticBinaryOperators.hpp"
-#include "utility/EqualsAnyConstant.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-bool ModuloBinaryOperation::canApplyToTypes(const Type &left, const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:  // Fall through
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      return (right.getSuperTypeID() == Type::kNumeric);
-    }
-    // TODO(jianqiao): Extend modulo operator to be applicable to DatetimeInterval
-    // and YearMonthInterval.
-    default:
-      return false;
-  }
-}
-
-const Type* ModuloBinaryOperation::resultTypeForArgumentTypes(const Type &left, const Type &right) const {
-  if (left.getSuperTypeID() == Type::kNumeric && right.getSuperTypeID() == Type::kNumeric) {
-    return TypeFactory::GetUnifyingType(left, right);
-  } else {
-    return nullptr;
-  }
-}
-
-const Type* ModuloBinaryOperation::resultTypeForPartialArgumentTypes(
-    const Type *left,
-    const Type *right) const {
-  if (left == nullptr) {
-    // Can't determine result type just based on right (divisor) type.
-    return nullptr;
-  } else if (right == nullptr) {
-    switch (left->getTypeID()) {
-      case kDouble:
-        // Double has highest precedence of numeric types.
-        return &TypeFactory::GetType(kDouble, true);
-      default:
-        // Ambiguous or inapplicable.
-        return nullptr;
-    }
-  } else {
-    return resultTypeForArgumentTypes(*left, *right);
-  }
-}
-
-bool ModuloBinaryOperation::partialTypeSignatureIsPlausible(
-    const Type *result_type,
-    const Type *left_argument_type,
-    const Type *right_argument_type) const {
-  // Early check: if either argument type is nullable or unknown, result type
-  // must also be nullable.
-  if ((left_argument_type == nullptr)
-      || left_argument_type->isNullable()
-      || (right_argument_type == nullptr)
-      || right_argument_type->isNullable()) {
-    if ((result_type != nullptr) && (!result_type->isNullable())) {
-      return false;
-    }
-  }
-
-  if (left_argument_type == nullptr) {
-    if (right_argument_type == nullptr) {
-      if (result_type == nullptr) {
-        // All types unknown.
-        return true;
-      } else {
-        // Only result type is known, just check that it is one of the types
-        // that can possibly be returned.
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble);
-      }
-    }
-
-    if (result_type == nullptr) {
-      // Right (divisor) argument type is known, left (dividend) argument and
-      // result types are unknown. Just check that it is possible to divide by
-      // the right (divisor) type.
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          right_argument_type->getTypeID(),
-          kInt, kLong, kFloat, kDouble);
-    }
-
-    // Return type and right (divisor) argument type are known, left (dividend)
-    // argument type is unknown. Check that result and divisor are compatible.
-    switch (right_argument_type->getTypeID()) {
-      case kInt:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble);
-      case kLong:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kLong, kDouble);
-      case kFloat:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kFloat, kDouble);
-      case kDouble:
-        return (result_type->getTypeID() == kDouble);
-      default:
-        return false;
-    }
-  } else {  // left_argument_type != nullptr
-    if (right_argument_type == nullptr) {
-      if (result_type == nullptr) {
-        // Left (dividend) argument type is known, right (divisor) argument
-        // type and result type are unknown. Just check that the left
-        // (dividend) type can be divided.
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            left_argument_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble);
-      }
-
-      // Result type and left (dividend) argument type are known, but right
-      // (divisor) argument type is unknown. Check that result and dividend are
-      // compatible.
-      switch (left_argument_type->getTypeID()) {
-        case kInt:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kInt, kLong, kFloat, kDouble);
-        case kLong:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kLong, kDouble);
-        case kFloat:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kFloat, kDouble);
-        case kDouble:
-          return (result_type->getTypeID() == kDouble);
-        default:
-          return false;
-      }
-    }
-
-    // Left and right (dividend and divisor) argument types are both known.
-    const Type *actual_result_type = resultTypeForArgumentTypes(*left_argument_type,
-                                                                *right_argument_type);
-    if (actual_result_type == nullptr) {
-      // Both argument Types are known, but this operation is NOT applicable to
-      // them. No matter what the result_type is, the signature is not
-      // plausible.
-      return false;
-    } else if (result_type == nullptr) {
-      return true;
-    } else {
-      // Check if result type matches.
-      return result_type->equals(*actual_result_type);
-    }
-  }
-}
-
-std::pair<const Type*, const Type*> ModuloBinaryOperation::pushDownTypeHint(
-    const Type *result_type_hint) const {
-  if (result_type_hint == nullptr) {
-    return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-
-  switch (result_type_hint->getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble:
-      return std::pair<const Type*, const Type*>(result_type_hint, result_type_hint);
-    default:
-      // Inapplicable.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-}
-
-TypedValue ModuloBinaryOperation::applyToChecked(const TypedValue &left,
-                                                 const Type &left_type,
-                                                 const TypedValue &right,
-                                                 const Type &right_type) const {
-  switch (left_type.getTypeID()) {
-    case kInt:
-    case kLong: {
-      if (right_type.getTypeID() == TypeID::kInt
-          || right_type.getTypeID() == TypeID::kLong) {
-        return applyToCheckedIntegerHelper<IntegerModuloFunctor>(left, left_type,
-                                                                 right, right_type);
-      }
-    }  // Fall through
-    case kFloat:
-    case kDouble: {
-      if (right_type.getSuperTypeID() == Type::kNumeric) {
-        return applyToCheckedNumericHelper<FloatModuloFunctor>(left, left_type,
-                                                               right, right_type);
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-             << left_type.getName() << " and " << right_type.getName();
-}
-
-UncheckedBinaryOperator* ModuloBinaryOperation::makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                                                    const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-    case kLong: {
-      if (right.getTypeID() == TypeID::kInt
-          || right.getTypeID() == TypeID::kLong) {
-        return makeIntegerBinaryOperatorOuterHelper<IntegerModuloArithmeticUncheckedBinaryOperator>(left, right);
-      }
-    }  // Fall through
-    case kFloat:
-    case kDouble: {
-      if (right.getSuperTypeID() == Type::kNumeric) {
-        return makeNumericBinaryOperatorOuterHelper<FloatModuloArithmeticUncheckedBinaryOperator>(left, right);
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/ModuloBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ModuloBinaryOperation.hpp b/types/operations/binary_operations/ModuloBinaryOperation.hpp
deleted file mode 100644
index 1d47a45..0000000
--- a/types/operations/binary_operations/ModuloBinaryOperation.hpp
+++ /dev/null
@@ -1,93 +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_TYPES_OPERATIONS_BINARY_OPERATIONS_MODULO_BINARY_OPERATION_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_MODULO_BINARY_OPERATION_HPP_
-
-#include <utility>
-
-#include "types/TypedValue.hpp"
-#include "types/operations/binary_operations/ArithmeticBinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-class Type;
-class UncheckedBinaryOperator;
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief The BinaryOperation for modulo.
- *
- * @note ModuloBinaryOperation is not commutative: the left argument is the
- *       dividend and the right argument is the divisor.
- **/
-class ModuloBinaryOperation : public ArithmeticBinaryOperation {
- public:
-  /**
-   * @brief Get a reference to the singleton instance of this Operation.
-   *
-   * @return A reference to the singleton instance of this Operation.
-   **/
-  static const ModuloBinaryOperation& Instance() {
-    static ModuloBinaryOperation instance;
-    return instance;
-  }
-
-  bool canApplyToTypes(const Type &left,
-                       const Type &right) const override;
-
-  const Type* resultTypeForArgumentTypes(const Type &left,
-                                         const Type &right) const override;
-
-  const Type* resultTypeForPartialArgumentTypes(const Type *left,
-                                                const Type *right) const override;
-
-  bool partialTypeSignatureIsPlausible(const Type *result_type,
-                                       const Type *left_argument_type,
-                                       const Type *right_argument_type) const override;
-
-  std::pair<const Type*, const Type*> pushDownTypeHint(
-      const Type *result_type_hint) const override;
-
-  TypedValue applyToChecked(const TypedValue &left,
-                            const Type &left_type,
-                            const TypedValue &right,
-                            const Type &right_type) const override;
-
-  UncheckedBinaryOperator* makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                               const Type &right) const override;
-
- private:
-  ModuloBinaryOperation()
-      : ArithmeticBinaryOperation(BinaryOperationID::kModulo) {
-  }
-
-  DISALLOW_COPY_AND_ASSIGN(ModuloBinaryOperation);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_MODULO_BINARY_OPERATION_HPP_


[05/32] incubator-quickstep git commit: Add common-subexpression support.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/tests/physical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Select.test b/query_optimizer/tests/physical_generator/Select.test
index f7de922..614347b 100644
--- a/query_optimizer/tests/physical_generator/Select.test
+++ b/query_optimizer/tests/physical_generator/Select.test
@@ -1022,34 +1022,51 @@ TopLevelPlan
 [Physical Plan]
 TopLevelPlan
 +-plan=Selection
-| +-input=Aggregate
-| | +-input=TableReference[relation=Test,alias=test]
-| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
-| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
-| | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
-| | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
-| | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
-| | | +-AttributeReference[id=5,name=vchar_col,relation=test,
-| | |   type=VarChar(20) NULL]
-| | +-grouping_expressions=
-| | | +-[]
-| | +-aggregate_expressions=
-| |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Long]
-| |   | +-AggregateFunction[function=COUNT]
-| |   |   +-[]
-| |   +-Alias[id=7,name=,alias=$aggregate1,relation=$aggregate,type=Long]
-| |   | +-AggregateFunction[function=COUNT]
-| |   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
-| |   +-Alias[id=8,name=,alias=$aggregate2,relation=$aggregate,type=Long NULL]
-| |   | +-AggregateFunction[function=SUM]
-| |   |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
-| |   +-Alias[id=9,name=,alias=$aggregate3,relation=$aggregate,type=Double NULL]
-| |   | +-AggregateFunction[function=AVG]
-| |   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
-| |   +-Alias[id=11,name=,alias=$aggregate4,relation=$aggregate,type=Double NULL]
-| |     +-AggregateFunction[function=MAX]
-| |       +-AttributeReference[id=3,name=double_col,relation=test,
-| |         type=Double NULL]
+| +-input=Selection
+| | +-input=Aggregate
+| | | +-input=TableReference[relation=Test,alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-grouping_expressions=
+| | | | +-[]
+| | | +-aggregate_expressions=
+| | |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Long]
+| | |   | +-AggregateFunction[function=COUNT]
+| | |   |   +-[]
+| | |   +-Alias[id=7,name=,alias=$aggregate1,relation=$aggregate,type=Long]
+| | |   | +-AggregateFunction[function=COUNT]
+| | |   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |   +-Alias[id=8,name=,alias=$aggregate2,relation=$aggregate,type=Long NULL]
+| | |   | +-AggregateFunction[function=SUM]
+| | |   |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | |   +-Alias[id=12,name=,alias=$aggregate3,relation=$aggregate,type=Long NULL]
+| | |   | +-AggregateFunction[function=SUM]
+| | |   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |   +-Alias[id=11,name=,alias=$aggregate4,relation=$aggregate,
+| | |     type=Double NULL]
+| | |     +-AggregateFunction[function=MAX]
+| | |       +-AttributeReference[id=3,name=double_col,relation=test,
+| | |         type=Double NULL]
+| | +-project_expressions=
+| |   +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+| |   | type=Long]
+| |   +-AttributeReference[id=7,name=,alias=$aggregate1,relation=$aggregate,
+| |   | type=Long]
+| |   +-AttributeReference[id=8,name=,alias=$aggregate2,relation=$aggregate,
+| |   | type=Long NULL]
+| |   +-Alias[id=9,name=,alias=$aggregate3,relation=$aggregate,type=Long NULL]
+| |   | +-Divide
+| |   |   +-AttributeReference[id=12,name=,alias=$aggregate3,
+| |   |   | relation=$aggregate,type=Long NULL]
+| |   |   +-AttributeReference[id=7,name=,alias=$aggregate1,relation=$aggregate,
+| |   |     type=Long]
+| |   +-AttributeReference[id=11,name=,alias=$aggregate4,relation=$aggregate,
+| |     type=Double NULL]
 | +-filter_predicate=Greater
 | | +-Add
 | | | +-AttributeReference[id=11,name=,alias=$aggregate4,relation=$aggregate,
@@ -1311,31 +1328,40 @@ TopLevelPlan
 | |   +-Alias[id=8,name=,alias=$aggregate0,relation=$aggregate,type=Long]
 | |   | +-AggregateFunction[function=COUNT]
 | |   |   +-Add
-| |   |     +-Add
-| |   |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
-| |   |     | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
-| |   |     +-Add
-| |   |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
-| |   |       +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |   |     +-CommonSubexpression[common_subexpression_id=13]
+| |   |     | +-Operand=Add
+| |   |     |   +-AttributeReference[id=0,name=int_col,relation=test,
+| |   |     |   | type=Int NULL]
+| |   |     |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |   |     +-CommonSubexpression[common_subexpression_id=14]
+| |   |       +-Operand=Add
+| |   |         +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |   |         +-AttributeReference[id=2,name=float_col,relation=test,
+| |   |           type=Float]
 | |   +-Alias[id=9,name=,alias=$aggregate1,relation=$aggregate,type=Long NULL]
 | |   | +-AggregateFunction[function=MAX]
 | |   |   +-Divide
-| |   |     +-Add
-| |   |     | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
-| |   |     | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |   |     +-CommonSubexpression[common_subexpression_id=13]
+| |   |     | +-Operand=Add
+| |   |     |   +-AttributeReference[id=0,name=int_col,relation=test,
+| |   |     |   | type=Int NULL]
+| |   |     |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
 | |   |     +-Literal[value=2,type=Int]
 | |   +-Alias[id=11,name=,alias=$aggregate2,relation=$aggregate,type=Long NULL]
 | |   | +-AggregateFunction[function=MAX]
 | |   |   +-Add
 | |   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
-| |   |     +-Add
-| |   |       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
-| |   |       +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |   |     +-CommonSubexpression[common_subexpression_id=13]
+| |   |       +-Operand=Add
+| |   |         +-AttributeReference[id=0,name=int_col,relation=test,
+| |   |         | type=Int NULL]
+| |   |         +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
 | |   +-Alias[id=12,name=,alias=$aggregate3,relation=$aggregate,type=Double NULL]
 | |     +-AggregateFunction[function=SUM]
-| |       +-Add
-| |         +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
-| |         +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |       +-CommonSubexpression[common_subexpression_id=14]
+| |         +-Operand=Add
+| |           +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |           +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
 | +-filter_predicate=Greater
 | | +-AttributeReference[id=11,name=,alias=$aggregate2,relation=$aggregate,
 | | | type=Long NULL]

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 39538ea..a6a3cd0 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -282,6 +282,7 @@ target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_utility_ColumnVectorCache
                       quickstep_utility_Macros
                       quickstep_utility_lipfilter_LIPFilterAdaptiveProber
                       quickstep_utility_lipfilter_LIPFilterUtil
@@ -331,6 +332,7 @@ target_link_libraries(quickstep_relationaloperators_NestedLoopsJoinOperator
                       quickstep_storage_TupleStorageSubBlock
                       quickstep_storage_ValueAccessor
                       quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_utility_ColumnVectorCache
                       quickstep_utility_Macros
                       tmb)
 target_link_libraries(quickstep_relationaloperators_RebuildWorkOrder

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index 0e75411..ea90828 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -50,6 +50,7 @@
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/ColumnVectorCache.hpp"
 #include "utility/lip_filter/LIPFilterAdaptiveProber.hpp"
 #include "utility/lip_filter/LIPFilterUtil.hpp"
 
@@ -532,6 +533,7 @@ void HashInnerJoinWorkOrder::executeWithoutCopyElision(ValueAccessor *probe_acce
     }
 
     ColumnVectorsValueAccessor temp_result;
+    std::unique_ptr<ColumnVectorCache> cv_cache = std::make_unique<ColumnVectorCache>();
     for (auto selection_cit = selection_.begin();
          selection_cit != selection_.end();
          ++selection_cit) {
@@ -539,8 +541,10 @@ void HashInnerJoinWorkOrder::executeWithoutCopyElision(ValueAccessor *probe_acce
                                                                   build_accessor.get(),
                                                                   probe_relation_id,
                                                                   probe_accessor,
-                                                                  build_block_entry.second));
+                                                                  build_block_entry.second,
+                                                                  cv_cache.get()));
     }
+    cv_cache.reset();
 
     output_destination_->bulkInsertTuples(&temp_result);
   }
@@ -649,12 +653,14 @@ void HashInnerJoinWorkOrder::executeWithCopyElision(ValueAccessor *probe_accesso
         zipped_joined_tuple_ids.emplace_back(build_tids[i], probe_tids[i]);
       }
 
+      ColumnVectorCache cv_cache;
       for (const Scalar *scalar : non_trivial_expressions) {
         temp_result.addColumn(scalar->getAllValuesForJoin(build_relation_id,
                                                           build_accessor.get(),
                                                           probe_relation_id,
                                                           probe_accessor,
-                                                          zipped_joined_tuple_ids));
+                                                          zipped_joined_tuple_ids,
+                                                          &cv_cache));
       }
     }
 
@@ -765,13 +771,16 @@ void HashSemiJoinWorkOrder::executeWithResidualPredicate() {
 
   std::unique_ptr<ValueAccessor> probe_accessor_with_filter(
       probe_store.createValueAccessor(&filter));
+
   ColumnVectorsValueAccessor temp_result;
+  std::unique_ptr<ColumnVectorCache> cv_cache = std::make_unique<ColumnVectorCache>();
   for (vector<unique_ptr<const Scalar>>::const_iterator selection_it = selection_.begin();
        selection_it != selection_.end();
        ++selection_it) {
     temp_result.addColumn((*selection_it)->getAllValues(
-        probe_accessor_with_filter.get(), &sub_blocks_ref));
+        probe_accessor_with_filter.get(), &sub_blocks_ref, cv_cache.get()));
   }
+  cv_cache.reset();
 
   output_destination_->bulkInsertTuples(&temp_result);
 }
@@ -828,12 +837,15 @@ void HashSemiJoinWorkOrder::executeWithoutResidualPredicate() {
 
   std::unique_ptr<ValueAccessor> probe_accessor_with_filter(
       probe_accessor->createSharedTupleIdSequenceAdapterVirtual(*existence_map));
+
   ColumnVectorsValueAccessor temp_result;
+  std::unique_ptr<ColumnVectorCache> cv_cache = std::make_unique<ColumnVectorCache>();
   for (vector<unique_ptr<const Scalar>>::const_iterator selection_it = selection_.begin();
        selection_it != selection_.end(); ++selection_it) {
     temp_result.addColumn((*selection_it)->getAllValues(
-        probe_accessor_with_filter.get(), &sub_blocks_ref));
+        probe_accessor_with_filter.get(), &sub_blocks_ref, cv_cache.get()));
   }
+  cv_cache.reset();
 
   output_destination_->bulkInsertTuples(&temp_result);
 }
@@ -886,12 +898,15 @@ void HashAntiJoinWorkOrder::executeWithoutResidualPredicate() {
 
   std::unique_ptr<ValueAccessor> probe_accessor_with_filter(
       probe_accessor->createSharedTupleIdSequenceAdapterVirtual(*existence_map));
+
   ColumnVectorsValueAccessor temp_result;
+  std::unique_ptr<ColumnVectorCache> cv_cache = std::make_unique<ColumnVectorCache>();
   for (vector<unique_ptr<const Scalar>>::const_iterator selection_it = selection_.begin();
        selection_it != selection_.end(); ++selection_it) {
     temp_result.addColumn((*selection_it)->getAllValues(
-        probe_accessor_with_filter.get(), &sub_blocks_ref));
+        probe_accessor_with_filter.get(), &sub_blocks_ref, cv_cache.get()));
   }
+  cv_cache.reset();
 
   output_destination_->bulkInsertTuples(&temp_result);
 }
@@ -976,14 +991,18 @@ void HashAntiJoinWorkOrder::executeWithResidualPredicate() {
 
   std::unique_ptr<ValueAccessor> probe_accessor_with_filter(
       probe_accessor->createSharedTupleIdSequenceAdapterVirtual(*existence_map));
+
   ColumnVectorsValueAccessor temp_result;
+  std::unique_ptr<ColumnVectorCache> cv_cache = std::make_unique<ColumnVectorCache>();
   for (vector<unique_ptr<const Scalar>>::const_iterator selection_it = selection_.begin();
        selection_it != selection_.end();
        ++selection_it) {
     temp_result.addColumn(
         (*selection_it)->getAllValues(probe_accessor_with_filter.get(),
-                                      &sub_blocks_ref));
+                                      &sub_blocks_ref,
+                                      cv_cache.get()));
   }
+  cv_cache.reset();
 
   output_destination_->bulkInsertTuples(&temp_result);
 }
@@ -1032,12 +1051,11 @@ void HashOuterJoinWorkOrder::execute() {
            &build_block_entry : *collector.getJoinedTupleMap()) {
     const BlockReference build_block =
         storage_manager_->getBlock(build_block_entry.first, build_relation_);
-    const TupleStorageSubBlock &build_store =
-        build_block->getTupleStorageSubBlock();
+    const TupleStorageSubBlock &build_store = build_block->getTupleStorageSubBlock();
+    std::unique_ptr<ValueAccessor> build_accessor(build_store.createValueAccessor());
 
-    std::unique_ptr<ValueAccessor> build_accessor(
-        build_store.createValueAccessor());
     ColumnVectorsValueAccessor temp_result;
+    std::unique_ptr<ColumnVectorCache> cv_cache = std::make_unique<ColumnVectorCache>();
     for (auto selection_it = selection_.begin();
          selection_it != selection_.end();
          ++selection_it) {
@@ -1047,8 +1065,11 @@ void HashOuterJoinWorkOrder::execute() {
               build_accessor.get(),
               probe_relation_id,
               probe_accessor.get(),
-              build_block_entry.second));
+              build_block_entry.second,
+              cv_cache.get()));
     }
+    cv_cache.reset();
+
     output_destination_->bulkInsertTuples(&temp_result);
   }
 
@@ -1061,8 +1082,9 @@ void HashOuterJoinWorkOrder::execute() {
   if (num_tuples_without_matches > 0) {
     std::unique_ptr<ValueAccessor> probe_accessor_with_filter(
         probe_accessor->createSharedTupleIdSequenceAdapterVirtual(*existence_map));
-    ColumnVectorsValueAccessor temp_result;
 
+    ColumnVectorsValueAccessor temp_result;
+    std::unique_ptr<ColumnVectorCache> cv_cache = std::make_unique<ColumnVectorCache>();
     for (std::size_t i = 0; i < selection_.size(); ++i) {
       if (is_selection_on_build_[i]) {
         // NOTE(harshad, jianqiao): The assumption here is that any operation
@@ -1090,9 +1112,12 @@ void HashOuterJoinWorkOrder::execute() {
       } else {
         temp_result.addColumn(
             selection_[i]->getAllValues(probe_accessor_with_filter.get(),
-                                        &sub_blocks_ref));
+                                        &sub_blocks_ref,
+                                        cv_cache.get()));
       }
     }
+    cv_cache.reset();
+
     output_destination_->bulkInsertTuples(&temp_result);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/relational_operators/NestedLoopsJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.cpp b/relational_operators/NestedLoopsJoinOperator.cpp
index f17402f..4ef2a70 100644
--- a/relational_operators/NestedLoopsJoinOperator.cpp
+++ b/relational_operators/NestedLoopsJoinOperator.cpp
@@ -38,6 +38,7 @@
 #include "storage/TupleStorageSubBlock.hpp"
 #include "storage/ValueAccessor.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/ColumnVectorCache.hpp"
 
 #include "glog/logging.h"
 
@@ -417,6 +418,7 @@ void NestedLoopsJoinWorkOrder::executeHelper(const TupleStorageSubBlock &left_st
     // evaluation and data movement, but low enough that temporary memory
     // requirements don't get out of hand).
     ColumnVectorsValueAccessor temp_result;
+    std::unique_ptr<ColumnVectorCache> cv_cache = std::make_unique<ColumnVectorCache>();
     for (vector<unique_ptr<const Scalar>>::const_iterator selection_cit = selection_.begin();
          selection_cit != selection_.end();
          ++selection_cit) {
@@ -424,8 +426,10 @@ void NestedLoopsJoinWorkOrder::executeHelper(const TupleStorageSubBlock &left_st
                                                                   left_accessor.get(),
                                                                   right_input_relation_id,
                                                                   right_accessor.get(),
-                                                                  joined_tuple_ids));
+                                                                  joined_tuple_ids,
+                                                                  cv_cache.get()));
     }
+    cv_cache.reset();
 
     output_destination_->bulkInsertTuples(&temp_result);
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index 90543c4..e5dc93e 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -57,6 +57,7 @@
 #include "types/containers/ColumnVector.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
 #include "types/containers/Tuple.hpp"
+#include "utility/ColumnVectorCache.hpp"
 #include "utility/lip_filter/LIPFilterAdaptiveProber.hpp"
 
 #include "gflags/gflags.h"
@@ -491,9 +492,10 @@ void AggregationOperationState::aggregateBlock(const block_id input_block,
     SubBlocksReference sub_blocks_ref(tuple_store,
                                       block->getIndices(),
                                       block->getIndicesConsistent());
+    ColumnVectorCache cv_cache;
     for (const auto &expression : non_trivial_expressions_) {
       non_trivial_results->addColumn(
-          expression->getAllValues(accessor, &sub_blocks_ref));
+          expression->getAllValues(accessor, &sub_blocks_ref, &cv_cache));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index cb1f098..c3db584 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -297,6 +297,7 @@ target_link_libraries(quickstep_storage_AggregationOperationState
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorsValueAccessor
                       quickstep_types_containers_Tuple
+                      quickstep_utility_ColumnVectorCache
                       quickstep_utility_Macros
                       quickstep_utility_lipfilter_LIPFilterAdaptiveProber)
 target_link_libraries(quickstep_storage_AggregationOperationState_proto
@@ -961,6 +962,7 @@ target_link_libraries(quickstep_storage_StorageBlock
                       quickstep_types_containers_ColumnVectorsValueAccessor
                       quickstep_types_containers_Tuple
                       quickstep_types_operations_comparisons_ComparisonUtil
+                      quickstep_utility_ColumnVectorCache
                       quickstep_utility_Macros
                       quickstep_utility_PtrVector)
 # CMAKE_VALIDATE_IGNORE_BEGIN
@@ -1100,6 +1102,7 @@ target_link_libraries(quickstep_storage_WindowAggregationOperationState
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorUtil
                       quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_utility_ColumnVectorCache
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_storage_WindowAggregationOperationState_proto
                       quickstep_expressions_Expressions_proto

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/storage/StorageBlock.cpp
----------------------------------------------------------------------
diff --git a/storage/StorageBlock.cpp b/storage/StorageBlock.cpp
index e91c1ac..31f1db2 100644
--- a/storage/StorageBlock.cpp
+++ b/storage/StorageBlock.cpp
@@ -56,6 +56,7 @@
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
 #include "types/containers/Tuple.hpp"
 #include "types/operations/comparisons/ComparisonUtil.hpp"
+#include "utility/ColumnVectorCache.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
@@ -369,15 +370,18 @@ void StorageBlock::select(const vector<unique_ptr<const Scalar>> &selection,
                                       indices_,
                                       indices_consistent_);
 
-    std::unique_ptr<ValueAccessor> accessor(
-        tuple_store_->createValueAccessor(filter));
+    std::unique_ptr<ValueAccessor> accessor(tuple_store_->createValueAccessor(filter));
+    ColumnVectorCache cv_cache;
 
     for (vector<unique_ptr<const Scalar>>::const_iterator selection_cit = selection.begin();
          selection_cit != selection.end();
          ++selection_cit) {
       // TODO(chasseur): Can probably elide some copies for parts of the
       // selection that are ScalarAttribute or ScalarLiteral.
-      temp_result.addColumn((*selection_cit)->getAllValues(accessor.get(), &sub_blocks_ref));
+      temp_result.addColumn(
+          (*selection_cit)->getAllValues(accessor.get(),
+                                         &sub_blocks_ref,
+                                         &cv_cache));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/storage/WindowAggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/WindowAggregationOperationState.cpp b/storage/WindowAggregationOperationState.cpp
index 58bdf18..30d91db 100644
--- a/storage/WindowAggregationOperationState.cpp
+++ b/storage/WindowAggregationOperationState.cpp
@@ -46,6 +46,7 @@
 #include "types/containers/ColumnVector.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
 #include "types/containers/ColumnVectorUtil.hpp"
+#include "utility/ColumnVectorCache.hpp"
 
 #include "glog/logging.h"
 
@@ -236,11 +237,16 @@ void WindowAggregationOperationState::windowAggregateBlocks(
       argument_accessor = new ColumnVectorsValueAccessor();
     }
 
+    std::unique_ptr<ColumnVectorCache> cv_cache = std::make_unique<ColumnVectorCache>();
     for (const std::unique_ptr<const Scalar> &argument : arguments_) {
       argument_accessor->addColumn(argument->getAllValues(tuple_accessor,
-                                                          &sub_block_ref));
+                                                          &sub_block_ref,
+                                                          cv_cache.get()));
     }
 
+    // Release common subexpression cache as early as possible.
+    cv_cache.reset();
+
     InvokeOnAnyValueAccessor(tuple_accessor,
                              [&] (auto *tuple_accessor) -> void {  // NOLINT(build/c++11)
       tuple_accessor->beginIteration();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/types/containers/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/containers/CMakeLists.txt b/types/containers/CMakeLists.txt
index c2a6623..97841c2 100644
--- a/types/containers/CMakeLists.txt
+++ b/types/containers/CMakeLists.txt
@@ -42,8 +42,7 @@ target_link_libraries(quickstep_types_containers_ColumnVectorsValueAccessor
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_Tuple
-                      quickstep_utility_Macros
-                      quickstep_utility_ScopedDeleter)
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_types_containers_Tuple
                       glog
                       quickstep_catalog_CatalogTypedefs

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/types/containers/ColumnVector.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.hpp b/types/containers/ColumnVector.hpp
index fc65656..5ef9871 100644
--- a/types/containers/ColumnVector.hpp
+++ b/types/containers/ColumnVector.hpp
@@ -43,6 +43,9 @@ namespace quickstep {
 // TODO(chasseur): Look into ways to allocate ColumnVector memory from the
 // StorageManager.
 
+class ColumnVector;
+typedef std::shared_ptr<const ColumnVector> ColumnVectorPtr;
+
 /**
  * @brief A vector of values of the same type. Two implementations exist:
  *        NativeColumnVector (an array of fixed-size data elements) and
@@ -107,6 +110,13 @@ class ColumnVector {
    **/
   virtual bool isNative() const = 0;
 
+  /**
+   * @brief Get the number of values in this ColumnVector.
+   *
+   * @return The number of values in this ColumnVector.
+   **/
+  virtual std::size_t size() const = 0;
+
  protected:
   const Type &type_;
 
@@ -176,7 +186,7 @@ class NativeColumnVector : public ColumnVector {
    *
    * @return The number of values in this NativeColumnVector.
    **/
-  inline std::size_t size() const {
+  inline std::size_t size() const override {
     return actual_length_;
   }
 
@@ -436,7 +446,7 @@ class IndirectColumnVector : public ColumnVector {
    *
    * @return The number of values in this IndirectColumnVector.
    **/
-  inline std::size_t size() const {
+  inline std::size_t size() const override {
     return values_.size();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/types/containers/ColumnVectorsValueAccessor.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVectorsValueAccessor.hpp b/types/containers/ColumnVectorsValueAccessor.hpp
index 6dc1124..ebd46d4 100644
--- a/types/containers/ColumnVectorsValueAccessor.hpp
+++ b/types/containers/ColumnVectorsValueAccessor.hpp
@@ -32,7 +32,6 @@
 #include "types/containers/ColumnVector.hpp"
 #include "types/containers/Tuple.hpp"
 #include "utility/Macros.hpp"
-#include "utility/ScopedDeleter.hpp"
 
 #include "glog/logging.h"
 
@@ -74,22 +73,17 @@ class ColumnVectorsValueAccessor : public ValueAccessor {
    *             this value-accessor is responsible for freeing this column
    *             vector.
    **/
-  void addColumn(ColumnVector *column, const bool owns = true) {
+  void addColumn(ColumnVectorPtr column) {
     // If this is not the first column to be added, make sure it is the same
     // length as the others.
-    DCHECK(columns_.empty()
-           || (column->isNative()
-               ? (static_cast<const NativeColumnVector*>(column)->size() == column_length_)
-               : (static_cast<const IndirectColumnVector*>(column)->size() == column_length_)));
+    DCHECK(columns_.empty() || column->size() == column_length_);
     columns_.push_back(column);
     column_native_.push_back(column->isNative());
-    if (owns) {
-      deleter_.addObject(column);
-    }
-    column_length_
-        = column->isNative()
-          ? static_cast<const NativeColumnVector*>(column)->size()
-          : static_cast<const IndirectColumnVector*>(column)->size();
+    column_length_ = column->size();
+  }
+
+  void addColumn(ColumnVector *column) {
+    addColumn(ColumnVectorPtr(column));
   }
 
   inline void beginIteration() {
@@ -309,11 +303,10 @@ class ColumnVectorsValueAccessor : public ValueAccessor {
            && (static_cast<std::vector<ColumnVector*>::size_type>(attr_id) < columns_.size());
   }
 
-  std::vector<ColumnVector*> columns_;
+  std::vector<ColumnVectorPtr> columns_;
   std::vector<bool> column_native_;
   std::size_t column_length_;
   std::size_t current_position_;
-  ScopedDeleter deleter_;
 
   DISALLOW_COPY_AND_ASSIGN(ColumnVectorsValueAccessor);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/types/operations/binary_operations/AddBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/AddBinaryOperation.hpp b/types/operations/binary_operations/AddBinaryOperation.hpp
index bc862bf..2309563 100644
--- a/types/operations/binary_operations/AddBinaryOperation.hpp
+++ b/types/operations/binary_operations/AddBinaryOperation.hpp
@@ -51,6 +51,10 @@ class AddBinaryOperation : public ArithmeticBinaryOperation {
     return instance;
   }
 
+  bool isCommutative() const override {
+    return true;
+  }
+
   bool canApplyToTypes(const Type &left,
                        const Type &right) const override;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/types/operations/binary_operations/BinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperation.hpp b/types/operations/binary_operations/BinaryOperation.hpp
index 585a1c6..bc8a083 100644
--- a/types/operations/binary_operations/BinaryOperation.hpp
+++ b/types/operations/binary_operations/BinaryOperation.hpp
@@ -335,6 +335,19 @@ class BinaryOperation : public Operation {
   }
 
   /**
+   * @brief Whether this binary operation is commutative.
+   *
+   * @note The commutative property provides more optimization opportunities,
+   *       e.g. common subexpression elimination. Meanwhile it is always safe
+   *       to assume that a binary operation is not commutative.
+   *
+   * @return True if this binary operation is commutative; false otherwise.
+   */
+  virtual bool isCommutative() const {
+    return false;
+  }
+
+  /**
    * @brief Determine whether this BinaryOperation can apply to the specified
    *        Types.
    * @note When the Types that an operator can apply to are changed,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/types/operations/binary_operations/MultiplyBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/MultiplyBinaryOperation.hpp b/types/operations/binary_operations/MultiplyBinaryOperation.hpp
index 6edc999..cc005e2 100644
--- a/types/operations/binary_operations/MultiplyBinaryOperation.hpp
+++ b/types/operations/binary_operations/MultiplyBinaryOperation.hpp
@@ -51,6 +51,10 @@ class MultiplyBinaryOperation : public ArithmeticBinaryOperation {
     return instance;
   }
 
+  bool isCommutative() const override {
+    return true;
+  }
+
   bool canApplyToTypes(const Type &left,
                        const Type &right) const override;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index ca04462..e1fb770 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -171,6 +171,7 @@ add_library(quickstep_utility_BloomFilter_proto
 add_library(quickstep_utility_CalculateInstalledMemory CalculateInstalledMemory.cpp CalculateInstalledMemory.hpp)
 add_library(quickstep_utility_Cast ../empty_src.cpp Cast.hpp)
 add_library(quickstep_utility_CheckSnprintf ../empty_src.cpp CheckSnprintf.hpp)
+add_library(quickstep_utility_ColumnVectorCache ../empty_src.cpp ColumnVectorCache.hpp)
 add_library(quickstep_utility_CompositeHash ../empty_src.cpp CompositeHash.hpp)
 add_library(quickstep_utility_BarrieredReadWriteConcurrentBitVector
             ../empty_src.cpp
@@ -182,6 +183,7 @@ add_library(quickstep_utility_ExecutionDAGVisualizer
             ExecutionDAGVisualizer.cpp
             ExecutionDAGVisualizer.hpp)
 add_library(quickstep_utility_Glob Glob.cpp Glob.hpp)
+add_library(quickstep_utility_HashError ../empty_src.cpp HashError.hpp)
 add_library(quickstep_utility_HashPair ../empty_src.cpp HashPair.hpp)
 add_library(quickstep_utility_Macros ../empty_src.cpp Macros.hpp)
 add_library(quickstep_utility_MemStream ../empty_src.cpp MemStream.hpp)
@@ -237,6 +239,9 @@ target_link_libraries(quickstep_utility_CalculateInstalledMemory
                       glog)
 target_link_libraries(quickstep_utility_CheckSnprintf
                       glog)
+target_link_libraries(quickstep_utility_ColumnVectorCache
+                      quickstep_types_containers_ColumnVector
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_CompositeHash
                       quickstep_types_TypedValue
                       quickstep_utility_HashPair
@@ -344,12 +349,14 @@ target_link_libraries(quickstep_utility
                       quickstep_utility_CalculateInstalledMemory
                       quickstep_utility_Cast
                       quickstep_utility_CheckSnprintf
+                      quickstep_utility_ColumnVectorCache
                       quickstep_utility_CompositeHash
                       quickstep_utility_DAG
                       quickstep_utility_DisjointTreeForest
                       quickstep_utility_EqualsAnyConstant
                       quickstep_utility_ExecutionDAGVisualizer
                       quickstep_utility_Glob
+                      quickstep_utility_HashError
                       quickstep_utility_HashPair
                       quickstep_utility_Macros
                       quickstep_utility_MemStream

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/utility/ColumnVectorCache.hpp
----------------------------------------------------------------------
diff --git a/utility/ColumnVectorCache.hpp b/utility/ColumnVectorCache.hpp
new file mode 100644
index 0000000..46e742c
--- /dev/null
+++ b/utility/ColumnVectorCache.hpp
@@ -0,0 +1,89 @@
+/**
+ * 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_COLUMN_VECTOR_CACHE_HPP_
+#define QUICKSTEP_UTILITY_COLUMN_VECTOR_CACHE_HPP_
+
+#include <unordered_map>
+
+#include "types/containers/ColumnVector.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief A memoization table for column vectors parameterized on integer IDs.
+ **/
+class ColumnVectorCache {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  ColumnVectorCache() {}
+
+  /**
+   * @brief Check whether this cache contains the column vector parameterized on
+   *        the given ID.
+   *
+   * @param share_id The integer ID for the column vector.
+   * @return True if this cache contains the column vector; false otherwise.
+   */
+  inline bool contains(const int share_id) const {
+    return cv_cache_.find(share_id) != cv_cache_.end();
+  }
+
+  /**
+   * @brief Get the cached column vector parameterized on the given ID.
+   *
+   * @param share_id The integer ID for the column vector.
+   * @return The cached column vector.
+   */
+  inline ColumnVectorPtr get(const int share_id) const {
+    DCHECK(contains(share_id));
+    return cv_cache_.at(share_id);
+  }
+
+  /**
+   * @brief Cache the column vector with the given ID as parameter.
+   *
+   * @param share_id The integer ID for the column vector.
+   * @param cv The column vector to be cached.
+   */
+  inline void set(const int share_id, const ColumnVectorPtr &cv) {
+    DCHECK(!contains(share_id));
+    cv_cache_.emplace(share_id, cv);
+  }
+
+ private:
+  std::unordered_map<int, ColumnVectorPtr> cv_cache_;
+
+  DISALLOW_COPY_AND_ASSIGN(ColumnVectorCache);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_COLUMN_VECTOR_CACHE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/utility/HashError.hpp
----------------------------------------------------------------------
diff --git a/utility/HashError.hpp b/utility/HashError.hpp
new file mode 100644
index 0000000..1f7b1b8
--- /dev/null
+++ b/utility/HashError.hpp
@@ -0,0 +1,62 @@
+/**
+ * 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_HASH_ERROR_HPP_
+#define QUICKSTEP_UTILITY_HASH_ERROR_HPP_
+
+#include <exception>
+#include <string>
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+/**
+ * @brief Exception thrown for non-supported hash().
+ **/
+class HashNotSupported : public std::exception {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param message The error message.
+   **/
+  explicit HashNotSupported(const std::string &message)
+      : message_(message) {}
+
+  /**
+   * @brief Destructor.
+   */
+  ~HashNotSupported() throw() {}
+
+  virtual const char* what() const throw() {
+    return message_.c_str();
+  }
+
+ private:
+  const std::string message_;
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_HASH_ERROR_HPP_



[14/32] incubator-quickstep git commit: Fixed a bug regarding the table name containing '_' in Commands.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/678e2c39/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index 857eaaf..a6d12e2 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -186,7 +186,7 @@ extern int quickstep_yydebug;
 
 union YYSTYPE
 {
-#line 121 "../SqlParser.ypp" /* yacc.c:1909  */
+#line 121 "../SqlParser.ypp" /* yacc.c:1915  */
 
   quickstep::ParseString *string_value_;
 
@@ -288,7 +288,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 292 "SqlParser_gen.hpp" /* yacc.c:1909  */
+#line 292 "SqlParser_gen.hpp" /* yacc.c:1915  */
 };
 
 typedef union YYSTYPE YYSTYPE;


[26/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/CMakeLists.txt b/types/CMakeLists.txt
index 769187b..df4462f 100644
--- a/types/CMakeLists.txt
+++ b/types/CMakeLists.txt
@@ -32,6 +32,8 @@ QS_PROTOBUF_GENERATE_CPP(types_TypedValue_proto_srcs types_TypedValue_proto_hdrs
 QS_PROTOBUF_GENERATE_CPP(types_Type_proto_srcs types_Type_proto_hdrs Type.proto)
 
 # Declare micro-libs:
+add_library(quickstep_types_AsciiStringSuperType ../empty_src.cpp AsciiStringSuperType.hpp)
+add_library(quickstep_types_BoolType BoolType.cpp BoolType.hpp)
 add_library(quickstep_types_CharType CharType.cpp CharType.hpp)
 add_library(quickstep_types_DateOperatorOverloads ../empty_src.cpp DateOperatorOverloads.hpp)
 add_library(quickstep_types_DateType DateType.cpp DateType.hpp)
@@ -47,11 +49,16 @@ add_library(quickstep_types_LongType LongType.cpp LongType.hpp)
 add_library(quickstep_types_NullCoercibilityCheckMacro ../empty_src.cpp NullCoercibilityCheckMacro.hpp)
 add_library(quickstep_types_NullType ../empty_src.cpp NullType.hpp)
 add_library(quickstep_types_NumericSuperType ../empty_src.cpp NumericSuperType.hpp)
+add_library(quickstep_types_NumericTypeSafeCoercibility ../empty_src.cpp NumericTypeSafeCoercibility.hpp)
 add_library(quickstep_types_NumericTypeUnifier ../empty_src.cpp NumericTypeUnifier.hpp)
 add_library(quickstep_types_Type Type.cpp Type.hpp)
 add_library(quickstep_types_TypeErrors ../empty_src.cpp TypeErrors.hpp)
 add_library(quickstep_types_TypeFactory TypeFactory.cpp TypeFactory.hpp)
 add_library(quickstep_types_TypeID TypeID.cpp TypeID.hpp)
+add_library(quickstep_types_TypeIDSelectors ../empty_src.cpp TypeIDSelectors.hpp)
+add_library(quickstep_types_TypeRegistrar ../empty_src.cpp TypeRegistrar.hpp)
+add_library(quickstep_types_TypeSynthesizer ../empty_src.cpp TypeSynthesizer.hpp)
+add_library(quickstep_types_TypeUtil ../empty_src.cpp TypeUtil.hpp)
 add_library(quickstep_types_Type_proto ${types_Type_proto_srcs})
 add_library(quickstep_types_TypedValue TypedValue.cpp TypedValue.hpp)
 add_library(quickstep_types_TypedValue_proto ${types_TypedValue_proto_srcs})
@@ -59,8 +66,20 @@ add_library(quickstep_types_VarCharType VarCharType.cpp VarCharType.hpp)
 add_library(quickstep_types_YearMonthIntervalType YearMonthIntervalType.cpp YearMonthIntervalType.hpp)
 
 # Link dependencies:
+target_link_libraries(quickstep_types_AsciiStringSuperType
+                      quickstep_types_Type
+                      quickstep_types_TypeID
+                      quickstep_types_TypeSynthesizer)
+target_link_libraries(quickstep_types_BoolType
+                      glog
+                      quickstep_types_NumericSuperType
+                      quickstep_types_TypeID
+                      quickstep_types_TypedValue
+                      quickstep_utility_Macros
+                      quickstep_utility_StringUtil)
 target_link_libraries(quickstep_types_CharType
                       glog
+                      quickstep_types_AsciiStringSuperType
                       quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_Type
                       quickstep_types_TypeID
@@ -77,9 +96,9 @@ target_link_libraries(quickstep_types_DateOperatorOverloads
 target_link_libraries(quickstep_types_DateType
                       glog
                       quickstep_types_DatetimeLit
-                      quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_Type
                       quickstep_types_TypeID
+                      quickstep_types_TypeSynthesizer
                       quickstep_types_TypedValue
                       quickstep_utility_CheckSnprintf
                       quickstep_utility_Macros)
@@ -87,9 +106,9 @@ target_link_libraries(quickstep_types_DatetimeIntervalType
                       glog
                       quickstep_types_IntervalLit
                       quickstep_types_IntervalParser
-                      quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_Type
                       quickstep_types_TypeID
+                      quickstep_types_TypeSynthesizer
                       quickstep_types_TypedValue
                       quickstep_utility_CheckSnprintf
                       quickstep_utility_Macros)
@@ -98,35 +117,27 @@ target_link_libraries(quickstep_types_DatetimeLit
 target_link_libraries(quickstep_types_DatetimeType
                       glog
                       quickstep_types_DatetimeLit
-                      quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_Type
                       quickstep_types_TypeID
+                      quickstep_types_TypeSynthesizer
                       quickstep_types_TypedValue
                       quickstep_types_port_gmtime_r
                       quickstep_types_port_timegm
                       quickstep_utility_CheckSnprintf
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_types_DoubleType
-                      quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_NumericSuperType
-                      quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
-                      quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_types_FloatType
-                      quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_NumericSuperType
-                      quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
-                      quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_types_IntType
                       glog
-                      quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_NumericSuperType
-                      quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_utility_Macros)
@@ -138,27 +149,30 @@ target_link_libraries(quickstep_types_IntervalParser
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_types_LongType
                       glog
-                      quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_NumericSuperType
-                      quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
-                      quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros)
-target_link_libraries(quickstep_types_NullCoercibilityCheckMacro
-                      quickstep_types_Type
-                      quickstep_types_TypeID)
 target_link_libraries(quickstep_types_NullType
                       glog
                       quickstep_types_Type
                       quickstep_types_TypeID
+                      quickstep_types_TypeSynthesizer
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_types_NumericSuperType
                       quickstep_types_NullCoercibilityCheckMacro
+                      quickstep_types_NumericTypeSafeCoercibility
                       quickstep_types_Type
                       quickstep_types_TypeID
+                      quickstep_types_TypeRegistrar
+                      quickstep_types_TypeSynthesizer
                       quickstep_types_TypedValue
-                      quickstep_utility_Macros)
+                      quickstep_utility_Macros
+                      quickstep_utility_meta_TMP)
+target_link_libraries(quickstep_types_NumericTypeSafeCoercibility
+                      quickstep_utility_meta_TMP)
+target_link_libraries(quickstep_types_NumericTypeUnifier
+                      quickstep_types_NumericTypeSafeCoercibility)
 target_link_libraries(quickstep_types_Type
                       glog
                       quickstep_types_Type_proto
@@ -167,6 +181,33 @@ target_link_libraries(quickstep_types_Type
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_types_TypeFactory
                       glog
+                      quickstep_types_Type
+                      quickstep_types_TypeID
+                      quickstep_types_TypeUtil
+                      quickstep_types_Type_proto
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_types_TypeID
+                      quickstep_types_Type_proto
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_types_TypeIDSelectors
+                      quickstep_types_TypeID
+                      quickstep_utility_meta_Common)
+target_link_libraries(quickstep_types_TypeRegistrar
+                      quickstep_types_DatetimeLit
+                      quickstep_types_IntervalLit
+                      quickstep_types_Type
+                      quickstep_types_TypeID
+                      quickstep_types_TypeIDSelectors
+                      quickstep_utility_meta_Common)
+target_link_libraries(quickstep_types_TypeSynthesizer
+                      quickstep_types_Type
+                      quickstep_types_TypeID
+                      quickstep_types_TypeRegistrar
+                      quickstep_types_Type_proto
+                      quickstep_utility_Macros
+                      quickstep_utility_PtrMap)
+target_link_libraries(quickstep_types_TypeUtil
+                      quickstep_types_BoolType
                       quickstep_types_CharType
                       quickstep_types_DateType
                       quickstep_types_DatetimeIntervalType
@@ -178,7 +219,7 @@ target_link_libraries(quickstep_types_TypeFactory
                       quickstep_types_NullType
                       quickstep_types_Type
                       quickstep_types_TypeID
-                      quickstep_types_Type_proto
+                      quickstep_types_TypeRegistrar
                       quickstep_types_VarCharType
                       quickstep_types_YearMonthIntervalType
                       quickstep_utility_Macros)
@@ -200,6 +241,7 @@ target_link_libraries(quickstep_types_Type_proto
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_types_VarCharType
                       glog
+                      quickstep_types_AsciiStringSuperType
                       quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_Type
                       quickstep_types_TypeID
@@ -211,9 +253,9 @@ target_link_libraries(quickstep_types_VarCharType
 target_link_libraries(quickstep_types_YearMonthIntervalType
                       quickstep_types_IntervalLit
                       quickstep_types_IntervalParser
-                      quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_Type
                       quickstep_types_TypeID
+                      quickstep_types_TypeSynthesizer
                       quickstep_types_TypedValue
                       quickstep_utility_CheckSnprintf
                       quickstep_utility_Macros)
@@ -221,6 +263,8 @@ target_link_libraries(quickstep_types_YearMonthIntervalType
 # Module all-in-one library:
 add_library(quickstep_types ../empty_src.cpp TypesModule.hpp)
 target_link_libraries(quickstep_types
+                      quickstep_types_AsciiStringSuperType
+                      quickstep_types_BoolType
                       quickstep_types_CharType
                       quickstep_types_DateOperatorOverloads
                       quickstep_types_DateType
@@ -236,12 +280,17 @@ target_link_libraries(quickstep_types
                       quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_NullType
                       quickstep_types_NumericSuperType
+                      quickstep_types_NumericTypeSafeCoercibility
                       quickstep_types_NumericTypeUnifier
                       quickstep_types_Type
+                      quickstep_types_TypeUtil
                       quickstep_types_Type_proto
                       quickstep_types_TypeErrors
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
+                      quickstep_types_TypeIDSelectors
+                      quickstep_types_TypeRegistrar
+                      quickstep_types_TypeSynthesizer
                       quickstep_types_TypedValue
                       quickstep_types_TypedValue_proto
                       quickstep_types_VarCharType

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/CharType.cpp
----------------------------------------------------------------------
diff --git a/types/CharType.cpp b/types/CharType.cpp
index 591c038..2ed469b 100644
--- a/types/CharType.cpp
+++ b/types/CharType.cpp
@@ -42,38 +42,6 @@ using std::string;
 
 namespace quickstep {
 
-template <bool nullable_internal>
-const CharType& CharType::InstanceInternal(const std::size_t length) {
-  static PtrMap<size_t, CharType> instance_map;
-  PtrMap<size_t, CharType>::iterator imit = instance_map.find(length);
-  if (imit == instance_map.end()) {
-    imit = instance_map.insert(length, new CharType(length, nullable_internal)).first;
-  }
-  return *(imit->second);
-}
-
-const CharType& CharType::InstanceNonNullable(const std::size_t length) {
-  return InstanceInternal<false>(length);
-}
-
-const CharType& CharType::InstanceNullable(const std::size_t length) {
-  return InstanceInternal<true>(length);
-}
-
-const CharType& CharType::InstanceFromProto(const serialization::Type &proto) {
-  return Instance(proto.GetExtension(serialization::CharType::length), proto.nullable());
-}
-
-serialization::Type CharType::getProto() const {
-  serialization::Type proto;
-  proto.set_type_id(serialization::Type::CHAR);
-
-  proto.set_nullable(nullable_);
-
-  proto.SetExtension(serialization::CharType::length, length_);
-  return proto;
-}
-
 bool CharType::isSafelyCoercibleFrom(const Type &original_type) const {
   QUICKSTEP_NULL_COERCIBILITY_CHECK();
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/CharType.hpp
----------------------------------------------------------------------
diff --git a/types/CharType.hpp b/types/CharType.hpp
index c7321f4..c90a8da 100644
--- a/types/CharType.hpp
+++ b/types/CharType.hpp
@@ -24,8 +24,8 @@
 #include <cstdio>
 #include <string>
 
+#include "types/AsciiStringSuperType.hpp"
 #include "types/Type.hpp"
-#include "types/Type.pb.h"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
@@ -43,75 +43,8 @@ namespace quickstep {
  *       represented WITHOUT a null-terminator character. Any strings shorter
  *       than the maximum length will have a null-terminator.
  **/
-class CharType : public AsciiStringSuperType {
+class CharType : public AsciiStringSuperType<kChar> {
  public:
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this Type
-   *        for the specified length.
-   *
-   * @param length The length parameter of the CharType.
-   * @return A reference to the non-nullable singleton instance of this Type
-   *         for the specified length.
-   **/
-  static const CharType& InstanceNonNullable(const std::size_t length);
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type for
-   *        the specified length.
-   *
-   * @param length The length parameter of the CharType.
-   * @return A reference to the nullable singleton instance of this Type for
-   *         the specified length.
-   **/
-  static const CharType& InstanceNullable(const std::size_t length);
-
-  /**
-   * @brief Get a reference to the singleton instance of this Type for the
-   *        specified length and nullability.
-   *
-   * @param length The length parameter of the CharType.
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the singleton instance of this Type for the
-   *         specified length.
-   **/
-  static const CharType& Instance(const std::size_t length, const bool nullable) {
-    if (nullable) {
-      return InstanceNullable(length);
-    } else {
-      return InstanceNonNullable(length);
-    }
-  }
-
-  /**
-   * @brief Get a reference to the singleton instance of this Type described
-   *        by the given Protocol Buffer serialization.
-   *
-   * @param proto The serialized Protocol Buffer representation of the desired
-   *        CharType.
-   * @return A reference to the singleton instance of this Type for the given
-   *         Protocol Buffer.
-   **/
-  static const CharType& InstanceFromProto(const serialization::Type &proto);
-
-  /**
-   * @brief Generate a serialized Protocol Buffer representation of this Type.
-   *
-   * @return The serialized Protocol Buffer representation of this Type.
-   **/
-  serialization::Type getProto() const override;
-
-  const Type& getNullableVersion() const override {
-    return InstanceNullable(length_);
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable(length_);
-  }
-
-  std::size_t estimateAverageByteLength() const override {
-    return length_;
-  }
-
   bool isSafelyCoercibleFrom(const Type &original_type) const override;
 
   std::string getName() const override;
@@ -134,11 +67,9 @@ class CharType : public AsciiStringSuperType {
 
  private:
   CharType(const std::size_t length, const bool nullable)
-      : AsciiStringSuperType(kChar, nullable, length, length, length) {
-  }
+      : AsciiStringSuperType<kChar>(nullable, length, length, length) {}
 
-  template <bool nullable_internal>
-  static const CharType& InstanceInternal(const std::size_t length);
+  template <typename, bool> friend class TypeInstance;
 
   DISALLOW_COPY_AND_ASSIGN(CharType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/DateType.cpp
----------------------------------------------------------------------
diff --git a/types/DateType.cpp b/types/DateType.cpp
index 5bb982c..de1e554 100644
--- a/types/DateType.cpp
+++ b/types/DateType.cpp
@@ -30,7 +30,6 @@
 #include <string>
 
 #include "types/DatetimeLit.hpp"
-#include "types/NullCoercibilityCheckMacro.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
@@ -46,16 +45,6 @@ using std::snprintf;
 
 namespace quickstep {
 
-bool DateType::isCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kDate);
-}
-
-bool DateType::isSafelyCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kDate);
-}
-
 std::string DateType::printValueToString(const TypedValue &value) const {
   DCHECK(!value.isNull());
 
@@ -86,14 +75,6 @@ std::string DateType::printValueToString(const TypedValue &value) const {
   return std::string(datebuf);
 }
 
-void DateType::printValueToFile(const TypedValue &value,
-                                FILE *file,
-                                const int padding) const {
-  // We simply re-use the logic from printValueToString(), as trying to do
-  // padding on-the fly with so many different fields is too much of a hassle.
-  std::fprintf(file, "%*s", padding, printValueToString(value).c_str());
-}
-
 bool DateType::parseValueFromString(const std::string &value_string,
                                     TypedValue *value) const {
   std::int32_t year;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/DateType.hpp
----------------------------------------------------------------------
diff --git a/types/DateType.hpp b/types/DateType.hpp
index 07225d5..088c125 100644
--- a/types/DateType.hpp
+++ b/types/DateType.hpp
@@ -27,6 +27,7 @@
 #include "types/DatetimeLit.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
+#include "types/TypeSynthesizer.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -40,73 +41,14 @@ class TypedValue;
 /**
  * @brief A type representing the date.
  **/
-class DateType : public Type {
+class DateType : public TypeSynthesizer<kDate> {
  public:
-  typedef DateLit cpptype;
-
-  static const TypeID kStaticTypeID = kDate;
-
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const DateType& InstanceNonNullable() {
-    static DateType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const DateType& InstanceNullable() {
-    static DateType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const DateType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
-  std::size_t estimateAverageByteLength() const override {
-    return sizeof(DateLit);
-  }
-
-  bool isCoercibleFrom(const Type &original_type) const override;
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override;
-
   int getPrintWidth() const override {
     return DateLit::kIsoChars;
   }
 
   std::string printValueToString(const TypedValue &value) const override;
 
-  void printValueToFile(const TypedValue &value,
-                        FILE *file,
-                        const int padding = 0) const override;
-
   /**
    * @note value_string is expected to be in (possibly extended) ISO-8601
    *       format. Extended ISO-8601 date format is "YYYY-MM-DD". YYYY is an
@@ -124,8 +66,9 @@ class DateType : public Type {
 
  private:
   explicit DateType(const bool nullable)
-      : Type(Type::kOther, kDate, nullable, sizeof(DateLit), sizeof(DateLit)) {
-  }
+      : TypeSynthesizer<kDate>(nullable) {}
+
+  template <typename, bool> friend class TypeInstance;
 
   DISALLOW_COPY_AND_ASSIGN(DateType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/DatetimeIntervalType.cpp
----------------------------------------------------------------------
diff --git a/types/DatetimeIntervalType.cpp b/types/DatetimeIntervalType.cpp
index 1eae03a..2c77f89 100644
--- a/types/DatetimeIntervalType.cpp
+++ b/types/DatetimeIntervalType.cpp
@@ -31,7 +31,6 @@
 
 #include "types/IntervalLit.hpp"
 #include "types/IntervalParser.hpp"
-#include "types/NullCoercibilityCheckMacro.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
@@ -47,16 +46,6 @@ using std::snprintf;
 
 namespace quickstep {
 
-bool DatetimeIntervalType::isCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kDatetimeInterval);
-}
-
-bool DatetimeIntervalType::isSafelyCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kDatetimeInterval);
-}
-
 std::string DatetimeIntervalType::printValueToString(const TypedValue &value) const {
   DCHECK(!value.isNull());
 
@@ -121,14 +110,6 @@ std::string DatetimeIntervalType::printValueToString(const TypedValue &value) co
   return std::string(interval_buf);
 }
 
-void DatetimeIntervalType::printValueToFile(const TypedValue &value,
-                                            FILE *file,
-                                            const int padding) const {
-  // We simply re-use the logic from printValueToString(), as trying to do
-  // padding on-the fly with so many different fields is too much of a hassle.
-  std::fprintf(file, "%*s", padding, printValueToString(value).c_str());
-}
-
 bool DatetimeIntervalType::parseValueFromString(const std::string &value_string,
                                                 TypedValue *value) const {
   // Try simple-format parse first.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/DatetimeIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeIntervalType.hpp b/types/DatetimeIntervalType.hpp
index 005cb31..d22f965 100644
--- a/types/DatetimeIntervalType.hpp
+++ b/types/DatetimeIntervalType.hpp
@@ -28,6 +28,7 @@
 #include "types/IntervalLit.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
+#include "types/TypeSynthesizer.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
@@ -40,73 +41,14 @@ namespace quickstep {
 /**
  * @brief A type representing the datetime interval.
  **/
-class DatetimeIntervalType : public Type {
+class DatetimeIntervalType : public TypeSynthesizer<kDatetimeInterval> {
  public:
-  typedef DatetimeIntervalLit cpptype;
-
-  static const TypeID kStaticTypeID = kDatetimeInterval;
-
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const DatetimeIntervalType& InstanceNonNullable() {
-    static DatetimeIntervalType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const DatetimeIntervalType& InstanceNullable() {
-    static DatetimeIntervalType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const DatetimeIntervalType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
-  std::size_t estimateAverageByteLength() const override {
-    return sizeof(DatetimeIntervalLit);
-  }
-
-  bool isCoercibleFrom(const Type &original_type) const override;
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override;
-
   int getPrintWidth() const override {
     return DatetimeIntervalLit::kPrintingChars;
   }
 
   std::string printValueToString(const TypedValue &value) const override;
 
-  void printValueToFile(const TypedValue &value,
-                        FILE *file,
-                        const int padding = 0) const override;
-
   TypedValue makeZeroValue() const override {
     return TypedValue(DatetimeIntervalLit{0});
   }
@@ -116,8 +58,9 @@ class DatetimeIntervalType : public Type {
 
  private:
   explicit DatetimeIntervalType(const bool nullable)
-      : Type(Type::kOther, kDatetimeInterval, nullable, sizeof(DatetimeIntervalLit), sizeof(DatetimeIntervalLit)) {
-  }
+      : TypeSynthesizer<kDatetimeInterval>(nullable) {}
+
+  template <typename, bool> friend class TypeInstance;
 
   DISALLOW_COPY_AND_ASSIGN(DatetimeIntervalType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/DatetimeLit.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeLit.hpp b/types/DatetimeLit.hpp
index 58c852f..db887eb 100644
--- a/types/DatetimeLit.hpp
+++ b/types/DatetimeLit.hpp
@@ -99,6 +99,10 @@ struct DateLit {
   inline std::int32_t monthField() const {
     return static_cast<std::int32_t>(month);
   }
+
+  inline std::int32_t dayField() const {
+    return static_cast<std::int32_t>(day);
+  }
 };
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/DatetimeType.cpp
----------------------------------------------------------------------
diff --git a/types/DatetimeType.cpp b/types/DatetimeType.cpp
index f54f318..723da61 100644
--- a/types/DatetimeType.cpp
+++ b/types/DatetimeType.cpp
@@ -33,7 +33,6 @@
 #include <string>
 
 #include "types/DatetimeLit.hpp"
-#include "types/NullCoercibilityCheckMacro.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
@@ -51,16 +50,6 @@ using std::snprintf;
 
 namespace quickstep {
 
-bool DatetimeType::isCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kDatetime);
-}
-
-bool DatetimeType::isSafelyCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kDatetime);
-}
-
 std::string DatetimeType::printValueToString(const TypedValue &value) const {
   DCHECK(!value.isNull());
 
@@ -114,14 +103,6 @@ std::string DatetimeType::printValueToString(const TypedValue &value) const {
   return std::string(datebuf);
 }
 
-void DatetimeType::printValueToFile(const TypedValue &value,
-                                    FILE *file,
-                                    const int padding) const {
-  // We simply re-use the logic from printValueToString(), as trying to do
-  // padding on-the fly with so many different fields is too much of a hassle.
-  std::fprintf(file, "%*s", padding, printValueToString(value).c_str());
-}
-
 bool DatetimeType::parseValueFromString(const std::string &value_string,
                                         TypedValue *value) const {
   int year;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/DatetimeType.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeType.hpp b/types/DatetimeType.hpp
index aad536a..6ee15c6 100644
--- a/types/DatetimeType.hpp
+++ b/types/DatetimeType.hpp
@@ -27,6 +27,7 @@
 #include "types/DatetimeLit.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
+#include "types/TypeSynthesizer.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -40,73 +41,15 @@ class TypedValue;
 /**
  * @brief A type representing the datetime.
  **/
-class DatetimeType : public Type {
+class DatetimeType
+    : public TypeSynthesizer<kDatetime> {
  public:
-  typedef DatetimeLit cpptype;
-
-  static const TypeID kStaticTypeID = kDatetime;
-
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const DatetimeType& InstanceNonNullable() {
-    static DatetimeType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const DatetimeType& InstanceNullable() {
-    static DatetimeType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const DatetimeType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
-  std::size_t estimateAverageByteLength() const override {
-    return sizeof(DatetimeLit);
-  }
-
-  bool isCoercibleFrom(const Type &original_type) const override;
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override;
-
   int getPrintWidth() const override {
     return DatetimeLit::kIsoChars;
   }
 
   std::string printValueToString(const TypedValue &value) const override;
 
-  void printValueToFile(const TypedValue &value,
-                        FILE *file,
-                        const int padding = 0) const override;
-
   /**
    * @note value_string is expected to be in (possibly extended) ISO-8601
    *       format. Extended ISO-8601 format is one of the following:
@@ -132,8 +75,9 @@ class DatetimeType : public Type {
 
  private:
   explicit DatetimeType(const bool nullable)
-      : Type(Type::kOther, kDatetime, nullable, sizeof(DatetimeLit), sizeof(DatetimeLit)) {
-  }
+      : TypeSynthesizer<kDatetime>(nullable) {}
+
+  template <typename, bool> friend class TypeInstance;
 
   DISALLOW_COPY_AND_ASSIGN(DatetimeType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/DoubleType.cpp
----------------------------------------------------------------------
diff --git a/types/DoubleType.cpp b/types/DoubleType.cpp
index 6a7914c..f5c2650 100644
--- a/types/DoubleType.cpp
+++ b/types/DoubleType.cpp
@@ -25,11 +25,8 @@
 #include <limits>
 #include <string>
 
-#include "types/NullCoercibilityCheckMacro.hpp"
-#include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
-#include "utility/EqualsAnyConstant.hpp"
 
 #include "glog/logging.h"
 
@@ -41,39 +38,6 @@ using std::snprintf;
 
 namespace quickstep {
 
-const TypeID DoubleType::kStaticTypeID = kDouble;
-
-bool DoubleType::isSafelyCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return QUICKSTEP_EQUALS_ANY_CONSTANT(original_type.getTypeID(),
-                                       kInt, kLong, kFloat, kDouble);
-}
-
-TypedValue DoubleType::coerceValue(const TypedValue &original_value,
-                                   const Type &original_type) const {
-  DCHECK(isCoercibleFrom(original_type))
-      << "Can't coerce value of Type " << original_type.getName()
-      << " to Type " << getName();
-
-  if (original_value.isNull()) {
-    return makeNullValue();
-  }
-
-  switch (original_type.getTypeID()) {
-    case kInt:
-      return TypedValue(static_cast<double>(original_value.getLiteral<int>()));
-    case kLong:
-      return TypedValue(static_cast<double>(original_value.getLiteral<std::int64_t>()));
-    case kFloat:
-      return TypedValue(static_cast<double>(original_value.getLiteral<float>()));
-    case kDouble:
-      return original_value;
-    default:
-      LOG(FATAL) << "Attempted to coerce Type " << original_type.getName()
-                 << " (not recognized as a numeric Type) to " << getName();
-  }
-}
-
 std::string DoubleType::printValueToString(const TypedValue &value) const {
   DCHECK(!value.isNull());
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/DoubleType.hpp
----------------------------------------------------------------------
diff --git a/types/DoubleType.hpp b/types/DoubleType.hpp
index b4175b0..05bec64 100644
--- a/types/DoubleType.hpp
+++ b/types/DoubleType.hpp
@@ -26,12 +26,12 @@
 
 #include "types/NumericSuperType.hpp"
 #include "types/TypeID.hpp"
-#include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
 class Type;
+class TypedValue;
 
 /** \addtogroup Types
  *  @{
@@ -40,55 +40,8 @@ class Type;
 /**
  * @brief A type representing a double-precision floating-point number.
  **/
-class DoubleType : public NumericSuperType<double> {
+class DoubleType : public NumericSuperType<kDouble> {
  public:
-  static const TypeID kStaticTypeID;
-
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const DoubleType& InstanceNonNullable() {
-    static DoubleType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const DoubleType& InstanceNullable() {
-    static DoubleType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const DoubleType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override;
-
   int getPrintWidth() const override {
     return kPrintWidth;
   }
@@ -102,9 +55,6 @@ class DoubleType : public NumericSuperType<double> {
   bool parseValueFromString(const std::string &value_string,
                             TypedValue *value) const override;
 
-  TypedValue coerceValue(const TypedValue &original_value,
-                         const Type &original_type) const override;
-
  private:
   static_assert((std::numeric_limits<double>::max_exponent10 < 1000)
                     && (std::numeric_limits<double>::min_exponent10 > -1000),
@@ -122,8 +72,9 @@ class DoubleType : public NumericSuperType<double> {
               // exponent never takes more than 3 base-10 digits to represent.
 
   explicit DoubleType(const bool nullable)
-      : NumericSuperType<double>(kDouble, nullable) {
-  }
+      : NumericSuperType<kDouble>(nullable) {}
+
+  template <typename, bool> friend class TypeInstance;
 
   DISALLOW_COPY_AND_ASSIGN(DoubleType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/FloatType.cpp
----------------------------------------------------------------------
diff --git a/types/FloatType.cpp b/types/FloatType.cpp
index e904f29..be22770 100644
--- a/types/FloatType.cpp
+++ b/types/FloatType.cpp
@@ -25,11 +25,8 @@
 #include <limits>
 #include <string>
 
-#include "types/NullCoercibilityCheckMacro.hpp"
-#include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
-#include "utility/EqualsAnyConstant.hpp"
 
 #include "glog/logging.h"
 
@@ -41,39 +38,6 @@ using std::snprintf;
 
 namespace quickstep {
 
-const TypeID FloatType::kStaticTypeID = kFloat;
-
-bool FloatType::isSafelyCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return QUICKSTEP_EQUALS_ANY_CONSTANT(original_type.getTypeID(),
-                                       kInt, kFloat);
-}
-
-TypedValue FloatType::coerceValue(const TypedValue &original_value,
-                                  const Type &original_type) const {
-  DCHECK(isCoercibleFrom(original_type))
-      << "Can't coerce value of Type " << original_type.getName()
-      << " to Type " << getName();
-
-  if (original_value.isNull()) {
-    return makeNullValue();
-  }
-
-  switch (original_type.getTypeID()) {
-    case kInt:
-      return TypedValue(static_cast<float>(original_value.getLiteral<int>()));
-    case kLong:
-      return TypedValue(static_cast<float>(original_value.getLiteral<std::int64_t>()));
-    case kFloat:
-      return original_value;
-    case kDouble:
-      return TypedValue(static_cast<float>(original_value.getLiteral<double>()));
-    default:
-      LOG(FATAL) << "Attempted to coerce Type " << original_type.getName()
-                 << " (not recognized as a numeric Type) to " << getName();
-  }
-}
-
 std::string FloatType::printValueToString(const TypedValue &value) const {
   DCHECK(!value.isNull());
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/FloatType.hpp
----------------------------------------------------------------------
diff --git a/types/FloatType.hpp b/types/FloatType.hpp
index 2a156e1..6c8466d 100644
--- a/types/FloatType.hpp
+++ b/types/FloatType.hpp
@@ -26,12 +26,12 @@
 
 #include "types/NumericSuperType.hpp"
 #include "types/TypeID.hpp"
-#include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
 class Type;
+class TypedValue;
 
 /** \addtogroup Types
  *  @{
@@ -40,55 +40,8 @@ class Type;
 /**
  * @brief A type representing a single-precision floating-point number.
  **/
-class FloatType : public NumericSuperType<float> {
+class FloatType : public NumericSuperType<kFloat> {
  public:
-  static const TypeID kStaticTypeID;
-
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type
-   **/
-  static const FloatType& InstanceNonNullable() {
-    static FloatType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type
-   *
-   * @return A reference to the nullable singleton instance of this Type
-   **/
-  static const FloatType& InstanceNullable() {
-    static FloatType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type
-   *
-   * @param nullable Whether to get the nullable version of this Type
-   * @return A reference to the desired singleton instance of this Type
-   **/
-  static const FloatType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override;
-
   int getPrintWidth() const override {
     return kPrintWidth;
   }
@@ -102,9 +55,6 @@ class FloatType : public NumericSuperType<float> {
   bool parseValueFromString(const std::string &value_string,
                             TypedValue *value) const override;
 
-  TypedValue coerceValue(const TypedValue &original_value,
-                         const Type &original_type) const override;
-
  private:
   static_assert((std::numeric_limits<float>::max_exponent10 < 100)
                     && (std::numeric_limits<float>::min_exponent10 > -100),
@@ -122,8 +72,9 @@ class FloatType : public NumericSuperType<float> {
               // never takes more than 2 base-10 digits to represent.
 
   explicit FloatType(const bool nullable)
-      : NumericSuperType<float>(kFloat, nullable) {
-  }
+      : NumericSuperType<kFloat>(nullable) {}
+
+  template <typename, bool> friend class TypeInstance;
 
   DISALLOW_COPY_AND_ASSIGN(FloatType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/IntType.cpp
----------------------------------------------------------------------
diff --git a/types/IntType.cpp b/types/IntType.cpp
index 9781675..1005aa9 100644
--- a/types/IntType.cpp
+++ b/types/IntType.cpp
@@ -19,12 +19,9 @@
 
 #include "types/IntType.hpp"
 
-#include <cstdint>
 #include <cstdio>
 #include <string>
 
-#include "types/NullCoercibilityCheckMacro.hpp"
-#include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
 
@@ -32,38 +29,6 @@
 
 namespace quickstep {
 
-const TypeID IntType::kStaticTypeID = kInt;
-
-bool IntType::isSafelyCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return original_type.getTypeID() == kInt;
-}
-
-TypedValue IntType::coerceValue(const TypedValue &original_value,
-                                const Type &original_type) const {
-  DCHECK(isCoercibleFrom(original_type))
-      << "Can't coerce value of Type " << original_type.getName()
-      << " to Type " << getName();
-
-  if (original_value.isNull()) {
-    return makeNullValue();
-  }
-
-  switch (original_type.getTypeID()) {
-    case kInt:
-      return original_value;
-    case kLong:
-      return TypedValue(static_cast<int>(original_value.getLiteral<std::int64_t>()));
-    case kFloat:
-      return TypedValue(static_cast<int>(original_value.getLiteral<float>()));
-    case kDouble:
-      return TypedValue(static_cast<int>(original_value.getLiteral<double>()));
-    default:
-      LOG(FATAL) << "Attempted to coerce Type " << original_type.getName()
-                 << " (not recognized as a numeric Type) to " << getName();
-  }
-}
-
 std::string IntType::printValueToString(const TypedValue &value) const {
   DCHECK(!value.isNull());
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/IntType.hpp
----------------------------------------------------------------------
diff --git a/types/IntType.hpp b/types/IntType.hpp
index 08d6b3d..84cc7ce 100644
--- a/types/IntType.hpp
+++ b/types/IntType.hpp
@@ -26,12 +26,12 @@
 
 #include "types/NumericSuperType.hpp"
 #include "types/TypeID.hpp"
-#include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
 class Type;
+class TypedValue;
 
 /** \addtogroup Types
  *  @{
@@ -40,55 +40,8 @@ class Type;
 /**
  * @brief A type representing a 32-bit integer.
  **/
-class IntType : public NumericSuperType<int> {
+class IntType : public NumericSuperType<kInt> {
  public:
-  static const TypeID kStaticTypeID;
-
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const IntType& InstanceNonNullable() {
-    static IntType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const IntType& InstanceNullable() {
-    static IntType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const IntType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override;
-
   int getPrintWidth() const override {
     // Fully represented digits, single leading digit, and possible '-'
     // character.
@@ -104,13 +57,11 @@ class IntType : public NumericSuperType<int> {
   bool parseValueFromString(const std::string &value_string,
                             TypedValue *value) const override;
 
-  TypedValue coerceValue(const TypedValue &original_value,
-                         const Type &original_type) const override;
-
  private:
   explicit IntType(const bool nullable)
-      : NumericSuperType<int>(kInt, nullable) {
-  }
+      : NumericSuperType<kInt>(nullable) {}
+
+  template <typename, bool> friend class TypeInstance;
 
   DISALLOW_COPY_AND_ASSIGN(IntType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/LongType.cpp
----------------------------------------------------------------------
diff --git a/types/LongType.cpp b/types/LongType.cpp
index fbf8d30..334821d 100644
--- a/types/LongType.cpp
+++ b/types/LongType.cpp
@@ -28,49 +28,13 @@
 #include <cstdio>
 #include <string>
 
-#include "types/NullCoercibilityCheckMacro.hpp"
-#include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
-#include "utility/EqualsAnyConstant.hpp"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
-const TypeID LongType::kStaticTypeID = kLong;
-
-bool LongType::isSafelyCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return QUICKSTEP_EQUALS_ANY_CONSTANT(original_type.getTypeID(),
-                                       kInt, kLong);
-}
-
-TypedValue LongType::coerceValue(const TypedValue &original_value,
-                                 const Type &original_type) const {
-  DCHECK(isCoercibleFrom(original_type))
-      << "Can't coerce value of Type " << original_type.getName()
-      << " to Type " << getName();
-
-  if (original_value.isNull()) {
-    return makeNullValue();
-  }
-
-  switch (original_type.getTypeID()) {
-    case kInt:
-      return TypedValue(static_cast<std::int64_t>(original_value.getLiteral<int>()));
-    case kLong:
-      return original_value;
-    case kFloat:
-      return TypedValue(static_cast<std::int64_t>(original_value.getLiteral<float>()));
-    case kDouble:
-      return TypedValue(static_cast<std::int64_t>(original_value.getLiteral<double>()));
-    default:
-      LOG(FATAL) << "Attempted to coerce Type " << original_type.getName()
-                 << " (not recognized as a numeric Type) to " << getName();
-  }
-}
-
 std::string LongType::printValueToString(const TypedValue &value) const {
   DCHECK(!value.isNull());
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/LongType.hpp
----------------------------------------------------------------------
diff --git a/types/LongType.hpp b/types/LongType.hpp
index a90dd32..e52a166 100644
--- a/types/LongType.hpp
+++ b/types/LongType.hpp
@@ -27,12 +27,12 @@
 
 #include "types/NumericSuperType.hpp"
 #include "types/TypeID.hpp"
-#include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
 class Type;
+class TypedValue;
 
 /** \addtogroup Types
  *  @{
@@ -41,55 +41,8 @@ class Type;
 /**
  * @brief A type representing a 64-bit integer.
  **/
-class LongType : public NumericSuperType<std::int64_t> {
+class LongType : public NumericSuperType<kLong> {
  public:
-  static const TypeID kStaticTypeID;
-
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const LongType& InstanceNonNullable() {
-    static LongType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const LongType& InstanceNullable() {
-    static LongType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const LongType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override;
-
   // Fully represented digits, single leading digit, and possible '-'
   // character.
   int getPrintWidth() const override {
@@ -105,13 +58,11 @@ class LongType : public NumericSuperType<std::int64_t> {
   bool parseValueFromString(const std::string &value_string,
                             TypedValue *value) const override;
 
-  TypedValue coerceValue(const TypedValue &original_value,
-                         const Type &original_type) const override;
-
  private:
   explicit LongType(const bool nullable)
-      : NumericSuperType<std::int64_t>(kLong, nullable) {
-  }
+      : NumericSuperType<kLong>(nullable) {}
+
+  template <typename, bool> friend class TypeInstance;
 
   DISALLOW_COPY_AND_ASSIGN(LongType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/NullCoercibilityCheckMacro.hpp
----------------------------------------------------------------------
diff --git a/types/NullCoercibilityCheckMacro.hpp b/types/NullCoercibilityCheckMacro.hpp
index 70e1beb..9cdd152 100644
--- a/types/NullCoercibilityCheckMacro.hpp
+++ b/types/NullCoercibilityCheckMacro.hpp
@@ -20,9 +20,6 @@
 #ifndef QUICKSTEP_TYPES_NULL_COERCIBILITY_CHECK_MACRO_HPP_
 #define QUICKSTEP_TYPES_NULL_COERCIBILITY_CHECK_MACRO_HPP_
 
-#include "types/Type.hpp"
-#include "types/TypeID.hpp"
-
 /** \addtogroup Types
  *  @{
  */
@@ -34,7 +31,7 @@
  **/
 #define QUICKSTEP_NULL_COERCIBILITY_CHECK()               \
   do {                                                    \
-    if (original_type.isNullable() && !nullable_) {       \
+    if (original_type.isNullable() && !this->nullable_) {       \
       return false;                                       \
     } else if (original_type.getTypeID() == kNullType) {  \
       return true;                                        \

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/NullType.hpp
----------------------------------------------------------------------
diff --git a/types/NullType.hpp b/types/NullType.hpp
index c27a584..c416a05 100644
--- a/types/NullType.hpp
+++ b/types/NullType.hpp
@@ -26,6 +26,7 @@
 
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
+#include "types/TypeSynthesizer.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
@@ -48,40 +49,20 @@ class TypedValue;
  *       a particular operation may accept. It is also assumed that applying
  *       any operation to an argument of NullType always yields NULL values.
  **/
-class NullType : public Type {
+class NullType : public TypeSynthesizer<kNullType> {
  public:
-  static const TypeID kStaticTypeID = kNullType;
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   * @note Unlike other Types, there is no corresponding method to get a
-   *       non-nullable version of NullType. NullType is ALWAYS nullable.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const NullType& InstanceNullable() {
-    static NullType instance;
-    return instance;
+  static const NullType& InstanceNonNullable() {
+    LOG(FATAL) << "Called NullType::InstanceNonNullable(), "
+               << "which is not allowed.";
   }
 
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    LOG(FATAL) << "Called NullType::getNonNullableVersion(), which is not allowed.";
-  }
-
-  std::size_t estimateAverageByteLength() const override {
-    return 0;
-  }
-
-  bool isCoercibleFrom(const Type &original_type) const override {
-    return original_type.getTypeID() == kNullType;
-  }
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override {
-    return original_type.getTypeID() == kNullType;
+  static const NullType& Instance(const bool nullable) {
+    if (nullable) {
+      return InstanceNullable();
+    } else {
+      LOG(FATAL) << "Called NullType::Instance(nullable = true), "
+                 << "which is not allowed.";
+    }
   }
 
   int getPrintWidth() const override {
@@ -106,9 +87,13 @@ class NullType : public Type {
  private:
   // NOTE(chasseur): NullType requires 0 bytes of inherent storage. It does,
   // however, require a bit in NULL bitmaps.
-  NullType() : Type(Type::kOther, kNullType, true, 0, 0) {
+  NullType(const bool nullable)
+      : TypeSynthesizer<kNullType>(nullable, 0, 0) {
+    DCHECK(nullable);
   }
 
+  template <typename, bool> friend class TypeInstance;
+
   DISALLOW_COPY_AND_ASSIGN(NullType);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/NumericSuperType.hpp
----------------------------------------------------------------------
diff --git a/types/NumericSuperType.hpp b/types/NumericSuperType.hpp
index 0cc1546..abe8b87 100644
--- a/types/NumericSuperType.hpp
+++ b/types/NumericSuperType.hpp
@@ -21,12 +21,17 @@
 #define QUICKSTEP_TYPES_NUMERIC_SUPER_TYPE_HPP_
 
 #include <cstddef>
+#include <unordered_set>
 
 #include "types/NullCoercibilityCheckMacro.hpp"
+#include "types/NumericTypeSafeCoercibility.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
+#include "types/TypeRegistrar.hpp"
+#include "types/TypeSynthesizer.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
+#include "utility/meta/TMP.hpp"
 
 namespace quickstep {
 
@@ -38,30 +43,72 @@ namespace quickstep {
  * @brief Templatized superclass for Numeric types. Contains code common to all
  *        Numeric types.
  **/
-template <typename CppType>
-class NumericSuperType : public Type {
+template <TypeID type_id>
+class NumericSuperType : public TypeSynthesizer<type_id> {
  public:
-  typedef CppType cpptype;
-
-  std::size_t estimateAverageByteLength() const override {
-    return sizeof(CppType);
+  bool isSafelyCoercibleFrom(const Type &original_type) const override {
+    QUICKSTEP_NULL_COERCIBILITY_CHECK();
+    const auto it = safe_coerce_cache_.find(original_type.getTypeID());
+    return it != safe_coerce_cache_.end();
   }
 
   bool isCoercibleFrom(const Type &original_type) const override {
     QUICKSTEP_NULL_COERCIBILITY_CHECK();
-    return (original_type.getSuperTypeID() == kNumeric);
+    return (original_type.getSuperTypeID() == Type::kNumeric);
   }
 
   TypedValue makeZeroValue() const override {
-    return TypedValue(static_cast<CppType>(0));
+    return TypedValue(static_cast<typename TypeIDTrait<type_id>::cpptype>(0));
   }
 
- protected:
-  NumericSuperType(const TypeID type_id, const bool nullable)
-      : Type(Type::kNumeric, type_id, nullable, sizeof(CppType), sizeof(CppType)) {
+  TypedValue coerceValue(const TypedValue &original_value,
+                         const Type &original_type) const override {
+    if (original_type.getSuperTypeID() != Type::kNumeric) {
+      LOG(FATAL) << "Attempted to coerce Type " << original_type.getName()
+                 << " (not recognized as a numeric Type) to " << Type::getName();
+    }
+
+    if (original_value.isNull()) {
+      return Type::makeNullValue();
+    }
+
+    return InvokeOnTypeID<TypeIDSelectorNumeric>(
+        original_type.getTypeID(),
+        [&](auto orig_tid) -> TypedValue {  // NOLINT(build/c++11)
+      using OrigCppType = typename TypeIDTrait<decltype(orig_tid)::value>::cpptype;
+      using TargetCppType = typename TypeIDTrait<type_id>::cpptype;
+
+      return TypedValue(
+          static_cast<TargetCppType>(original_value.getLiteral<OrigCppType>()));
+    });
   }
 
+ protected:
+  explicit NumericSuperType(const bool nullable)
+      : TypeSynthesizer<type_id>(nullable),
+        safe_coerce_cache_(CreateSafeCoercibilityCache()) {}
+
  private:
+  using TargetType = typename TypeIDTrait<type_id>::TypeClass;
+
+  template <typename SourceTypeID>
+  struct SafeCoercibilityFilter {
+    static constexpr bool value =
+        NumericTypeSafeCoercibility<
+            typename TypeIDTrait<SourceTypeID::value>::TypeClass,
+            TargetType>::value;
+  };
+
+  inline static auto CreateSafeCoercibilityCache() {
+    using SourceTypeIDs = TypeIDSequenceAll::template bind_to<meta::TypeList>;
+    using ResultTypeIDs = SourceTypeIDs::template filter<SafeCoercibilityFilter>;
+
+    return ResultTypeIDs::template as_sequence<TypeID>
+        ::template Instantiate<std::unordered_set<TypeID>>();
+  };
+
+  const std::unordered_set<TypeID> safe_coerce_cache_;
+
   DISALLOW_COPY_AND_ASSIGN(NumericSuperType);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/NumericTypeSafeCoercibility.hpp
----------------------------------------------------------------------
diff --git a/types/NumericTypeSafeCoercibility.hpp b/types/NumericTypeSafeCoercibility.hpp
new file mode 100644
index 0000000..914927c
--- /dev/null
+++ b/types/NumericTypeSafeCoercibility.hpp
@@ -0,0 +1,61 @@
+/**
+ * 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_TYPES_NUMERIC_TYPE_SAFE_COERCIBILITY_HPP_
+#define QUICKSTEP_TYPES_NUMERIC_TYPE_SAFE_COERCIBILITY_HPP_
+
+#include "utility/meta/TMP.hpp"
+
+namespace quickstep {
+
+class BoolType;
+class DoubleType;
+class FloatType;
+class IntType;
+class LongType;
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename LeftType, typename RightType>
+using IsSafelyCoercible = meta::TypeList<LeftType, RightType>;
+
+using NumericTypeSafeCoersionPartialOrder = meta::TypeList<
+    IsSafelyCoercible<BoolType, IntType>,
+    IsSafelyCoercible<IntType, FloatType>,
+    IsSafelyCoercible<IntType, LongType>,
+    IsSafelyCoercible<FloatType, DoubleType>,
+    IsSafelyCoercible<LongType, DoubleType>>;
+
+using NumericTypeSafeCoersionClosure =
+    meta::TransitiveClosure<NumericTypeSafeCoersionPartialOrder>;
+
+template <typename LeftType, typename RightType>
+struct NumericTypeSafeCoercibility {
+  static constexpr bool value =
+      NumericTypeSafeCoersionClosure::contains<
+          IsSafelyCoercible<LeftType, RightType>>::value;
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_NUMERIC_TYPE_SAFE_COERCIBILITY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/NumericTypeUnifier.hpp
----------------------------------------------------------------------
diff --git a/types/NumericTypeUnifier.hpp b/types/NumericTypeUnifier.hpp
index 168dfb1..0c3f13f 100644
--- a/types/NumericTypeUnifier.hpp
+++ b/types/NumericTypeUnifier.hpp
@@ -20,16 +20,9 @@
 #ifndef QUICKSTEP_TYPES_NUMERIC_TYPE_UNIFIER_HPP_
 #define QUICKSTEP_TYPES_NUMERIC_TYPE_UNIFIER_HPP_
 
-namespace quickstep {
-
-class DoubleType;
-class FloatType;
-class IntType;
-class LongType;
+#include "types/NumericTypeSafeCoercibility.hpp"
 
-/** \addtogroup Types
- *  @{
- */
+namespace quickstep {
 
 /**
  * @brief A traits template that resolves what the "unifying" Type of two
@@ -51,92 +44,47 @@ class LongType;
  * @tparam RightType The second Quickstep numeric Type class to unify.
  **/
 template <typename LeftType, typename RightType>
-struct NumericTypeUnifier {
-};
+struct NumericTypeUnifier;
 
-/** @} */
 
-// Explicit template specializations for all combinations of builtin numeric
-// types.
-template<>
-struct NumericTypeUnifier<IntType, IntType> {
-  typedef IntType type;
-};
+namespace internal {
 
-template<>
-struct NumericTypeUnifier<IntType, LongType> {
-  typedef LongType type;
-};
-
-template<>
-struct NumericTypeUnifier<IntType, FloatType> {
-  typedef FloatType type;
-};
+template <typename LeftType, typename RightType, typename EnableT = void>
+struct NumericTypeUnifierHelper;
 
-template<>
-struct NumericTypeUnifier<IntType, DoubleType> {
-  typedef DoubleType type;
-};
-
-template<>
-struct NumericTypeUnifier<LongType, IntType> {
-  typedef LongType type;
-};
-
-template<>
-struct NumericTypeUnifier<LongType, LongType> {
-  typedef LongType type;
-};
-
-template<>
-struct NumericTypeUnifier<LongType, FloatType> {
-  typedef DoubleType type;
-};
-
-template<>
-struct NumericTypeUnifier<LongType, DoubleType> {
-  typedef DoubleType type;
-};
-
-template<>
-struct NumericTypeUnifier<FloatType, IntType> {
-  typedef FloatType type;
-};
-
-template<>
-struct NumericTypeUnifier<FloatType, LongType> {
-  typedef DoubleType type;
+template <typename LeftType, typename RightType>
+struct NumericTypeUnifierHelper<
+    LeftType, RightType,
+    std::enable_if_t<NumericTypeSafeCoercibility<LeftType, RightType>::value>> {
+  typedef RightType type;
 };
 
-template<>
-struct NumericTypeUnifier<FloatType, FloatType> {
-  typedef FloatType type;
+template <typename LeftType, typename RightType>
+struct NumericTypeUnifierHelper<
+    LeftType, RightType,
+    std::enable_if_t<!std::is_same<LeftType, RightType>::value &&
+                     NumericTypeSafeCoercibility<RightType, LeftType>::value>> {
+  typedef LeftType type;
 };
 
+// Explicit template specializations
 template<>
-struct NumericTypeUnifier<FloatType, DoubleType> {
+struct NumericTypeUnifierHelper<LongType, FloatType> {
   typedef DoubleType type;
 };
 
 template<>
-struct NumericTypeUnifier<DoubleType, IntType> {
+struct NumericTypeUnifierHelper<FloatType, LongType> {
   typedef DoubleType type;
 };
 
-template<>
-struct NumericTypeUnifier<DoubleType, LongType> {
-  typedef DoubleType type;
-};
+}  // namespace internal
 
-template<>
-struct NumericTypeUnifier<DoubleType, FloatType> {
-  typedef DoubleType type;
-};
+template <typename LeftType, typename RightType>
+struct NumericTypeUnifier
+    : internal::NumericTypeUnifierHelper<LeftType, RightType> {};
 
-template<>
-struct NumericTypeUnifier<DoubleType, DoubleType> {
-  typedef DoubleType type;
-};
+/** @} */
 
 }  // namespace quickstep
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/Type.cpp
----------------------------------------------------------------------
diff --git a/types/Type.cpp b/types/Type.cpp
index f3d3f1b..b69cb65 100644
--- a/types/Type.cpp
+++ b/types/Type.cpp
@@ -28,43 +28,35 @@
 
 namespace quickstep {
 
-serialization::Type Type::getProto() const {
-  serialization::Type proto;
-  switch (type_id_) {
-    case kInt:
-      proto.set_type_id(serialization::Type::INT);
-      break;
-    case kLong:
-      proto.set_type_id(serialization::Type::LONG);
-      break;
-    case kFloat:
-      proto.set_type_id(serialization::Type::FLOAT);
-      break;
-    case kDouble:
-      proto.set_type_id(serialization::Type::DOUBLE);
-      break;
-    case kDate:
-      proto.set_type_id(serialization::Type::DATE);
-      break;
-    case kDatetime:
-      proto.set_type_id(serialization::Type::DATETIME);
-      break;
-    case kDatetimeInterval:
-      proto.set_type_id(serialization::Type::DATETIME_INTERVAL);
-      break;
-    case kYearMonthInterval:
-      proto.set_type_id(serialization::Type::YEAR_MONTH_INTERVAL);
-      break;
-    case kNullType:
-      proto.set_type_id(serialization::Type::NULL_TYPE);
-      break;
-    default:
-      FATAL_ERROR("Unrecognized TypeID in Type::getProto");
+bool Type::isCoercibleFrom(const Type &original_type) const {
+  return isSafelyCoercibleFrom(original_type);
+}
+
+bool Type::isSafelyCoercibleFrom(const Type &original_type) const {
+  if (original_type.isNullable() && !this->nullable_) {
+    return false;
   }
+  if (original_type.getTypeID() == kNullType) {
+    return true;
+  }
+  return (original_type.getTypeID() == type_id_);
+}
 
-  proto.set_nullable(nullable_);
+std::size_t Type::estimateAverageByteLength() const {
+  if (minimum_byte_length_ == maximum_byte_length_) {
+    return maximum_byte_length_;
+  }
+  if (maximum_byte_length_ > 160) {
+    return 80;
+  } else {
+    return (maximum_byte_length_ >> 1) + 1;
+  }
+}
 
-  return proto;
+void Type::printValueToFile(const TypedValue &value,
+                            FILE *file,
+                            const int padding) const {
+  std::fprintf(file, "%*s", padding, printValueToString(value).c_str());
 }
 
 TypedValue Type::coerceValue(const TypedValue &original_value,
@@ -85,12 +77,4 @@ TypedValue Type::coerceValue(const TypedValue &original_value,
   return original_value;
 }
 
-bool AsciiStringSuperType::isCoercibleFrom(const Type &original_type) const {
-  if (original_type.isNullable() && !nullable_) {
-    return false;
-  }
-  return (original_type.getSuperTypeID() == kAsciiString)
-         || (original_type.getTypeID() == kNullType);
-}
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/Type.hpp
----------------------------------------------------------------------
diff --git a/types/Type.hpp b/types/Type.hpp
index 0e8c4e5..bf6c167 100644
--- a/types/Type.hpp
+++ b/types/Type.hpp
@@ -112,7 +112,7 @@ class Type {
    *
    * @return The serialized Protocol Buffer representation of this Type.
    **/
-  virtual serialization::Type getProto() const;
+  virtual serialization::Type getProto() const = 0;
 
   /**
    * @brief Determine what supertype this type belongs to.
@@ -154,16 +154,7 @@ class Type {
     TypeSignature sig;
     sig.id = type_id_;
     sig.nullable = nullable_;
-    switch (type_id_) {
-      case kChar:
-        sig.length = maximum_byte_length_;
-        break;
-      case kVarChar:
-        sig.length = maximum_byte_length_ - 1;
-        break;
-      default:
-        sig.length = 0;
-    }
+    sig.length = parameter_;
     return sig;
   }
 
@@ -227,7 +218,7 @@ class Type {
    * @return An estimate of the average number of bytes used by data items of
    *         this type.
    **/
-  virtual std::size_t estimateAverageByteLength() const = 0;
+  virtual std::size_t estimateAverageByteLength() const;
 
   /**
    * @brief Determine whether this Type is exactly the same as another.
@@ -255,7 +246,7 @@ class Type {
    * @param original_type The original Type for coercion to this Type.
    * @return true if coercion is supported, false otherwise.
    **/
-  virtual bool isCoercibleFrom(const Type &original_type) const = 0;
+  virtual bool isCoercibleFrom(const Type &original_type) const;
 
   /**
    * @brief Determine whether data items of another type can be coerced (used
@@ -264,7 +255,7 @@ class Type {
    * @note It is NOT possible to coerce a nullable type to a non-nullable type,
    *       even if coercion would otherwise be possible.
    * @note Integer types are safely coercible to other integer or
-   *       floating-poin types of equal or greater length.
+   *       floating-point types of equal or greater length.
    * @note Floating-point types are safely coercible to other floating-point
    *       types of equal or greater precision.
    * @note ASCII string types are safely coercible to other ASCII string types
@@ -277,7 +268,7 @@ class Type {
    * @param original_type The original Type for coercion to this Type.
    * @return true if coercion is supported, false otherwise.
    **/
-  virtual bool isSafelyCoercibleFrom(const Type &original_type) const = 0;
+  virtual bool isSafelyCoercibleFrom(const Type &original_type) const;
 
   /**
    * @brief Determine whether data items of this type are always guaranteed to
@@ -348,7 +339,7 @@ class Type {
    **/
   virtual void printValueToFile(const TypedValue &value,
                                 FILE *file,
-                                const int padding = 0) const = 0;
+                                const int padding = 0) const;
 
   /**
    * @brief Make a TypedValue of this Type.
@@ -453,10 +444,12 @@ class Type {
        const TypeID type_id,
        const bool nullable,
        const std::size_t minimum_byte_length,
-       const std::size_t maximum_byte_length)
+       const std::size_t maximum_byte_length,
+       const std::size_t parameter = 0)
       : super_type_id_(super_type_id),
         type_id_(type_id),
         nullable_(nullable),
+        parameter_(parameter),
         minimum_byte_length_(minimum_byte_length),
         maximum_byte_length_(maximum_byte_length) {
   }
@@ -464,6 +457,7 @@ class Type {
   const SuperTypeID super_type_id_;
   const TypeID type_id_;
   const bool nullable_;
+  const std::size_t parameter_;
   const std::size_t minimum_byte_length_;
   const std::size_t maximum_byte_length_;
 
@@ -471,38 +465,6 @@ class Type {
   DISALLOW_COPY_AND_ASSIGN(Type);
 };
 
-/**
- * @brief A superclass for ASCII string types.
- **/
-class AsciiStringSuperType : public Type {
- public:
-  bool isCoercibleFrom(const Type &original_type) const override;
-
-  /**
-   * @brief Get the character-length of this string type.
-   *
-   * @return The maximum length of a string of this type.
-   **/
-  inline std::size_t getStringLength() const {
-    return length_;
-  }
-
- protected:
-  AsciiStringSuperType(const TypeID type_id,
-                       const bool nullable,
-                       const std::size_t minimum_byte_length,
-                       const std::size_t maximum_byte_length,
-                       const std::size_t string_length)
-      : Type(Type::kAsciiString, type_id, nullable, minimum_byte_length, maximum_byte_length),
-        length_(string_length) {
-  }
-
-  const std::size_t length_;
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(AsciiStringSuperType);
-};
-
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/Type.proto
----------------------------------------------------------------------
diff --git a/types/Type.proto b/types/Type.proto
index d03b5a4..ed8df36 100644
--- a/types/Type.proto
+++ b/types/Type.proto
@@ -19,39 +19,13 @@ syntax = "proto2";
 
 package quickstep.serialization;
 
-message Type {
-  enum TypeID {
-    INT = 0;
-    LONG = 1;
-    FLOAT = 2;
-    DOUBLE = 3;
-    CHAR = 4;
-    VAR_CHAR = 5;
-    DATETIME = 6;
-    DATETIME_INTERVAL = 7;
-    YEAR_MONTH_INTERVAL = 8;
-    NULL_TYPE = 9;
-    DATE = 10;
-  }
+message TypeID {
+  required int32 id = 1;
+}
 
+message Type {
   required TypeID type_id = 1;
   required bool nullable = 2;
-
-  // The convention for extension numbering is that extensions for a particular
-  // TypeID should begin from (type_id + 1) * 32.
-  extensions 32 to max;
+  optional uint64 length = 3;
 }
 
-message CharType {
-  extend Type {
-    // Required when type_id == CHAR.
-    optional uint64 length = 160;
-  }
-}
-
-message VarCharType {
-  extend Type {
-    // Required when type_id == VAR_CHAR.
-    optional uint64 length = 192;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/TypeFactory.cpp
----------------------------------------------------------------------
diff --git a/types/TypeFactory.cpp b/types/TypeFactory.cpp
index 7403dc9..66efc92 100644
--- a/types/TypeFactory.cpp
+++ b/types/TypeFactory.cpp
@@ -22,66 +22,45 @@
 #include <cstddef>
 #include <string>
 
-#include "types/CharType.hpp"
-#include "types/DateType.hpp"
-#include "types/DatetimeIntervalType.hpp"
-#include "types/DatetimeType.hpp"
-#include "types/DoubleType.hpp"
-#include "types/FloatType.hpp"
-#include "types/IntType.hpp"
-#include "types/LongType.hpp"
-#include "types/NullType.hpp"
 #include "types/Type.hpp"
 #include "types/Type.pb.h"
 #include "types/TypeID.hpp"
-#include "types/VarCharType.hpp"
-#include "types/YearMonthIntervalType.hpp"
+#include "types/TypeUtil.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
+bool TypeFactory::TypeRequiresLengthParameter(const TypeID id) {
+  return TypeUtil::IsParameterized(id);
+}
+
 const Type& TypeFactory::GetType(const TypeID id,
                                  const bool nullable) {
-  switch (id) {
-    case kInt:
-      return IntType::Instance(nullable);
-    case kLong:
-      return LongType::Instance(nullable);
-    case kFloat:
-      return FloatType::Instance(nullable);
-    case kDouble:
-      return DoubleType::Instance(nullable);
-    case kDate:
-      return DateType::Instance(nullable);
-    case kDatetime:
-      return DatetimeType::Instance(nullable);
-    case kDatetimeInterval:
-      return DatetimeIntervalType::Instance(nullable);
-    case kYearMonthInterval:
-      return YearMonthIntervalType::Instance(nullable);
-    case kNullType:
-      DCHECK(nullable);
-      return NullType::InstanceNullable();
-    default:
-      FATAL_ERROR("Called TypeFactory::GetType() for a type which requires "
-                  " a length parameter without specifying one.");
-  }
+  DCHECK(!TypeRequiresLengthParameter(id))
+      << "Called TypeFactory::GetType() for a type which requires "
+      << " a length parameter without specifying one.";
+
+  return *InvokeOnTypeID<TypeIDSelectorNonParameterized>(
+      id,
+      [&](auto id) -> const Type* {  // NOLINT(build/c++11)
+    return &TypeIDTrait<decltype(id)::value>::TypeClass::Instance(nullable);
+  });
 }
 
 const Type& TypeFactory::GetType(const TypeID id,
                                  const std::size_t length,
                                  const bool nullable) {
-  switch (id) {
-    case kChar:
-      return CharType::Instance(length, nullable);
-    case kVarChar:
-      return VarCharType::Instance(length, nullable);
-    default:
-      FATAL_ERROR("Provided a length parameter to TypeFactory::GetType() for "
-                  "a type which does not take one.");
-  }
+  DCHECK(TypeRequiresLengthParameter(id))
+      << "Provided a length parameter to TypeFactory::GetType() for "
+      << "a type which does not take one.";
+
+  return *InvokeOnTypeID<TypeIDSelectorParameterized>(
+      id,
+      [&](auto id) -> const Type* {  // NOLINT(build/c++11)
+    return &TypeIDTrait<decltype(id)::value>::TypeClass::Instance(nullable, length);
+  });
 }
 
 bool TypeFactory::ProtoIsValid(const serialization::Type &proto) {
@@ -90,26 +69,18 @@ bool TypeFactory::ProtoIsValid(const serialization::Type &proto) {
     return false;
   }
 
-  // Check that the type_id is valid, and extensions if any.
-  switch (proto.type_id()) {
-    case serialization::Type::INT:
-    case serialization::Type::LONG:
-    case serialization::Type::FLOAT:
-    case serialization::Type::DOUBLE:
-    case serialization::Type::DATE:
-    case serialization::Type::DATETIME:
-    case serialization::Type::DATETIME_INTERVAL:
-    case serialization::Type::YEAR_MONTH_INTERVAL:
-      return true;
-    case serialization::Type::CHAR:
-      return proto.HasExtension(serialization::CharType::length);
-    case serialization::Type::VAR_CHAR:
-      return proto.HasExtension(serialization::VarCharType::length);
-    case serialization::Type::NULL_TYPE:
-      return proto.nullable();
-    default:
-      return false;
+  // Check that the type_id is valid, and has length if parameterized.
+  const TypeID type_id = TypeIDFactory::ReconstructFromProto(proto.type_id());
+
+  if (type_id == kNullType) {
+    return proto.nullable();
+  }
+
+  if (TypeRequiresLengthParameter(type_id)) {
+    return proto.has_length();
   }
+
+  return true;
 }
 
 const Type& TypeFactory::ReconstructFromProto(const serialization::Type &proto) {
@@ -117,32 +88,12 @@ const Type& TypeFactory::ReconstructFromProto(const serialization::Type &proto)
       << "Attempted to create Type from an invalid proto description:\n"
       << proto.DebugString();
 
-  switch (proto.type_id()) {
-    case serialization::Type::INT:
-      return IntType::Instance(proto.nullable());
-    case serialization::Type::LONG:
-      return LongType::Instance(proto.nullable());
-    case serialization::Type::FLOAT:
-      return FloatType::Instance(proto.nullable());
-    case serialization::Type::DOUBLE:
-      return DoubleType::Instance(proto.nullable());
-    case serialization::Type::DATE:
-      return DateType::Instance(proto.nullable());
-    case serialization::Type::DATETIME:
-      return DatetimeType::Instance(proto.nullable());
-    case serialization::Type::DATETIME_INTERVAL:
-      return DatetimeIntervalType::Instance(proto.nullable());
-    case serialization::Type::YEAR_MONTH_INTERVAL:
-      return YearMonthIntervalType::Instance(proto.nullable());
-    case serialization::Type::CHAR:
-      return CharType::InstanceFromProto(proto);
-    case serialization::Type::VAR_CHAR:
-      return VarCharType::InstanceFromProto(proto);
-    case serialization::Type::NULL_TYPE:
-      DCHECK(proto.nullable());
-      return NullType::InstanceNullable();
-    default:
-      FATAL_ERROR("Unrecognized TypeID in TypeFactory::ReconstructFromProto");
+  const TypeID type_id = TypeIDFactory::ReconstructFromProto(proto.type_id());
+
+  if (TypeRequiresLengthParameter(type_id)) {
+    return GetType(type_id, proto.length(), proto.nullable());
+  } else {
+    return GetType(type_id, proto.nullable());
   }
 }
 
@@ -157,9 +108,11 @@ const Type* TypeFactory::GetMostSpecificType(const Type &first, const Type &seco
 }
 
 const Type* TypeFactory::GetUnifyingType(const Type &first, const Type &second) {
+  // TODO: cache
   const Type *unifier = nullptr;
   if (first.isNullable() || second.isNullable()) {
-    unifier = GetMostSpecificType(first.getNullableVersion(), second.getNullableVersion());
+    unifier = GetMostSpecificType(first.getNullableVersion(),
+                                  second.getNullableVersion());
     if (unifier == nullptr) {
       if (((first.getTypeID() == kLong) && (second.getTypeID() == kFloat))
             || ((first.getTypeID() == kFloat) && (second.getTypeID() == kLong))) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/TypeFactory.hpp
----------------------------------------------------------------------
diff --git a/types/TypeFactory.hpp b/types/TypeFactory.hpp
index 742348e..89ff497 100644
--- a/types/TypeFactory.hpp
+++ b/types/TypeFactory.hpp
@@ -48,25 +48,7 @@ class TypeFactory {
    * @param id The id of the desired Type.
    * @return Whether a length must be specified for Types of the given id.
    **/
-  static bool TypeRequiresLengthParameter(const TypeID id) {
-    switch (id) {
-      case kInt:
-      case kLong:
-      case kFloat:
-      case kDouble:
-      case kDate:
-      case kDatetime:
-      case kDatetimeInterval:
-      case kYearMonthInterval:
-      case kNullType:
-        return false;
-      case kChar:
-      case kVarChar:
-        return true;
-      default:
-        FATAL_ERROR("Unrecognized TypeID in TypeFactory::TypeRequiresLengthParameter");
-    }
-  }
+  static bool TypeRequiresLengthParameter(const TypeID id);
 
   /**
    * @brief Factory method to get a Type by its TypeID.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/TypeID.cpp
----------------------------------------------------------------------
diff --git a/types/TypeID.cpp b/types/TypeID.cpp
index ff2f8e6..afc9c2b 100644
--- a/types/TypeID.cpp
+++ b/types/TypeID.cpp
@@ -22,6 +22,7 @@
 namespace quickstep {
 
 const char *kTypeNames[] = {
+  "Bool",
   "Int",
   "Long",
   "Float",

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/TypeID.hpp
----------------------------------------------------------------------
diff --git a/types/TypeID.hpp b/types/TypeID.hpp
index c54d8a5..d27368c 100644
--- a/types/TypeID.hpp
+++ b/types/TypeID.hpp
@@ -21,6 +21,10 @@
 #define QUICKSTEP_TYPES_TYPE_ID_HPP_
 
 #include <cstddef>
+#include <type_traits>
+
+#include "types/Type.pb.h"
+#include "utility/Macros.hpp"
 
 namespace quickstep {
 
@@ -29,8 +33,9 @@ namespace quickstep {
  *
  * @note TypedValue assumes that this doesn't exceed 64 TypeIDs.
  **/
-enum TypeID {
-  kInt = 0,
+enum TypeID : int {
+  kBool = 0,
+  kInt,
   kLong,
   kFloat,
   kDouble,
@@ -44,6 +49,13 @@ enum TypeID {
   kNumTypeIDs  // Not a real TypeID, exists for counting purposes.
 };
 
+enum TypeStorageLayout {
+  kNativeEmbedded,
+  kNativeInline,
+  kNonNativeInline,
+  kOutOfLine
+};
+
 /**
  * @brief Provides basic information about a Type in the Quickstep type system.
  *
@@ -65,6 +77,26 @@ struct TypeSignature {
  **/
 extern const char *kTypeNames[kNumTypeIDs];
 
+class TypeIDFactory {
+ public:
+  inline static serialization::TypeID GetProto(const TypeID type_id) {
+    serialization::TypeID proto;
+    proto.set_id(static_cast<std::underlying_type_t<TypeID>>(type_id));
+    return proto;
+  }
+
+  inline static TypeID ReconstructFromProto(const serialization::TypeID &proto) {
+    return static_cast<TypeID>(proto.id());
+  }
+
+  inline static bool ProtoIsValid(const serialization::TypeID &proto) {
+    return proto.id() < static_cast<std::underlying_type_t<TypeID>>(kNumTypeIDs);
+  }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(TypeIDFactory);
+};
+
 }  // namespace quickstep
 
 #endif  // QUICKSTEP_TYPES_TYPE_ID_HPP_


[03/32] incubator-quickstep git commit: Add protobuf support for union all operator.

Posted by ji...@apache.org.
Add protobuf support for union all operator.


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

Branch: refs/heads/new-op
Commit: 758f07a1613e68823fab9323461fdcae56714365
Parents: 3c83c93
Author: Tianrun <Ti...@node-0.tianrun-qv23700.quickstep-pg0.wisc.cloudlab.us>
Authored: Mon Apr 17 11:03:13 2017 -0600
Committer: Tianrun <Ti...@node-2.tianrun-qv24978.quickstep-pg0.wisc.cloudlab.us>
Committed: Thu Apr 20 14:41:28 2017 -0600

----------------------------------------------------------------------
 relational_operators/CMakeLists.txt       |  1 +
 relational_operators/UnionAllOperator.cpp | 58 ++++++++++++++++++++++----
 relational_operators/UnionAllOperator.hpp | 11 ++++-
 relational_operators/WorkOrder.proto      | 10 +++++
 relational_operators/WorkOrderFactory.cpp | 40 ++++++++++++++++++
 5 files changed, 110 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/758f07a1/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 4ea809b..39538ea 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -586,6 +586,7 @@ target_link_libraries(quickstep_relationaloperators_WorkOrderFactory
                       quickstep_relationaloperators_SortRunGenerationOperator
                       quickstep_relationaloperators_TableGeneratorOperator
                       quickstep_relationaloperators_TextScanOperator
+                      quickstep_relationaloperators_UnionAllOperator
                       quickstep_relationaloperators_UpdateOperator
                       quickstep_relationaloperators_WindowAggregationOperator
                       quickstep_relationaloperators_WorkOrder_proto

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/758f07a1/relational_operators/UnionAllOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UnionAllOperator.cpp b/relational_operators/UnionAllOperator.cpp
index 141b3cf..1d82aef 100644
--- a/relational_operators/UnionAllOperator.cpp
+++ b/relational_operators/UnionAllOperator.cpp
@@ -65,7 +65,7 @@ void UnionAllOperator::addWorkOrdersSingleRelation(
       container->addNormalWorkOrder(
           new UnionAllWorkOrder(
               query_id_,
-              input_relations_[relation_index],
+              *input_relations_[relation_index],
               input_block_id,
               select_attribute_ids_[relation_index],
               output_destination,
@@ -75,11 +75,11 @@ void UnionAllOperator::addWorkOrdersSingleRelation(
   } else {
     std::size_t num_generated = num_workorders_generated_[relation_index];
     const std::vector<block_id> &all_blocks = input_relations_block_ids_[relation_index];
-    while (num_generated < all_blocks .size()) {
+    while (num_generated < all_blocks.size()) {
       container->addNormalWorkOrder(
           new UnionAllWorkOrder(
               query_id_,
-              input_relations_[relation_index],
+              *input_relations_[relation_index],
               all_blocks[num_generated],
               select_attribute_ids_[relation_index],
               output_destination,
@@ -124,18 +124,60 @@ bool UnionAllOperator::getAllWorkOrders(
                                    relation_index);
     }
   }
-  return stored_generated_ && done_feeding_input_relation_;
+  return done_feeding_input_relation_;
 }
 
 bool UnionAllOperator::getAllWorkOrderProtos(WorkOrderProtosContainer* container) {
-  // TODO(tianrun): Add protobuf for UnionAllWorkOrder to support distributed mode.
-  LOG(FATAL) << "UnionAllOperator is not supported in distributed mode yet.";
-  return true;
+  if (!stored_generated_) {
+    for (std::size_t relation_index = 0; relation_index < input_relations_.size(); ++relation_index) {
+      if (input_relations_are_stored_[relation_index]) {
+        const std::vector<block_id> &all_blocks = input_relations_block_ids_[relation_index];
+        const relation_id relation = input_relations_[relation_index]->getID();
+        const std::vector<attribute_id> &attributes = select_attribute_ids_[relation_index];
+        for (const block_id block : all_blocks) {
+          container->addWorkOrderProto(createWorkOrderProto(block, relation, attributes), op_index_);
+        }
+      }
+    }
+    stored_generated_ = true;
+  }
+
+  for (std::size_t relation_index = 0; relation_index < input_relations_.size(); ++relation_index) {
+    if (!input_relations_are_stored_[relation_index]) {
+      const std::vector<block_id> &all_blocks = input_relations_block_ids_[relation_index];
+      std::size_t num_generated = num_workorders_generated_[relation_index];
+      const relation_id relation = input_relations_[relation_index]->getID();
+      const std::vector<attribute_id> &attributes = select_attribute_ids_[relation_index];
+      while (num_generated < all_blocks.size()) {
+        container->addWorkOrderProto(createWorkOrderProto(all_blocks[num_generated], relation, attributes), op_index_);
+        ++num_generated;
+      }
+      num_workorders_generated_[relation_index] = num_generated;
+    }
+  }
+  return done_feeding_input_relation_;
+}
+
+serialization::WorkOrder* UnionAllOperator::createWorkOrderProto(
+    const block_id block,
+    const relation_id relation,
+    const std::vector<attribute_id> &attributes) {
+  serialization::WorkOrder *proto = new serialization::WorkOrder;
+  proto->set_work_order_type(serialization::UNION_ALL);
+  proto->set_query_id(query_id_);
+
+  proto->SetExtension(serialization::UnionAllWorkOrder::relation_id, relation);
+  proto->SetExtension(serialization::UnionAllWorkOrder::insert_destination_index, output_destination_index_);
+  proto->SetExtension(serialization::UnionAllWorkOrder::block_id, block);
+  for (const attribute_id attr : attributes) {
+    proto->AddExtension(serialization::UnionAllWorkOrder::select_attribute_id, attr);
+  }
+  return proto;
 }
 
 void UnionAllWorkOrder::execute() {
   BlockReference block(
-      storage_manager_->getBlock(input_block_id_, *input_relation_));
+      storage_manager_->getBlock(input_block_id_, input_relation_));
   block->selectSimple(select_attribute_id_,
                       nullptr,
                       output_destination_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/758f07a1/relational_operators/UnionAllOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UnionAllOperator.hpp b/relational_operators/UnionAllOperator.hpp
index 3bfed82..4fc2906 100644
--- a/relational_operators/UnionAllOperator.hpp
+++ b/relational_operators/UnionAllOperator.hpp
@@ -45,6 +45,8 @@ class StorageManager;
 class WorkOrderProtosContainer;
 class WorkOrdersContainer;
 
+namespace serialization { class WorkOrder; }
+
 /** \addtogroup RelationalOperators
  *  @{
  */
@@ -149,6 +151,11 @@ class UnionAllOperator : public RelationalOperator {
                                    InsertDestination *output_destination,
                                    const std::size_t relation_index);
 
+  // Create work order proto
+  serialization::WorkOrder* createWorkOrderProto(const block_id block,
+                                                 const relation_id relation,
+                                                 const std::vector<attribute_id> &attributes);
+
   const std::vector<const CatalogRelation*> input_relations_;
   const std::vector<bool> input_relations_are_stored_;
 
@@ -194,7 +201,7 @@ class UnionAllWorkOrder : public WorkOrder {
    * @param storage_manager The StorageManager to use.
    */
   UnionAllWorkOrder(const std::size_t query_id,
-                    const CatalogRelationSchema *input_relation,
+                    const CatalogRelationSchema &input_relation,
                     const block_id input_block_id,
                     const std::vector<attribute_id> &select_attribute_id,
                     InsertDestination *output_destination,
@@ -211,7 +218,7 @@ class UnionAllWorkOrder : public WorkOrder {
   void execute() override;
 
  private:
-  const CatalogRelationSchema *input_relation_;
+  const CatalogRelationSchema &input_relation_;
   const block_id input_block_id_;
   const std::vector<attribute_id> select_attribute_id_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/758f07a1/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index d0d0753..12a65ca 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -45,6 +45,7 @@ enum WorkOrderType {
   WINDOW_AGGREGATION = 21;
   DESTROY_AGGREGATION_STATE = 22;
   BUILD_AGGREGATION_EXISTENCE_MAP = 23;
+  UNION_ALL = 24;
 }
 
 message WorkOrder {
@@ -291,3 +292,12 @@ message BuildAggregationExistenceMapWorkOrder {
     optional uint32 aggr_state_index = 371;
   }
 }
+
+message UnionAllWorkOrder {
+  extend WorkOrder {
+    optional int32 relation_id = 384;
+    optional int32 insert_destination_index = 385;
+    optional fixed64 block_id = 386;
+    repeated int32 select_attribute_id = 387;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/758f07a1/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 56f431b..d63bb62 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -49,6 +49,7 @@
 #include "relational_operators/SortRunGenerationOperator.hpp"
 #include "relational_operators/TableGeneratorOperator.hpp"
 #include "relational_operators/TextScanOperator.hpp"
+#include "relational_operators/UnionAllOperator.hpp"
 #include "relational_operators/UpdateOperator.hpp"
 #include "relational_operators/WindowAggregationOperator.hpp"
 #include "relational_operators/WorkOrder.pb.h"
@@ -492,6 +493,23 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
           hdfs);
     }
+    case serialization::UNION_ALL: {
+      LOG(INFO) << "Creating UnionAllWorkOrder for Query " << proto.query_id() << " in Shiftboss " << shiftboss_index;
+      vector<attribute_id> select_attribute_id;
+      for (int i = 0; i < proto.ExtensionSize(serialization::UnionAllWorkOrder::select_attribute_id); ++i) {
+        select_attribute_id.push_back(
+            proto.GetExtension(serialization::UnionAllWorkOrder::select_attribute_id, i));
+      }
+      return new UnionAllWorkOrder(
+          proto.query_id(),
+          catalog_database->getRelationSchemaById(
+              proto.GetExtension(serialization::UnionAllWorkOrder::relation_id)),
+          proto.GetExtension(serialization::UnionAllWorkOrder::block_id),
+          select_attribute_id,
+          query_context->getInsertDestination(
+              proto.GetExtension(serialization::UnionAllWorkOrder::insert_destination_index)),
+          storage_manager);
+    }
     case serialization::UPDATE: {
       LOG(INFO) << "Creating UpdateWorkOrder for Query " << proto.query_id() << " in Shiftboss " << shiftboss_index;
       return new UpdateWorkOrder(
@@ -892,6 +910,28 @@ bool WorkOrderFactory::ProtoIsValid(const serialization::WorkOrder &proto,
              query_context.isValidInsertDestinationId(
                  proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index));
     }
+    case serialization::UNION_ALL: {
+      if (!proto.HasExtension(serialization::UnionAllWorkOrder::relation_id) ||
+          !proto.HasExtension(serialization::UnionAllWorkOrder::block_id) ||
+          !proto.HasExtension(serialization::UnionAllWorkOrder::insert_destination_index) ||
+          !query_context.isValidInsertDestinationId(
+              proto.GetExtension(serialization::UnionAllWorkOrder::insert_destination_index))) {
+        return false;
+      }
+
+      const relation_id rel_id = proto.GetExtension(serialization::UnionAllWorkOrder::relation_id);
+      if (!catalog_database.hasRelationWithId(rel_id)) {
+        return false;
+      }
+      const CatalogRelationSchema &relation = catalog_database.getRelationSchemaById(rel_id);
+      for (int i = 0; i < proto.ExtensionSize(serialization::UnionAllWorkOrder::select_attribute_id); ++i) {
+        if (!relation.hasAttributeWithId(
+                 proto.GetExtension(serialization::UnionAllWorkOrder::select_attribute_id, i))) {
+          return false;
+        }
+      }
+      return true;
+    }
     case serialization::UPDATE: {
       return proto.HasExtension(serialization::UpdateWorkOrder::relation_id) &&
              catalog_database.hasRelationWithId(


[02/32] incubator-quickstep git commit: API to get total pending work orders for an operator

Posted by ji...@apache.org.
API to get total pending work orders for an operator

- Total includes normal and rebuild work orders.


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

Branch: refs/heads/new-op
Commit: 3c83c937acd6d2484cfa7c96c21e695fd3cdaba3
Parents: 6e3499a
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Wed Apr 19 13:31:08 2017 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Wed Apr 19 14:38:24 2017 -0500

----------------------------------------------------------------------
 query_execution/WorkOrdersContainer.hpp | 13 +++++++++++++
 1 file changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/3c83c937/query_execution/WorkOrdersContainer.hpp
----------------------------------------------------------------------
diff --git a/query_execution/WorkOrdersContainer.hpp b/query_execution/WorkOrdersContainer.hpp
index c1739bc..1fb3ca6 100644
--- a/query_execution/WorkOrdersContainer.hpp
+++ b/query_execution/WorkOrdersContainer.hpp
@@ -311,6 +311,19 @@ class WorkOrdersContainer {
     return rebuild_workorders_[operator_index].getNumWorkOrders();
   }
 
+  /**
+   * @brief Get the total number of work orders for the given operator.
+   *
+   * @param operator_index The index of the operator.
+   *
+   * @return The total number of WorkOrders (normal + rebuild).
+   **/
+  inline std::size_t getNumTotalWorkOrders(
+      const std::size_t operator_index) const {
+    return getNumNormalWorkOrders(operator_index) +
+           getNumRebuildWorkOrders(operator_index);
+  }
+
  private:
   /**
    * @brief An internal queue-based container structure to hold the WorkOrders.


[06/32] incubator-quickstep git commit: Add common-subexpression support.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/rules/CollapseSelection.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CollapseSelection.hpp b/query_optimizer/rules/CollapseSelection.hpp
new file mode 100644
index 0000000..bc5e4a3
--- /dev/null
+++ b/query_optimizer/rules/CollapseSelection.hpp
@@ -0,0 +1,62 @@
+/**
+ * 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_COLLAPSE_SELECTION_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_COLLAPSE_SELECTION_HPP_
+
+#include <string>
+
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/BottomUpRule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+/**
+ * @brief Merges nested Selections into one single Selection.
+ */
+class CollapseSelection : public BottomUpRule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  CollapseSelection() {}
+
+  std::string getName() const override {
+    return "CollapseSelection";
+  }
+
+ protected:
+  physical::PhysicalPtr applyToNode(const physical::PhysicalPtr &input) override;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(CollapseSelection);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_COLLAPSE_SELECTION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/rules/ExtractCommonSubexpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ExtractCommonSubexpression.cpp b/query_optimizer/rules/ExtractCommonSubexpression.cpp
new file mode 100644
index 0000000..e3f996c
--- /dev/null
+++ b/query_optimizer/rules/ExtractCommonSubexpression.cpp
@@ -0,0 +1,376 @@
+/**
+ * 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/ExtractCommonSubexpression.hpp"
+
+#include <cstddef>
+#include <memory>
+#include <unordered_set>
+#include <vector>
+
+#include "query_optimizer/OptimizerContext.hpp"
+#include "query_optimizer/expressions/AggregateFunction.hpp"
+#include "query_optimizer/expressions/Alias.hpp"
+#include "query_optimizer/expressions/CommonSubexpression.hpp"
+#include "query_optimizer/expressions/ExpressionType.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
+#include "query_optimizer/expressions/Scalar.hpp"
+#include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/NestedLoopsJoin.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "query_optimizer/physical/Selection.hpp"
+#include "utility/HashError.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+ExtractCommonSubexpression::ExtractCommonSubexpression(
+    OptimizerContext *optimizer_context)
+    : optimizer_context_(optimizer_context) {
+  const std::vector<E::ExpressionType> homogeneous_expr_types = {
+      E::ExpressionType::kAlias,
+      E::ExpressionType::kAttributeReference,
+      E::ExpressionType::kBinaryExpression,
+      E::ExpressionType::kCast,
+      E::ExpressionType::kCommonSubexpression,
+      E::ExpressionType::kScalarLiteral,
+      E::ExpressionType::kUnaryExpression
+  };
+
+  for (const auto &expr_type : homogeneous_expr_types) {
+    homogeneous_expression_types_.emplace(expr_type);
+  }
+}
+
+P::PhysicalPtr ExtractCommonSubexpression::apply(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  return applyInternal(input);
+}
+
+P::PhysicalPtr ExtractCommonSubexpression::applyInternal(
+    const P::PhysicalPtr &input) {
+  // First process all child nodes.
+  std::vector<P::PhysicalPtr> new_children;
+  for (const auto &child : input->children()) {
+    new_children.emplace_back(applyInternal(child));
+  }
+
+  const P::PhysicalPtr node =
+      new_children == input->children()
+          ? input
+          : input->copyWithNewChildren(new_children);
+
+  // Process expressions of the current node.
+  switch (node->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      const P::AggregatePtr aggregate =
+          std::static_pointer_cast<const P::Aggregate>(node);
+
+      std::vector<E::ExpressionPtr> expressions;
+      // Gather grouping expressions and aggregate functions' argument expressions.
+      for (const auto &expr : aggregate->grouping_expressions()) {
+        expressions.emplace_back(expr);
+      }
+      for (const auto &expr : aggregate->aggregate_expressions()) {
+        const E::AggregateFunctionPtr &func =
+            std::static_pointer_cast<const E::AggregateFunction>(expr->expression());
+        for (const auto &arg : func->getArguments()) {
+          expressions.emplace_back(arg);
+        }
+      }
+
+      // Transform the expressions so that common subexpressions are labelled.
+      const std::vector<E::ExpressionPtr> new_expressions =
+          transformExpressions(expressions);
+
+      if (new_expressions != expressions) {
+        std::vector<E::AliasPtr> new_aggregate_expressions;
+        std::vector<E::NamedExpressionPtr> new_grouping_expressions;
+
+        // Reconstruct grouping expressions.
+        const std::size_t num_grouping_expressions =
+            aggregate->grouping_expressions().size();
+        for (std::size_t i = 0; i < num_grouping_expressions; ++i) {
+          DCHECK(E::SomeNamedExpression::Matches(new_expressions[i]));
+          new_grouping_expressions.emplace_back(
+              std::static_pointer_cast<const E::NamedExpression>(new_expressions[i]));
+        }
+
+        // Reconstruct aggregate expressions.
+        auto it = new_expressions.begin() + num_grouping_expressions;
+        for (const auto &expr : aggregate->aggregate_expressions()) {
+          const E::AggregateFunctionPtr &func =
+              std::static_pointer_cast<const E::AggregateFunction>(
+                  expr->expression());
+
+          std::vector<E::ScalarPtr> new_arguments;
+          for (std::size_t i = 0; i < func->getArguments().size(); ++i, ++it) {
+            DCHECK(E::SomeScalar::Matches(*it));
+            new_arguments.emplace_back(std::static_pointer_cast<const E::Scalar>(*it));
+          }
+
+          if (new_arguments == func->getArguments()) {
+            new_aggregate_expressions.emplace_back(expr);
+          } else {
+            const E::AggregateFunctionPtr new_func =
+                E::AggregateFunction::Create(func->getAggregate(),
+                                             new_arguments,
+                                             func->is_vector_aggregate(),
+                                             func->is_distinct());
+            new_aggregate_expressions.emplace_back(
+                E::Alias::Create(expr->id(),
+                                 new_func,
+                                 expr->attribute_name(),
+                                 expr->attribute_alias(),
+                                 expr->relation_name()));
+          }
+        }
+        return P::Aggregate::Create(aggregate->input(),
+                                    new_grouping_expressions,
+                                    new_aggregate_expressions,
+                                    aggregate->filter_predicate());
+      }
+      break;
+    }
+    case P::PhysicalType::kSelection: {
+      const P::SelectionPtr selection =
+          std::static_pointer_cast<const P::Selection>(node);
+
+      // Transform Selection's project expressions.
+      const std::vector<E::NamedExpressionPtr> new_expressions =
+          DownCast<E::NamedExpression>(
+              transformExpressions(UpCast(selection->project_expressions())));
+
+      if (new_expressions != selection->project_expressions()) {
+        return P::Selection::Create(selection->input(),
+                                    new_expressions,
+                                    selection->filter_predicate());
+      }
+      break;
+    }
+    case P::PhysicalType::kHashJoin: {
+      const P::HashJoinPtr hash_join =
+          std::static_pointer_cast<const P::HashJoin>(node);
+
+      // Transform HashJoin's project expressions.
+      const std::vector<E::NamedExpressionPtr> new_expressions =
+          DownCast<E::NamedExpression>(
+              transformExpressions(UpCast(hash_join->project_expressions())));
+
+      if (new_expressions != hash_join->project_expressions()) {
+        return P::HashJoin::Create(hash_join->left(),
+                                   hash_join->right(),
+                                   hash_join->left_join_attributes(),
+                                   hash_join->right_join_attributes(),
+                                   hash_join->residual_predicate(),
+                                   new_expressions,
+                                   hash_join->join_type());
+      }
+      break;
+    }
+    case P::PhysicalType::kNestedLoopsJoin: {
+      const P::NestedLoopsJoinPtr nested_loops_join =
+          std::static_pointer_cast<const P::NestedLoopsJoin>(node);
+
+      // Transform NestedLoopsJoin's project expressions.
+      const std::vector<E::NamedExpressionPtr> new_expressions =
+          DownCast<E::NamedExpression>(
+              transformExpressions(UpCast(nested_loops_join->project_expressions())));
+
+      if (new_expressions != nested_loops_join->project_expressions()) {
+        return P::NestedLoopsJoin::Create(nested_loops_join->left(),
+                                          nested_loops_join->right(),
+                                          nested_loops_join->join_predicate(),
+                                          new_expressions);
+      }
+      break;
+    }
+    default:
+      break;
+  }
+
+  return node;
+}
+
+std::vector<E::ExpressionPtr> ExtractCommonSubexpression::transformExpressions(
+    const std::vector<E::ExpressionPtr> &expressions) {
+  // Step 1. For each subexpression, count the number of its occurrences.
+  ScalarCounter counter;
+  ScalarHashable hashable;
+  for (const auto &expr : expressions) {
+    visitAndCount(expr, &counter, &hashable);
+  }
+
+  // Note that any subexpression with count > 1 is a common subexpression.
+  // However, it is not necessary to create a CommonSubexpression node for every
+  // such subexpression. E.g. consider the case
+  // --
+  //   SELECT (x+1)*(x+2), (x+1)*(x+2)*3 FROM s;
+  // --
+  // We only need to create one *dominant* CommonSubexpression (x+1)*(x+2) and
+  // do not need to create the child (x+1) or (x+2) nodes.
+  //
+  // To address this issue. We define that a subtree S *dominates* its descendent
+  // subtree (or leaf node) T if and only if counter[S] >= counter[T].
+  //
+  // Then we create CommonSubexpression nodes for every subexpression that is
+  // not dominated by any ancestor.
+
+  ScalarMap substitution_map;
+  std::vector<E::ExpressionPtr> new_expressions;
+  for (const auto &expr : expressions) {
+    new_expressions.emplace_back(
+        visitAndTransform(expr, 1, counter, hashable, &substitution_map));
+  }
+  return new_expressions;
+}
+
+E::ExpressionPtr ExtractCommonSubexpression::transformExpression(
+    const E::ExpressionPtr &expression) {
+  return transformExpressions({expression}).front();
+}
+
+bool ExtractCommonSubexpression::visitAndCount(
+    const E::ExpressionPtr &expression,
+    ScalarCounter *counter,
+    ScalarHashable *hashable) {
+  // This bool flag is for avoiding some unnecessary hash() computation.
+  bool children_hashable = true;
+
+  const auto homogeneous_expression_types_it =
+      homogeneous_expression_types_.find(expression->getExpressionType());
+  if (homogeneous_expression_types_it != homogeneous_expression_types_.end()) {
+    for (const auto &child : expression->children()) {
+      children_hashable &= visitAndCount(child, counter, hashable);
+    }
+  }
+
+  E::ScalarPtr scalar;
+  if (children_hashable &&
+      E::SomeScalar::MatchesWithConditionalCast(expression, &scalar)) {
+    // A scalar expression may or may not support the hash() computation.
+    // In the later case, a HashNotSupported exception will be thrown and we
+    // simply ignore this expression (and all its ancestor expressions).
+    try {
+      ++(*counter)[scalar];
+    } catch (const HashNotSupported &e) {
+      return false;
+    }
+    hashable->emplace(scalar);
+    return true;
+  }
+  return false;
+}
+
+E::ExpressionPtr ExtractCommonSubexpression::visitAndTransform(
+    const E::ExpressionPtr &expression,
+    const std::size_t max_reference_count,
+    const ScalarCounter &counter,
+    const ScalarHashable &hashable,
+    ScalarMap *substitution_map) {
+  // TODO(jianqiao): Currently it is hardly beneficial to make AttributeReference
+  // a common subexpression due to the inefficiency of ScalarAttribute's
+  // size-not-known-at-compile-time std::memcpy() calls, compared to copy-elision
+  // at selection level. Even in the case of compressed column store, it requires
+  // an attribute to occur at least 4 times for the common subexpression version
+  // to outperform the direct decoding version. We may look into ScalarAttribute
+  // and modify the heuristic here later.
+  if (expression->getExpressionType() == E::ExpressionType::kScalarLiteral ||
+      expression->getExpressionType() == E::ExpressionType::kAttributeReference) {
+    return expression;
+  }
+
+  E::ScalarPtr scalar;
+  const bool is_hashable =
+      E::SomeScalar::MatchesWithConditionalCast(expression, &scalar)
+          && hashable.find(scalar) != hashable.end();
+
+  std::size_t new_max_reference_count;
+  if (is_hashable) {
+    // CommonSubexpression node already generated somewhere. Just refer to that
+    // and return.
+    const auto substitution_map_it = substitution_map->find(scalar);
+    if (substitution_map_it != substitution_map->end()) {
+      return substitution_map_it->second;
+    }
+
+    // Otherwise, update the dominance count.
+    const auto counter_it = counter.find(scalar);
+    DCHECK(counter_it != counter.end());
+    DCHECK_LE(max_reference_count, counter_it->second);
+    new_max_reference_count = counter_it->second;
+  } else {
+    new_max_reference_count = max_reference_count;
+  }
+
+  // Process children.
+  std::vector<E::ExpressionPtr> new_children;
+  const auto homogeneous_expression_types_it =
+      homogeneous_expression_types_.find(expression->getExpressionType());
+  if (homogeneous_expression_types_it == homogeneous_expression_types_.end()) {
+    // If child subexpressions cannot be hoisted through the current expression,
+    // treat child expressions as isolated sub-optimizations.
+    for (const auto &child : expression->children()) {
+      new_children.emplace_back(transformExpression(child));
+    }
+  } else {
+    for (const auto &child : expression->children()) {
+      new_children.emplace_back(
+          visitAndTransform(child,
+                            new_max_reference_count,
+                            counter,
+                            hashable,
+                            substitution_map));
+    }
+  }
+
+  E::ExpressionPtr output;
+  if (new_children == expression->children()) {
+    output = expression;
+  } else {
+    output = std::static_pointer_cast<const E::Scalar>(
+        expression->copyWithNewChildren(new_children));
+  }
+
+  // Wrap it with a new CommonSubexpression node if the current expression is a
+  // dominant subexpression.
+  if (is_hashable && new_max_reference_count > max_reference_count) {
+    DCHECK(E::SomeScalar::Matches(output));
+    const E::CommonSubexpressionPtr common_subexpression =
+        E::CommonSubexpression::Create(
+            optimizer_context_->nextExprId(),
+            std::static_pointer_cast<const E::Scalar>(output));
+    substitution_map->emplace(scalar, common_subexpression);
+    output = common_subexpression;
+  }
+
+  return output;
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/rules/ExtractCommonSubexpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ExtractCommonSubexpression.hpp b/query_optimizer/rules/ExtractCommonSubexpression.hpp
new file mode 100644
index 0000000..3cdd70e
--- /dev/null
+++ b/query_optimizer/rules/ExtractCommonSubexpression.hpp
@@ -0,0 +1,179 @@
+/**
+ * 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_EXTRACT_COMMON_SUBEXPRESSION_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_EXTRACT_COMMON_SUBEXPRESSION_HPP_
+
+#include <cstddef>
+#include <functional>
+#include <memory>
+#include <string>
+#include <type_traits>
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "query_optimizer/expressions/CommonSubexpression.hpp"
+#include "query_optimizer/expressions/Expression.hpp"
+#include "query_optimizer/expressions/ExpressionType.hpp"
+#include "query_optimizer/expressions/Scalar.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+class OptimizerContext;
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+/**
+ * @brief Rule that applies to a physical plan to identify and label common
+ *        subexpressions.
+ *
+ * @note This is essentially a logical optimization pass. However, we need some
+ *       of the physical passes (e.g. ReuseAggregateExpressions) to be finalized
+ *       before this one to maximize optimization opportunities.
+ */
+class ExtractCommonSubexpression : public Rule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param optimizer_context The optimizer context.
+   */
+  explicit ExtractCommonSubexpression(OptimizerContext *optimizer_context);
+
+  ~ExtractCommonSubexpression() override {}
+
+  std::string getName() const override {
+    return "ExtractCommonSubexpression";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  physical::PhysicalPtr applyInternal(const physical::PhysicalPtr &input);
+
+  struct ScalarHash {
+    inline std::size_t operator()(const expressions::ScalarPtr &scalar) const {
+      return scalar->hash();
+    }
+  };
+
+  struct ScalarEqual {
+    inline bool operator()(const expressions::ScalarPtr &lhs,
+                           const expressions::ScalarPtr &rhs) const {
+      return lhs->equals(rhs);
+    }
+  };
+
+  // For memorizing whether a subexpression is hashable.
+  using ScalarHashable = std::unordered_set<expressions::ScalarPtr>;
+
+  // For counting the number of occurrences of each subexpression.
+  using ScalarCounter =
+      std::unordered_map<expressions::ScalarPtr, std::size_t, ScalarHash, ScalarEqual>;
+
+  // For mapping each subexpression to its transformed version.
+  using ScalarMap =
+      std::unordered_map<expressions::ScalarPtr,
+                         expressions::CommonSubexpressionPtr,
+                         ScalarHash,
+                         ScalarEqual>;
+
+  std::vector<expressions::ExpressionPtr> transformExpressions(
+      const std::vector<expressions::ExpressionPtr> &expressions);
+
+  expressions::ExpressionPtr transformExpression(
+      const expressions::ExpressionPtr &expression);
+
+  // Traverse the expression tree and increase the count of each subexpression.
+  bool visitAndCount(
+      const expressions::ExpressionPtr &expression,
+      ScalarCounter *counter,
+      ScalarHashable *hashable);
+
+  // Traverse the expression tree and transform subexpressions (to common
+  // subexpressions) if applicable.
+  expressions::ExpressionPtr visitAndTransform(
+      const expressions::ExpressionPtr &expression,
+      const std::size_t max_reference_count,
+      const ScalarCounter &counter,
+      const ScalarHashable &hashable,
+      ScalarMap *substitution_map);
+
+  template <typename ScalarSubclassT>
+  static std::vector<expressions::ExpressionPtr> UpCast(
+      const std::vector<std::shared_ptr<const ScalarSubclassT>> &expressions) {
+    std::vector<expressions::ExpressionPtr> output;
+    for (const auto &expr : expressions) {
+      output.emplace_back(expr);
+    }
+    return output;
+  }
+
+  template <typename ScalarSubclassT>
+  static std::vector<std::shared_ptr<const ScalarSubclassT>> DownCast(
+      const std::vector<expressions::ExpressionPtr> &expressions) {
+    std::vector<std::shared_ptr<const ScalarSubclassT>> output;
+    for (const auto &expr : expressions) {
+      output.emplace_back(std::static_pointer_cast<const ScalarSubclassT>(expr));
+    }
+    return output;
+  }
+
+  struct ExpressionTypeHash {
+    using UnderT = std::underlying_type<expressions::ExpressionType>::type;
+
+    inline std::size_t operator()(const expressions::ExpressionType &et) const {
+      return std::hash<UnderT>()(static_cast<UnderT>(et));
+    }
+  };
+
+  // Here we define that an expression type is homogeneous if at execution time
+  // the result column vector of that expression has a one-to-one positional
+  // correspondence with all the result column vectors from its child expressions.
+  // E.g. aggregate functions and CASE expressions are not homogeneous.
+  //
+  // Being homogeneous enables common subexpressions to be hoisted through.
+  // For example, consider the case
+  // --
+  //   (x * 2) + F(x * 2)
+  // --
+  // where F is some unary expression. Then if F is homogenous, we can mark the
+  // two (x * 2) as common subexpressions. Otherwise, we cannot do that since
+  // the two subexpressions will generate different result column vectors.
+  std::unordered_set<expressions::ExpressionType,
+                     ExpressionTypeHash> homogeneous_expression_types_;
+
+  OptimizerContext *optimizer_context_;
+
+  DISALLOW_COPY_AND_ASSIGN(ExtractCommonSubexpression);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_EXTRACT_COMMON_SUBEXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/rules/ReuseAggregateExpressions.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReuseAggregateExpressions.cpp b/query_optimizer/rules/ReuseAggregateExpressions.cpp
new file mode 100644
index 0000000..79dede6
--- /dev/null
+++ b/query_optimizer/rules/ReuseAggregateExpressions.cpp
@@ -0,0 +1,349 @@
+/**
+ * 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/ReuseAggregateExpressions.hpp"
+
+#include <cstddef>
+#include <list>
+#include <map>
+#include <memory>
+#include <unordered_map>
+#include <vector>
+
+#include "expressions/aggregation/AggregateFunction.hpp"
+#include "expressions/aggregation/AggregateFunctionFactory.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "query_optimizer/OptimizerContext.hpp"
+#include "query_optimizer/expressions/AggregateFunction.hpp"
+#include "query_optimizer/expressions/Alias.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/BinaryExpression.hpp"
+#include "query_optimizer/expressions/ExpressionType.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/Scalar.hpp"
+#include "query_optimizer/physical/Aggregate.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 "types/operations/binary_operations/BinaryOperation.hpp"
+#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
+#include "types/operations/binary_operations/BinaryOperationID.hpp"
+#include "utility/HashError.hpp"
+
+#include "gflags/gflags.h"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+DEFINE_uint64(reuse_aggregate_group_size_threshold, 1000u,
+              "The threshold on estimated number of groups for an aggregation "
+              "below which the ReuseAggregateExpressions optimization will be "
+              "performed.");
+
+DEFINE_double(reuse_aggregate_ratio_threshold, 0.3,
+              "The threshold on the ratio of (# of eliminable columns) to "
+              "(# of original columns) for an aggregation above which the "
+              "ReuseAggregateExpressions optimization will be performed.");
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+void ReuseAggregateExpressions::init(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  // Initialize cost model.
+  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()));
+}
+
+P::PhysicalPtr ReuseAggregateExpressions::applyToNode(
+    const P::PhysicalPtr &input) {
+  P::AggregatePtr aggregate;
+  if (!P::SomeAggregate::MatchesWithConditionalCast(input, &aggregate)) {
+    return input;
+  }
+
+  const std::vector<E::AliasPtr> &agg_exprs = aggregate->aggregate_expressions();
+
+  // Maps aggregated expressions to AggregationID + positions.
+  //
+  // For example,
+  // --
+  //   SELECT SUM(x+1), AVG(x+1), SUM(x+1), COUNT(*), SUM(y) FROM s;
+  // --
+  // will generate *agg_expr_info* as
+  // --
+  // {
+  //   x+1: { kSum: [0, 2], kAvg: [1] },
+  //   y: { kSum: [4] },
+  // }
+  // --
+  // and COUNT(*) will be recorded inside *count_star_info*.
+  std::unordered_map<E::ScalarPtr,
+                     std::map<AggregationID, std::vector<std::size_t>>,
+                     ScalarHash, ScalarEqual> agg_expr_info;
+  std::list<std::size_t> count_star_info;
+
+  for (std::size_t i = 0; i < agg_exprs.size(); ++i) {
+    DCHECK(agg_exprs[i]->expression()->getExpressionType()
+               == E::ExpressionType::kAggregateFunction);
+    const E::AggregateFunctionPtr agg_expr =
+        std::static_pointer_cast<const E::AggregateFunction>(
+            agg_exprs[i]->expression());
+
+    // Skip DISTINCT aggregations.
+    if (agg_expr->is_distinct()) {
+      continue;
+    }
+
+    const AggregationID agg_id = agg_expr->getAggregate().getAggregationID();
+    const std::vector<E::ScalarPtr> &arguments = agg_expr->getArguments();
+
+    // Currently we only consider aggregate functions with 0 or 1 argument.
+    if (agg_id == AggregationID::kCount) {
+      if (arguments.empty()) {
+        count_star_info.emplace_front(i);
+        continue;
+      } else if (!arguments.front()->getValueType().isNullable()) {
+        // For COUNT(x) where x is not null, we view it as a more general COUNT(*).
+        count_star_info.emplace_back(i);
+        continue;
+      }
+    }
+    if (arguments.size() == 1) {
+      try {
+        agg_expr_info[arguments.front()][agg_id].emplace_back(i);
+      } catch (const HashNotSupported &e) {
+        continue;
+      }
+    }
+  }
+
+  // Now for each aggregate expression, figure out whether we can avoid the
+  // computation by reusing other aggregate expression's result.
+  std::vector<std::unique_ptr<AggregateReference>> agg_refs(agg_exprs.size());
+
+  constexpr std::size_t kInvalidRef = static_cast<std::size_t>(-1);
+  std::size_t count_star_ref;
+
+  // Check whether COUNT(*) is available.
+  if (count_star_info.empty()) {
+    count_star_ref = kInvalidRef;
+  } else {
+    auto it = count_star_info.begin();
+    count_star_ref = *it;
+
+    for (++it; it != count_star_info.end(); ++it) {
+      agg_refs[*it].reset(new AggregateReference(count_star_ref));
+    }
+  }
+
+  // Iterate through agg_expr_info to find all transformation opportunities,
+  // and record them into agg_refs.
+  for (const auto &it : agg_expr_info) {
+    const auto &ref_map = it.second;
+
+    // First, check whether AVG can be reduced to SUM/COUNT.
+    bool is_avg_processed = false;
+
+    const auto avg_it = ref_map.find(AggregationID::kAvg);
+    if (avg_it != ref_map.end()) {
+      std::size_t count_ref = kInvalidRef;
+
+      // To reduce AVG to SUM/COUNT, we need a COUNT available.
+      // TODO(jianqiao): We may even add a new COUNT(*) aggregation if it is not
+      // there. E.g. when there are at least two AVG(...) aggregate functions.
+      if (it.first->getValueType().isNullable()) {
+        const auto count_it = ref_map.find(AggregationID::kCount);
+        if (count_it != ref_map.end()) {
+          DCHECK(!count_it->second.empty());
+          count_ref = count_it->second.front();
+        }
+      } else {
+        count_ref = count_star_ref;
+      }
+
+      if (count_ref != kInvalidRef) {
+        // It is done if there is an existing SUM. Otherwise we strength-reduce
+        // the current AVG to SUM.
+        const auto sum_it = ref_map.find(AggregationID::kSum);
+        const std::size_t sum_ref =
+            sum_it == ref_map.end() ? kInvalidRef : sum_it->second.front();
+
+        for (const std::size_t idx : avg_it->second) {
+          agg_refs[idx].reset(new AggregateReference(sum_ref, count_ref));
+        }
+        is_avg_processed = true;
+      }
+    }
+
+    // Then, identify duplicate aggregate expressions.
+    for (const auto &ref_it : ref_map) {
+      if (ref_it.first == AggregationID::kAvg && is_avg_processed) {
+        continue;
+      }
+      const auto &indices = ref_it.second;
+      DCHECK(!indices.empty());
+      const std::size_t ref = indices.front();
+      for (std::size_t i = 1; i < indices.size(); ++i) {
+        agg_refs[indices[i]].reset(new AggregateReference(ref));
+      }
+    }
+  }
+
+  // Count the number of eliminable aggregate expressions.
+  std::size_t num_eliminable = 0;
+  for (const auto &agg_ref : agg_refs) {
+    if (agg_ref != nullptr) {
+      ++num_eliminable;
+    }
+  }
+
+  if (num_eliminable == 0) {
+    return input;
+  }
+
+  // Now we need to make a decision since it is not always benefitial to perform
+  // the transformation. Currently we employ a simple heuristic that if either
+  // (1) The estimated number of groups for this Aggregate node is smaller than
+  //     the specified FLAGS_reuse_aggregate_group_size_threshold.
+  // or
+  // (2) The ratio of (# of eliminable columns) to (# of original columns) is
+  //     greater than the specified FLAGS_reuse_aggregate_ratio_threshold.
+  // then we perform the transformation.
+  const bool is_group_size_condition_satisfied =
+      cost_model_->estimateNumGroupsForAggregate(aggregate)
+          < FLAGS_reuse_aggregate_group_size_threshold;
+  const bool is_ratio_condition_satisfied =
+      static_cast<double>(num_eliminable) / agg_exprs.size()
+          > FLAGS_reuse_aggregate_ratio_threshold;
+
+  if (!is_group_size_condition_satisfied && !is_ratio_condition_satisfied) {
+    return input;
+  }
+
+  // Now we transform the original Aggregate to a reduced Aggregate + a wrapping
+  // Selection.
+
+  // Aggregate expressions for the new Aggregate.
+  std::vector<E::AliasPtr> new_agg_exprs;
+
+  // Project expressions for the new Selection.
+  std::vector<E::NamedExpressionPtr> new_select_exprs;
+
+  for (const auto &grouping_expr : aggregate->grouping_expressions()) {
+    new_select_exprs.emplace_back(E::ToRef(grouping_expr));
+  }
+
+  const std::vector<E::AttributeReferencePtr> agg_attrs = E::ToRefVector(agg_exprs);
+
+  for (std::size_t i = 0; i < agg_refs.size(); ++i) {
+    const auto &agg_ref = agg_refs[i];
+    const E::AliasPtr &agg_expr = agg_exprs[i];
+
+    if (agg_ref == nullptr) {
+      // Case 1: this aggregate expression can not be eliminated.
+      new_agg_exprs.emplace_back(agg_expr);
+      new_select_exprs.emplace_back(
+          E::AttributeReference::Create(agg_expr->id(),
+                                        agg_expr->attribute_name(),
+                                        agg_expr->attribute_alias(),
+                                        agg_expr->relation_name(),
+                                        agg_expr->getValueType(),
+                                        E::AttributeReferenceScope::kLocal));
+    } else {
+      switch (agg_ref->kind) {
+        // Case 2.1: this aggregate expression can be eliminated.
+        case AggregateReference::kDirect: {
+          new_select_exprs.emplace_back(
+              E::Alias::Create(agg_expr->id(),
+                               agg_attrs[agg_ref->first_ref],
+                               agg_expr->attribute_name(),
+                               agg_expr->attribute_alias(),
+                               agg_expr->relation_name()));
+          break;
+        }
+        // Case 2.2: this aggregate expression is an AVG.
+        case AggregateReference::kAvg: {
+          E::AttributeReferencePtr sum_attr;
+          if (agg_ref->first_ref == kInvalidRef) {
+            // Case 2.2.1: If there is no existing SUM, we need to convert this
+            //             AVG to SUM.
+            const E::AggregateFunctionPtr avg_expr =
+                std::static_pointer_cast<const E::AggregateFunction>(agg_expr->expression());
+
+            const AggregateFunction &sum_func =
+                AggregateFunctionFactory::Get(AggregationID::kSum);
+            const E::AggregateFunctionPtr sum_expr =
+                E::AggregateFunction::Create(sum_func,
+                                             avg_expr->getArguments(),
+                                             avg_expr->is_vector_aggregate(),
+                                             avg_expr->is_distinct());
+            new_agg_exprs.emplace_back(
+                E::Alias::Create(optimizer_context_->nextExprId(),
+                                 sum_expr,
+                                 agg_expr->attribute_name(),
+                                 agg_expr->attribute_alias(),
+                                 agg_expr->relation_name()));
+
+            sum_attr = E::ToRef(new_agg_exprs.back());
+          } else {
+            // Case 2.2.2: If there is a SUM somewhere, we just eliminate this
+            //             AVG and use the result from that SUM.
+            sum_attr = agg_attrs[agg_ref->first_ref];
+          }
+
+          // Obtain AVG by evaluating SUM/COUNT in Selection.
+          const BinaryOperation &divide_op =
+              BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide);
+          const E::BinaryExpressionPtr avg_expr =
+              E::BinaryExpression::Create(divide_op,
+                                          sum_attr,
+                                          agg_attrs[agg_ref->second_ref]);
+          new_select_exprs.emplace_back(
+              E::Alias::Create(agg_expr->id(),
+                               avg_expr,
+                               agg_expr->attribute_name(),
+                               agg_expr->attribute_alias(),
+                               agg_expr->relation_name()));
+        }
+      }
+    }
+  }
+
+  // Construct the reduced Aggregate.
+  const P::AggregatePtr new_aggregate =
+      P::Aggregate::Create(aggregate->input(),
+                           aggregate->grouping_expressions(),
+                           new_agg_exprs,
+                           aggregate->filter_predicate());
+
+  // Construct the wrapping Selection.
+  return P::Selection::Create(new_aggregate, new_select_exprs, nullptr);
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/rules/ReuseAggregateExpressions.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/ReuseAggregateExpressions.hpp b/query_optimizer/rules/ReuseAggregateExpressions.hpp
new file mode 100644
index 0000000..182e9d9
--- /dev/null
+++ b/query_optimizer/rules/ReuseAggregateExpressions.hpp
@@ -0,0 +1,154 @@
+/**
+ * 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_REUSE_AGGREGATE_EXPRESSIONS_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_REUSE_AGGREGATE_EXPRESSIONS_HPP_
+
+#include <cstddef>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/Scalar.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/BottomUpRule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+class OptimizerContext;
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+/**
+ * @brief Rule that applies to a physical plan to eliminate duplicate aggregate
+ *        expressions and convert AVG to SUM/COUNT if appropriate.
+ *
+ * This rule rewrites Aggregate to Selection + Aggregate to eliminate duplicates
+ * and strength-reduce AVG functions. E.g.
+ * --
+ *   SELECT SUM(x), SUM(y), SUM(x) * 2, AVG(y), COUNT(*)
+ *   FROM s;
+ * --
+ * will be rewritten as (assume y is not null)
+ * --
+ *   SELECT sum_x, sum_y, sum_x * 2, sum_y / cnt, cnt
+ *   FROM (
+ *     SELECT SUM(x) AS sum_x, SUM(y) AS sum_y, COUNT(*) AS cnt
+ *   ) t;
+ * --
+ *
+ * Meanwhile, note that currently it is not free-of-cost to "re-project" the
+ * columns. So it may not worth doing the transformation in some situations.
+ * E.g. it may actually slow down the query to rewrite
+ * --
+ *   SELECT SUM(a), SUM(b), SUM(c), SUM(d), SUM(a)
+ *   FROM s
+ *   GROUP BY x;
+ * --
+ * as
+ * --
+ *   SELECT sum_a, sum_b, sum_c, sum_d, sum_a
+ *   FROM (
+ *     SELECT SUM(a) AS sum_a, SUM(b) AS sum_b, SUM(c) AS sum_c, SUM(d) AS sum_d
+ *     FROM s
+ *     GROUP BY x;
+ *   ) t;
+ * --
+ * if the number of distinct values of attribute x is large -- in this case, we
+ * avoid one duplicate computation of SUM(a), but introduce 5 extra expensive
+ * column copying with the outer Selection.
+ *
+ * To address the above problem, currently we employ a simple heuristic that if
+ * either
+ * (1) The estimated number of groups for this Aggregate node is smaller than
+ *     the specified FLAGS_reuse_aggregate_group_size_threshold.
+ * or
+ * (2) The ratio of (# of eliminable columns) to (# of original columns) is
+ *     greater than the specified FLAGS_reuse_aggregate_ratio_threshold.
+ * then we perform the transformation.
+ */
+class ReuseAggregateExpressions : public BottomUpRule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param optimizer_context The optimizer context.
+   */
+  explicit ReuseAggregateExpressions(OptimizerContext *optimizer_context)
+      : optimizer_context_(optimizer_context) {}
+
+  std::string getName() const override {
+    return "ReuseAggregateExpressions";
+  }
+
+ protected:
+  void init(const physical::PhysicalPtr &input) override;
+
+  physical::PhysicalPtr applyToNode(const physical::PhysicalPtr &input) override;
+
+ private:
+  struct ScalarHash {
+    inline std::size_t operator()(const expressions::ScalarPtr &scalar) const {
+      return scalar->hash();
+    }
+  };
+
+  struct ScalarEqual {
+    inline bool operator()(const expressions::ScalarPtr &lhs,
+                           const expressions::ScalarPtr &rhs) const {
+      return lhs->equals(rhs);
+    }
+  };
+
+  // This data structure indicates for each aggregate expression whether the
+  // expression can be eliminated by refering to another identical expression,
+  // or can be strength-reduced from AVG to SUM.
+  struct AggregateReference {
+    enum Kind {
+      kDirect = 0,
+      kAvg
+    };
+
+    explicit AggregateReference(const std::size_t ref)
+        : kind(kDirect), first_ref(ref), second_ref(0) {}
+
+    AggregateReference(const std::size_t sum_ref, const std::size_t count_ref)
+        : kind(kAvg), first_ref(sum_ref), second_ref(count_ref) {}
+
+    const Kind kind;
+    const std::size_t first_ref;
+    const std::size_t second_ref;
+  };
+
+  OptimizerContext *optimizer_context_;
+
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
+
+  DISALLOW_COPY_AND_ASSIGN(ReuseAggregateExpressions);
+};
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_REUSE_AGGREGATE_EXPRESSIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/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 40629ee..595d09d 100644
--- a/query_optimizer/tests/execution_generator/CMakeLists.txt
+++ b/query_optimizer/tests/execution_generator/CMakeLists.txt
@@ -15,6 +15,11 @@
 # specific language governing permissions and limitations
 # under the License.
 
+add_test(quickstep_queryoptimizer_tests_executiongenerator_commonsubexpression
+         "../quickstep_queryoptimizer_tests_ExecutionGeneratorTest"
+         "${CMAKE_CURRENT_SOURCE_DIR}/CommonSubexpression.test"
+         "${CMAKE_CURRENT_BINARY_DIR}/CommonSubexpression.test"
+         "${CMAKE_CURRENT_BINARY_DIR}/CommonSubexpression/")
 add_test(quickstep_queryoptimizer_tests_executiongenerator_create
          "../quickstep_queryoptimizer_tests_ExecutionGeneratorTest"
          "${CMAKE_CURRENT_SOURCE_DIR}/Create.test"
@@ -77,6 +82,11 @@ add_test(quickstep_queryoptimizer_tests_executiongenerator_update
          "${CMAKE_CURRENT_BINARY_DIR}/Update/")
 
 if (ENABLE_DISTRIBUTED)
+  add_test(quickstep_queryoptimizer_tests_executiongenerator_commonsubexpression_distributed
+           "../quickstep_queryoptimizer_tests_DistributedExecutionGeneratorTest"
+           "${CMAKE_CURRENT_SOURCE_DIR}/CommonSubexpression.test"
+           "${CMAKE_CURRENT_BINARY_DIR}/CommonSubexpressionDistributed.test"
+           "${CMAKE_CURRENT_BINARY_DIR}/CommonSubexpressionDistributed/")
   add_test(quickstep_queryoptimizer_tests_executiongenerator_create_distributed
            "../quickstep_queryoptimizer_tests_DistributedExecutionGeneratorTest"
            "${CMAKE_CURRENT_SOURCE_DIR}/Create.test"
@@ -141,6 +151,7 @@ endif(ENABLE_DISTRIBUTED)
 
 # Create the folders where the unit tests will store their data blocks for the
 # duration of their test.
+file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/CommonSubexpression)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Create)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Delete)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Distinct)
@@ -155,6 +166,7 @@ file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/TableGenerator)
 file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/Update)
 
 if (ENABLE_DISTRIBUTED)
+  file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/CommonSubexpressionDistributed)
   file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/CreateDistributed)
   file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/DeleteDistributed)
   file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/DistinctDistributed)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/tests/execution_generator/CommonSubexpression.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/execution_generator/CommonSubexpression.test b/query_optimizer/tests/execution_generator/CommonSubexpression.test
new file mode 100644
index 0000000..e0b5e2d
--- /dev/null
+++ b/query_optimizer/tests/execution_generator/CommonSubexpression.test
@@ -0,0 +1,52 @@
+# 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.
+
+SELECT int_col+1, 1+int_col
+FROM test
+ORDER BY int_col
+LIMIT 5;
+--
++-----------+-----------+
+|(int_col+1)|(1+int_col)|
++-----------+-----------+
+|        -22|        -22|
+|        -20|        -20|
+|        -18|        -18|
+|        -16|        -16|
+|        -14|        -14|
++-----------+-----------+
+==
+
+SELECT SUM(int_col+long_col), SUM((long_col+int_col)*2)
+FROM test;
+--
++-----------------------+---------------------------+
+|SUM((int_col+long_col))|SUM(((long_col+int_col)*2))|
++-----------------------+---------------------------+
+|                   4382|                       8764|
++-----------------------+---------------------------+
+==
+
+SELECT SUM(long_col+1), SUM(long_col+1)*2, AVG(long_col+1), AVG(long_col+1)*2, COUNT(*)
+FROM test;
+--
++--------------------+---------------------+--------------------+---------------------+--------------------+
+|SUM((long_col+1))   |(SUM((long_col+1))*2)|AVG((long_col+1))   |(AVG((long_col+1))*2)|COUNT(*)            |
++--------------------+---------------------+--------------------+---------------------+--------------------+
+|                4925|                 9850|                 197|                  394|                  25|
++--------------------+---------------------+--------------------+---------------------+--------------------+
+==

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/tests/physical_generator/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/CMakeLists.txt b/query_optimizer/tests/physical_generator/CMakeLists.txt
index c752cdd..fc3390e 100644
--- a/query_optimizer/tests/physical_generator/CMakeLists.txt
+++ b/query_optimizer/tests/physical_generator/CMakeLists.txt
@@ -15,6 +15,10 @@
 # specific language governing permissions and limitations
 # under the License.
 
+add_test(quickstep_queryoptimizer_tests_physicalgenerator_commonsubexpression
+         "../quickstep_queryoptimizer_tests_OptimizerTextTest"
+         "${CMAKE_CURRENT_SOURCE_DIR}/CommonSubexpression.test"
+         "${CMAKE_CURRENT_BINARY_DIR}/CommonSubexpression.test")
 add_test(quickstep_queryoptimizer_tests_physicalgenerator_copy
          "../quickstep_queryoptimizer_tests_OptimizerTextTest"
          "${CMAKE_CURRENT_SOURCE_DIR}/Copy.test"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/tests/physical_generator/CommonSubexpression.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/CommonSubexpression.test b/query_optimizer/tests/physical_generator/CommonSubexpression.test
new file mode 100644
index 0000000..b23a97d
--- /dev/null
+++ b/query_optimizer/tests/physical_generator/CommonSubexpression.test
@@ -0,0 +1,772 @@
+# 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.
+
+[default optimized_logical_plan physical_plan]
+
+SELECT int_col+1, 1+int_col
+FROM test;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=TableReference[relation_name=Test,relation_alias=test]
+| | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | +-AttributeReference[id=5,name=vchar_col,relation=test,type=VarChar(20) NULL]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=(int_col+1),relation=,type=Int NULL]
+|   | +-Add
+|   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |   +-Literal[value=1,type=Int]
+|   +-Alias[id=7,name=,alias=(1+int_col),relation=,type=Int NULL]
+|     +-Add
+|       +-Literal[value=1,type=Int]
+|       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=(int_col+1),relation=,type=Int NULL]
+  +-AttributeReference[id=7,name=,alias=(1+int_col),relation=,type=Int NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=TableReference[relation=Test,alias=test]
+| | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | +-AttributeReference[id=5,name=vchar_col,relation=test,type=VarChar(20) NULL]
+| +-project_expressions=
+|   +-Alias[id=6,name=,alias=(int_col+1),relation=,type=Int NULL]
+|   | +-CommonSubexpression[common_subexpression_id=8]
+|   |   +-Operand=Add
+|   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |     +-Literal[value=1,type=Int]
+|   +-Alias[id=7,name=,alias=(1+int_col),relation=,type=Int NULL]
+|     +-CommonSubexpression[common_subexpression_id=8]
+|       +-Operand=Add
+|         +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|         +-Literal[value=1,type=Int]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=(int_col+1),relation=,type=Int NULL]
+  +-AttributeReference[id=7,name=,alias=(1+int_col),relation=,type=Int NULL]
+==
+
+# Aggregate
+SELECT SUM(int_col+float_col), MIN((float_col+int_col)*2)
+FROM test;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=Aggregate
+| | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   type=VarChar(20) NULL]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Double NULL]
+| |   | +-AggregateFunction[function=SUM]
+| |   |   +-Add
+| |   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |   |     +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |   +-Alias[id=7,name=,alias=$aggregate1,relation=$aggregate,type=Float NULL]
+| |     +-AggregateFunction[function=MIN]
+| |       +-Multiply
+| |         +-Add
+| |         | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |         | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |         +-Literal[value=2,type=Int]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=SUM((int_col+float_col)),relation=,type=Double NULL]
+|   | +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+|   |   type=Double NULL]
+|   +-Alias[id=7,name=,alias=MIN(((float_col+int_col)*2)),relation=,
+|     type=Float NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate1,relation=$aggregate,
+|       type=Float NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=SUM((int_col+float_col)),relation=,
+  | type=Double NULL]
+  +-AttributeReference[id=7,name=,alias=MIN(((float_col+int_col)*2)),relation=,
+    type=Float NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=Aggregate
+| | +-input=TableReference[relation=Test,alias=test]
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   type=VarChar(20) NULL]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Double NULL]
+| |   | +-AggregateFunction[function=SUM]
+| |   |   +-CommonSubexpression[common_subexpression_id=8]
+| |   |     +-Operand=Add
+| |   |       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |   |       +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |   +-Alias[id=7,name=,alias=$aggregate1,relation=$aggregate,type=Float NULL]
+| |     +-AggregateFunction[function=MIN]
+| |       +-Multiply
+| |         +-CommonSubexpression[common_subexpression_id=8]
+| |         | +-Operand=Add
+| |         |   +-AttributeReference[id=0,name=int_col,relation=test,
+| |         |   | type=Int NULL]
+| |         |   +-AttributeReference[id=2,name=float_col,relation=test,
+| |         |     type=Float]
+| |         +-Literal[value=2,type=Int]
+| +-project_expressions=
+|   +-Alias[id=6,name=,alias=SUM((int_col+float_col)),relation=,type=Double NULL]
+|   | +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+|   |   type=Double NULL]
+|   +-Alias[id=7,name=,alias=MIN(((float_col+int_col)*2)),relation=,
+|     type=Float NULL]
+|     +-AttributeReference[id=7,name=,alias=$aggregate1,relation=$aggregate,
+|       type=Float NULL]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=SUM((int_col+float_col)),relation=,
+  | type=Double NULL]
+  +-AttributeReference[id=7,name=,alias=MIN(((float_col+int_col)*2)),relation=,
+    type=Float NULL]
+==
+
+# HashJoin
+SELECT int_col + j, (int_col + j) * float_col
+FROM test, (SELECT i, i * i AS j FROM generate_series(1, 10) AS g(i)) t
+WHERE int_col = i AND (int_col + i) < float_col;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=Filter
+| | +-input=HashJoin
+| | | +-left=TableReference[relation_name=Test,relation_alias=test]
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | | |   type=VarChar(20) NULL]
+| | | +-right=Project
+| | | | +-input=TableGenerator[function_name=generate_series,table_alias=g]
+| | | | | +-AttributeReference[id=6,name=generate_series,alias=g,
+| | | | |   relation=generate_series,type=Int]
+| | | | +-project_list=
+| | | |   +-Alias[id=7,name=i,relation=,type=Int]
+| | | |   | +-AttributeReference[id=6,name=generate_series,alias=g,
+| | | |   |   relation=generate_series,type=Int]
+| | | |   +-Alias[id=8,name=j,relation=t,type=Int]
+| | | |     +-Multiply
+| | | |       +-AttributeReference[id=6,name=generate_series,alias=g,
+| | | |       | relation=generate_series,type=Int]
+| | | |       +-AttributeReference[id=6,name=generate_series,alias=g,
+| | | |         relation=generate_series,type=Int]
+| | | +-left_join_attributes=
+| | | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-right_join_attributes=
+| | |   +-AttributeReference[id=7,name=i,relation=,type=Int]
+| | +-filter_predicate=Less
+| |   +-Add
+| |   | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |   | +-AttributeReference[id=7,name=i,relation=,type=Int]
+| |   +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| +-project_list=
+|   +-Alias[id=9,name=,alias=(int_col+j),relation=,type=Int NULL]
+|   | +-Add
+|   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |   +-AttributeReference[id=8,name=j,relation=t,type=Int]
+|   +-Alias[id=10,name=,alias=((int_col+j)*float_col),relation=,type=Float NULL]
+|     +-Multiply
+|       +-Add
+|       | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|       | +-AttributeReference[id=8,name=j,relation=t,type=Int]
+|       +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
++-output_attributes=
+  +-AttributeReference[id=9,name=,alias=(int_col+j),relation=,type=Int NULL]
+  +-AttributeReference[id=10,name=,alias=((int_col+j)*float_col),relation=,
+    type=Float NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=HashJoin
+| +-left=TableGenerator[function_name=generate_series,table_alias=g]
+| | +-AttributeReference[id=6,name=generate_series,alias=g,
+| |   relation=generate_series,type=Int]
+| +-right=TableReference[relation=Test,alias=test]
+| | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | +-AttributeReference[id=5,name=vchar_col,relation=test,type=VarChar(20) NULL]
+| +-residual_predicate=Less
+| | +-Add
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=6,name=generate_series,alias=g,
+| | |   relation=generate_series,type=Int]
+| | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| +-project_expressions=
+| | +-Alias[id=9,name=,alias=(int_col+j),relation=,type=Int NULL]
+| | | +-CommonSubexpression[common_subexpression_id=11]
+| | |   +-Operand=Add
+| | |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | |     +-Multiply
+| | |       +-AttributeReference[id=6,name=generate_series,alias=g,
+| | |       | relation=generate_series,type=Int]
+| | |       +-AttributeReference[id=6,name=generate_series,alias=g,
+| | |         relation=generate_series,type=Int]
+| | +-Alias[id=10,name=,alias=((int_col+j)*float_col),relation=,type=Float NULL]
+| |   +-Multiply
+| |     +-CommonSubexpression[common_subexpression_id=11]
+| |     | +-Operand=Add
+| |     |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |     |   +-Multiply
+| |     |     +-AttributeReference[id=6,name=generate_series,alias=g,
+| |     |     | relation=generate_series,type=Int]
+| |     |     +-AttributeReference[id=6,name=generate_series,alias=g,
+| |     |       relation=generate_series,type=Int]
+| |     +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| +-left_join_attributes=
+| | +-AttributeReference[id=6,name=generate_series,alias=g,
+| |   relation=generate_series,type=Int]
+| +-right_join_attributes=
+|   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
++-output_attributes=
+  +-AttributeReference[id=9,name=,alias=(int_col+j),relation=,type=Int NULL]
+  +-AttributeReference[id=10,name=,alias=((int_col+j)*float_col),relation=,
+    type=Float NULL]
+==
+
+# NestedLoopsJoin
+SELECT int_col + j, (int_col + j) * float_col
+FROM test, (SELECT i, i * i AS j FROM generate_series(1, 10) AS g(i)) t
+WHERE (int_col + i) < float_col;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=NestedLoopsJoin
+| | +-left=TableReference[relation_name=Test,relation_alias=test]
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   type=VarChar(20) NULL]
+| | +-right=Project
+| | | +-input=TableGenerator[function_name=generate_series,table_alias=g]
+| | | | +-AttributeReference[id=6,name=generate_series,alias=g,
+| | | |   relation=generate_series,type=Int]
+| | | +-project_list=
+| | |   +-Alias[id=7,name=i,relation=,type=Int]
+| | |   | +-AttributeReference[id=6,name=generate_series,alias=g,
+| | |   |   relation=generate_series,type=Int]
+| | |   +-Alias[id=8,name=j,relation=t,type=Int]
+| | |     +-Multiply
+| | |       +-AttributeReference[id=6,name=generate_series,alias=g,
+| | |       | relation=generate_series,type=Int]
+| | |       +-AttributeReference[id=6,name=generate_series,alias=g,
+| | |         relation=generate_series,type=Int]
+| | +-join_predicate=Less
+| |   +-Add
+| |   | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| |   | +-AttributeReference[id=7,name=i,relation=,type=Int]
+| |   +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| +-project_list=
+|   +-Alias[id=9,name=,alias=(int_col+j),relation=,type=Int NULL]
+|   | +-Add
+|   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |   +-AttributeReference[id=8,name=j,relation=t,type=Int]
+|   +-Alias[id=10,name=,alias=((int_col+j)*float_col),relation=,type=Float NULL]
+|     +-Multiply
+|       +-Add
+|       | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|       | +-AttributeReference[id=8,name=j,relation=t,type=Int]
+|       +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
++-output_attributes=
+  +-AttributeReference[id=9,name=,alias=(int_col+j),relation=,type=Int NULL]
+  +-AttributeReference[id=10,name=,alias=((int_col+j)*float_col),relation=,
+    type=Float NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=NestedLoopsJoin
+| +-left=TableReference[relation=Test,alias=test]
+| | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | +-AttributeReference[id=5,name=vchar_col,relation=test,type=VarChar(20) NULL]
+| +-right=TableGenerator[function_name=generate_series,table_alias=g]
+| | +-AttributeReference[id=6,name=generate_series,alias=g,
+| |   relation=generate_series,type=Int]
+| +-join_predicate=Less
+| | +-Add
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=6,name=generate_series,alias=g,
+| | |   relation=generate_series,type=Int]
+| | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| +-project_expressions=
+|   +-Alias[id=9,name=,alias=(int_col+j),relation=,type=Int NULL]
+|   | +-CommonSubexpression[common_subexpression_id=11]
+|   |   +-Operand=Add
+|   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |     +-Multiply
+|   |       +-AttributeReference[id=6,name=generate_series,alias=g,
+|   |       | relation=generate_series,type=Int]
+|   |       +-AttributeReference[id=6,name=generate_series,alias=g,
+|   |         relation=generate_series,type=Int]
+|   +-Alias[id=10,name=,alias=((int_col+j)*float_col),relation=,type=Float NULL]
+|     +-Multiply
+|       +-CommonSubexpression[common_subexpression_id=11]
+|       | +-Operand=Add
+|       |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|       |   +-Multiply
+|       |     +-AttributeReference[id=6,name=generate_series,alias=g,
+|       |     | relation=generate_series,type=Int]
+|       |     +-AttributeReference[id=6,name=generate_series,alias=g,
+|       |       relation=generate_series,type=Int]
+|       +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
++-output_attributes=
+  +-AttributeReference[id=9,name=,alias=(int_col+j),relation=,type=Int NULL]
+  +-AttributeReference[id=10,name=,alias=((int_col+j)*float_col),relation=,
+    type=Float NULL]
+==
+
+# Case expression
+SELECT long_col+1,
+       CASE WHEN int_col = 1 THEN (long_col+1)*(long_col+1)
+            WHEN int_col = 2 THEN (long_col+1)*(long_col+1)
+            ELSE long_col+1 END AS result
+FROM test;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=TableReference[relation_name=Test,relation_alias=test]
+| | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | +-AttributeReference[id=5,name=vchar_col,relation=test,type=VarChar(20) NULL]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=(long_col+1),relation=,type=Long]
+|   | +-Add
+|   |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|   |   +-Literal[value=1,type=Int]
+|   +-Alias[id=7,name=result,relation=,type=Long]
+|     +-SearchedCase
+|       +-else_result_expression=Add
+|       | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|       | +-Literal[value=1,type=Long]
+|       +-condition_perdicates=
+|       | +-Equal
+|       | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|       | | +-Literal[value=1,type=Int]
+|       | +-Equal
+|       |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|       |   +-Literal[value=2,type=Int]
+|       +-conditional_result_expressions=
+|         +-Multiply
+|         | +-Add
+|         | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|         | | +-Literal[value=1,type=Int]
+|         | +-Add
+|         |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|         |   +-Literal[value=1,type=Int]
+|         +-Multiply
+|           +-Add
+|           | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|           | +-Literal[value=1,type=Int]
+|           +-Add
+|             +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|             +-Literal[value=1,type=Int]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=(long_col+1),relation=,type=Long]
+  +-AttributeReference[id=7,name=result,relation=,type=Long]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=TableReference[relation=Test,alias=test]
+| | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | +-AttributeReference[id=5,name=vchar_col,relation=test,type=VarChar(20) NULL]
+| +-project_expressions=
+|   +-Alias[id=6,name=,alias=(long_col+1),relation=,type=Long]
+|   | +-Add
+|   |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|   |   +-Literal[value=1,type=Int]
+|   +-Alias[id=7,name=result,relation=,type=Long]
+|     +-SearchedCase
+|       +-else_result_expression=Add
+|       | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|       | +-Literal[value=1,type=Long]
+|       +-condition_perdicates=
+|       | +-Equal
+|       | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|       | | +-Literal[value=1,type=Int]
+|       | +-Equal
+|       |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|       |   +-Literal[value=2,type=Int]
+|       +-conditional_result_expressions=
+|         +-Multiply
+|         | +-CommonSubexpression[common_subexpression_id=8]
+|         | | +-Operand=Add
+|         | |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|         | |   +-Literal[value=1,type=Int]
+|         | +-CommonSubexpression[common_subexpression_id=8]
+|         |   +-Operand=Add
+|         |     +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|         |     +-Literal[value=1,type=Int]
+|         +-Multiply
+|           +-CommonSubexpression[common_subexpression_id=9]
+|           | +-Operand=Add
+|           |   +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|           |   +-Literal[value=1,type=Int]
+|           +-CommonSubexpression[common_subexpression_id=9]
+|             +-Operand=Add
+|               +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+|               +-Literal[value=1,type=Int]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=(long_col+1),relation=,type=Long]
+  +-AttributeReference[id=7,name=result,relation=,type=Long]
+==
+
+SELECT (int_col+1)*(int_col+2)*(int_col+3), (int_col+1)*(int_col+2), int_col+1
+FROM test;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=TableReference[relation_name=Test,relation_alias=test]
+| | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | +-AttributeReference[id=5,name=vchar_col,relation=test,type=VarChar(20) NULL]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=(((int_col+1)*(int_col+2))*(int_col+3)),relation=,
+|   | type=Int NULL]
+|   | +-Multiply
+|   |   +-Multiply
+|   |   | +-Add
+|   |   | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |   | | +-Literal[value=1,type=Int]
+|   |   | +-Add
+|   |   |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |   |   +-Literal[value=2,type=Int]
+|   |   +-Add
+|   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |     +-Literal[value=3,type=Int]
+|   +-Alias[id=7,name=,alias=((int_col+1)*(int_col+2)),relation=,type=Int NULL]
+|   | +-Multiply
+|   |   +-Add
+|   |   | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |   | +-Literal[value=1,type=Int]
+|   |   +-Add
+|   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |     +-Literal[value=2,type=Int]
+|   +-Alias[id=8,name=,alias=(int_col+1),relation=,type=Int NULL]
+|     +-Add
+|       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|       +-Literal[value=1,type=Int]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=(((int_col+1)*(int_col+2))*(int_col+3)),
+  | relation=,type=Int NULL]
+  +-AttributeReference[id=7,name=,alias=((int_col+1)*(int_col+2)),relation=,
+  | type=Int NULL]
+  +-AttributeReference[id=8,name=,alias=(int_col+1),relation=,type=Int NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=TableReference[relation=Test,alias=test]
+| | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | +-AttributeReference[id=5,name=vchar_col,relation=test,type=VarChar(20) NULL]
+| +-project_expressions=
+|   +-Alias[id=6,name=,alias=(((int_col+1)*(int_col+2))*(int_col+3)),relation=,
+|   | type=Int NULL]
+|   | +-Multiply
+|   |   +-CommonSubexpression[common_subexpression_id=10]
+|   |   | +-Operand=Multiply
+|   |   |   +-CommonSubexpression[common_subexpression_id=9]
+|   |   |   | +-Operand=Add
+|   |   |   |   +-AttributeReference[id=0,name=int_col,relation=test,
+|   |   |   |   | type=Int NULL]
+|   |   |   |   +-Literal[value=1,type=Int]
+|   |   |   +-Add
+|   |   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |   |     +-Literal[value=2,type=Int]
+|   |   +-Add
+|   |     +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |     +-Literal[value=3,type=Int]
+|   +-Alias[id=7,name=,alias=((int_col+1)*(int_col+2)),relation=,type=Int NULL]
+|   | +-CommonSubexpression[common_subexpression_id=10]
+|   |   +-Operand=Multiply
+|   |     +-CommonSubexpression[common_subexpression_id=9]
+|   |     | +-Operand=Add
+|   |     |   +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |     |   +-Literal[value=1,type=Int]
+|   |     +-Add
+|   |       +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|   |       +-Literal[value=2,type=Int]
+|   +-Alias[id=8,name=,alias=(int_col+1),relation=,type=Int NULL]
+|     +-CommonSubexpression[common_subexpression_id=9]
+|       +-Operand=Add
+|         +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+|         +-Literal[value=1,type=Int]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=(((int_col+1)*(int_col+2))*(int_col+3)),
+  | relation=,type=Int NULL]
+  +-AttributeReference[id=7,name=,alias=((int_col+1)*(int_col+2)),relation=,
+  | type=Int NULL]
+  +-AttributeReference[id=8,name=,alias=(int_col+1),relation=,type=Int NULL]
+==
+
+# Reuse aggregate expressions
+SELECT SUM(long_col+1), AVG(1+long_col), MIN(float_col), MIN(float_col)*2, COUNT(*)
+FROM test;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=Aggregate
+| | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   type=VarChar(20) NULL]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
+| |   | +-AggregateFunction[function=SUM]
+| |   |   +-Add
+| |   |     +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |   |     +-Literal[value=1,type=Int]
+| |   +-Alias[id=7,name=,alias=$aggregate1,relation=$aggregate,type=Double NULL]
+| |   | +-AggregateFunction[function=AVG]
+| |   |   +-Add
+| |   |     +-Literal[value=1,type=Int]
+| |   |     +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |   +-Alias[id=8,name=,alias=$aggregate2,relation=$aggregate,type=Float NULL]
+| |   | +-AggregateFunction[function=MIN]
+| |   |   +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |   +-Alias[id=9,name=,alias=$aggregate3,relation=$aggregate,type=Float NULL]
+| |   | +-AggregateFunction[function=MIN]
+| |   |   +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |   +-Alias[id=11,name=,alias=$aggregate4,relation=$aggregate,type=Long]
+| |     +-AggregateFunction[function=COUNT]
+| |       +-[]
+| +-project_list=
+|   +-Alias[id=6,name=,alias=SUM((long_col+1)),relation=,type=Long NULL]
+|   | +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+|   |   type=Long NULL]
+|   +-Alias[id=7,name=,alias=AVG((1+long_col)),relation=,type=Double NULL]
+|   | +-AttributeReference[id=7,name=,alias=$aggregate1,relation=$aggregate,
+|   |   type=Double NULL]
+|   +-Alias[id=8,name=,alias=MIN(float_col),relation=,type=Float NULL]
+|   | +-AttributeReference[id=8,name=,alias=$aggregate2,relation=$aggregate,
+|   |   type=Float NULL]
+|   +-Alias[id=10,name=,alias=(MIN(float_col)*2),relation=,type=Float NULL]
+|   | +-Multiply
+|   |   +-AttributeReference[id=9,name=,alias=$aggregate3,relation=$aggregate,
+|   |   | type=Float NULL]
+|   |   +-Literal[value=2,type=Int]
+|   +-Alias[id=11,name=,alias=COUNT(*),relation=,type=Long]
+|     +-AttributeReference[id=11,name=,alias=$aggregate4,relation=$aggregate,
+|       type=Long]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=SUM((long_col+1)),relation=,
+  | type=Long NULL]
+  +-AttributeReference[id=7,name=,alias=AVG((1+long_col)),relation=,
+  | type=Double NULL]
+  +-AttributeReference[id=8,name=,alias=MIN(float_col),relation=,type=Float NULL]
+  +-AttributeReference[id=10,name=,alias=(MIN(float_col)*2),relation=,
+  | type=Float NULL]
+  +-AttributeReference[id=11,name=,alias=COUNT(*),relation=,type=Long]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=Aggregate
+| | +-input=TableReference[relation=Test,alias=test]
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   type=VarChar(20) NULL]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
+| |   | +-AggregateFunction[function=SUM]
+| |   |   +-Add
+| |   |     +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |   |     +-Literal[value=1,type=Int]
+| |   +-Alias[id=8,name=,alias=$aggregate2,relation=$aggregate,type=Float NULL]
+| |   | +-AggregateFunction[function=MIN]
+| |   |   +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| |   +-Alias[id=11,name=,alias=$aggregate4,relation=$aggregate,type=Long]
+| |     +-AggregateFunction[function=COUNT]
+| |       +-[]
+| +-project_expressions=
+|   +-Alias[id=6,name=,alias=SUM((long_col+1)),relation=,type=Long NULL]
+|   | +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+|   |   type=Long NULL]
+|   +-Alias[id=7,name=,alias=AVG((1+long_col)),relation=,type=Long NULL]
+|   | +-Divide
+|   |   +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+|   |   | type=Long NULL]
+|   |   +-AttributeReference[id=11,name=,alias=$aggregate4,relation=$aggregate,
+|   |     type=Long]
+|   +-Alias[id=8,name=,alias=MIN(float_col),relation=,type=Float NULL]
+|   | +-AttributeReference[id=8,name=,alias=$aggregate2,relation=$aggregate,
+|   |   type=Float NULL]
+|   +-Alias[id=10,name=,alias=(MIN(float_col)*2),relation=,type=Float NULL]
+|   | +-Multiply
+|   |   +-AttributeReference[id=8,name=,alias=$aggregate2,relation=$aggregate,
+|   |   | type=Float NULL]
+|   |   +-Literal[value=2,type=Int]
+|   +-Alias[id=11,name=,alias=COUNT(*),relation=,type=Long]
+|     +-AttributeReference[id=11,name=,alias=$aggregate4,relation=$aggregate,
+|       type=Long]
++-output_attributes=
+  +-AttributeReference[id=6,name=,alias=SUM((long_col+1)),relation=,
+  | type=Long NULL]
+  +-AttributeReference[id=7,name=,alias=AVG((1+long_col)),relation=,
+  | type=Long NULL]
+  +-AttributeReference[id=8,name=,alias=MIN(float_col),relation=,type=Float NULL]
+  +-AttributeReference[id=10,name=,alias=(MIN(float_col)*2),relation=,
+  | type=Float NULL]
+  +-AttributeReference[id=11,name=,alias=COUNT(*),relation=,type=Long]
+==
+
+SELECT SUM(long_col+1)+2, (SUM(long_col+1)+2)*3
+FROM test;
+--
+[Optimized Logical Plan]
+TopLevelPlan
++-plan=Project
+| +-input=Aggregate
+| | +-input=TableReference[relation_name=Test,relation_alias=test]
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   type=VarChar(20) NULL]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
+| |   | +-AggregateFunction[function=SUM]
+| |   |   +-Add
+| |   |     +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |   |     +-Literal[value=1,type=Int]
+| |   +-Alias[id=8,name=,alias=$aggregate1,relation=$aggregate,type=Long NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-Add
+| |         +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |         +-Literal[value=1,type=Int]
+| +-project_list=
+|   +-Alias[id=7,name=,alias=(SUM((long_col+1))+2),relation=,type=Long NULL]
+|   | +-Add
+|   |   +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+|   |   | type=Long NULL]
+|   |   +-Literal[value=2,type=Int]
+|   +-Alias[id=9,name=,alias=((SUM((long_col+1))+2)*3),relation=,type=Long NULL]
+|     +-Multiply
+|       +-Add
+|       | +-AttributeReference[id=8,name=,alias=$aggregate1,relation=$aggregate,
+|       | | type=Long NULL]
+|       | +-Literal[value=2,type=Int]
+|       +-Literal[value=3,type=Int]
++-output_attributes=
+  +-AttributeReference[id=7,name=,alias=(SUM((long_col+1))+2),relation=,
+  | type=Long NULL]
+  +-AttributeReference[id=9,name=,alias=((SUM((long_col+1))+2)*3),relation=,
+    type=Long NULL]
+[Physical Plan]
+TopLevelPlan
++-plan=Selection
+| +-input=Aggregate
+| | +-input=TableReference[relation=Test,alias=test]
+| | | +-AttributeReference[id=0,name=int_col,relation=test,type=Int NULL]
+| | | +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| | | +-AttributeReference[id=2,name=float_col,relation=test,type=Float]
+| | | +-AttributeReference[id=3,name=double_col,relation=test,type=Double NULL]
+| | | +-AttributeReference[id=4,name=char_col,relation=test,type=Char(20)]
+| | | +-AttributeReference[id=5,name=vchar_col,relation=test,
+| | |   type=VarChar(20) NULL]
+| | +-grouping_expressions=
+| | | +-[]
+| | +-aggregate_expressions=
+| |   +-Alias[id=6,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
+| |     +-AggregateFunction[function=SUM]
+| |       +-Add
+| |         +-AttributeReference[id=1,name=long_col,relation=test,type=Long]
+| |         +-Literal[value=1,type=Int]
+| +-project_expressions=
+|   +-Alias[id=7,name=,alias=(SUM((long_col+1))+2),relation=,type=Long NULL]
+|   | +-CommonSubexpression[common_subexpression_id=10]
+|   |   +-Operand=Add
+|   |     +-AttributeReference[id=6,name=,alias=$aggregate0,relation=$aggregate,
+|   |     | type=Long NULL]
+|   |     +-Literal[value=2,type=Int]
+|   +-Alias[id=9,name=,alias=((SUM((long_col+1))+2)*3),relation=,type=Long NULL]
+|     +-Multiply
+|       +-CommonSubexpression[common_subexpression_id=10]
+|       | +-Operand=Add
+|       |   +-AttributeReference[id=6,name=,alias=$aggregate0,
+|       |   | relation=$aggregate,type=Long NULL]
+|       |   +-Literal[value=2,type=Int]
+|       +-Literal[value=3,type=Int]
++-output_attributes=
+  +-AttributeReference[id=7,name=,alias=(SUM((long_col+1))+2),relation=,
+  | type=Long NULL]
+  +-AttributeReference[id=9,name=,alias=((SUM((long_col+1))+2)*3),relation=,
+    type=Long NULL]
+==



[32/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
Refactor type system and operations.


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

Branch: refs/heads/new-op
Commit: 583724ea83731fc1fe7a25a39da568b590f64432
Parents: 0572f40
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Sat Mar 4 12:11:13 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Wed May 3 22:22:35 2017 -0500

----------------------------------------------------------------------
 expressions/CMakeLists.txt                      |    4 +-
 expressions/ExpressionFactories.cpp             |   61 +-
 expressions/Expressions.proto                   |   12 +-
 .../aggregation/AggregateFunctionAvg.cpp        |   21 +-
 .../aggregation/AggregateFunctionSum.cpp        |    9 +-
 .../aggregation/AggregationHandleAvg.cpp        |   30 +-
 .../aggregation/AggregationHandleSum.cpp        |   15 +-
 expressions/aggregation/CMakeLists.txt          |   12 +-
 expressions/predicate/CMakeLists.txt            |    1 -
 expressions/scalar/CMakeLists.txt               |   16 +-
 expressions/scalar/ScalarBinaryExpression.cpp   |  101 +-
 expressions/scalar/ScalarBinaryExpression.hpp   |   31 +-
 expressions/scalar/ScalarUnaryExpression.cpp    |  100 +-
 expressions/scalar/ScalarUnaryExpression.hpp    |   27 +-
 expressions/window_aggregation/CMakeLists.txt   |    9 +-
 .../WindowAggregateFunctionAvg.cpp              |   17 +-
 .../WindowAggregateFunctionSum.cpp              |    8 +-
 .../WindowAggregationHandle.cpp                 |    8 +-
 .../WindowAggregationHandle.hpp                 |    2 -
 .../WindowAggregationHandleAvg.cpp              |   47 +-
 parser/CMakeLists.txt                           |    6 -
 parser/ParseBasicExpressions.cpp                |  103 -
 parser/ParseBasicExpressions.hpp                |  291 +-
 parser/ParseExpression.hpp                      |    3 -
 parser/SqlLexer.lpp                             |    1 +
 parser/SqlParser.ypp                            |   93 +-
 parser/preprocessed/SqlLexer_gen.cpp            | 1507 ++++----
 parser/preprocessed/SqlLexer_gen.hpp            |  192 +-
 parser/preprocessed/SqlParser_gen.cpp           | 3348 +++++++++---------
 parser/preprocessed/SqlParser_gen.hpp           |  203 +-
 query_optimizer/LogicalGenerator.cpp            |    7 +-
 .../expressions/BinaryExpression.cpp            |   85 +-
 .../expressions/BinaryExpression.hpp            |   70 +-
 query_optimizer/expressions/CMakeLists.txt      |   23 +-
 query_optimizer/expressions/Cast.cpp            |   91 -
 query_optimizer/expressions/Cast.hpp            |  125 -
 query_optimizer/expressions/UnaryExpression.cpp |   51 +-
 query_optimizer/expressions/UnaryExpression.hpp |   43 +-
 query_optimizer/resolver/CMakeLists.txt         |    6 +-
 query_optimizer/resolver/Resolver.cpp           |  415 +--
 query_optimizer/resolver/Resolver.hpp           |    9 +-
 query_optimizer/rules/CMakeLists.txt            |    4 +-
 .../rules/ReuseAggregateExpressions.cpp         |   19 +-
 query_optimizer/rules/tests/CMakeLists.txt      |    2 -
 query_optimizer/strategy/tests/CMakeLists.txt   |    2 -
 query_optimizer/tests/CMakeLists.txt            |    2 -
 query_optimizer/tests/OptimizerTest.cpp         |    2 -
 relational_operators/CMakeLists.txt             |    2 -
 storage/CMakeLists.txt                          |    5 +-
 storage/PackedPayloadHashTable.cpp              |    4 +-
 storage/SMAIndexSubBlock.cpp                    |   10 +-
 types/AsciiStringSuperType.hpp                  |   78 +
 types/BoolType.cpp                              |   63 +
 types/BoolType.hpp                              |   73 +
 types/CMakeLists.txt                            |   89 +-
 types/CharType.cpp                              |   32 -
 types/CharType.hpp                              |   77 +-
 types/DateType.cpp                              |   19 -
 types/DateType.hpp                              |   67 +-
 types/DatetimeIntervalType.cpp                  |   19 -
 types/DatetimeIntervalType.hpp                  |   67 +-
 types/DatetimeLit.hpp                           |    4 +
 types/DatetimeType.cpp                          |   19 -
 types/DatetimeType.hpp                          |   68 +-
 types/DoubleType.cpp                            |   36 -
 types/DoubleType.hpp                            |   59 +-
 types/FloatType.cpp                             |   36 -
 types/FloatType.hpp                             |   59 +-
 types/IntType.cpp                               |   35 -
 types/IntType.hpp                               |   59 +-
 types/LongType.cpp                              |   36 -
 types/LongType.hpp                              |   59 +-
 types/NullCoercibilityCheckMacro.hpp            |    5 +-
 types/NullType.hpp                              |   49 +-
 types/NumericSuperType.hpp                      |   69 +-
 types/NumericTypeSafeCoercibility.hpp           |   61 +
 types/NumericTypeUnifier.hpp                    |  102 +-
 types/Type.cpp                                  |   68 +-
 types/Type.hpp                                  |   60 +-
 types/Type.proto                                |   36 +-
 types/TypeFactory.cpp                           |  133 +-
 types/TypeFactory.hpp                           |   20 +-
 types/TypeID.cpp                                |    1 +
 types/TypeID.hpp                                |   36 +-
 types/TypeIDSelectors.hpp                       |  152 +
 types/TypeRegistrar.hpp                         |  122 +
 types/TypeSynthesizer.hpp                       |  210 ++
 types/TypeUtil.hpp                              |   70 +
 types/TypedValue.cpp                            |   47 +-
 types/TypedValue.hpp                            |   48 +-
 types/TypedValue.proto                          |   21 +-
 types/VarCharType.cpp                           |   32 -
 types/VarCharType.hpp                           |   73 +-
 types/YearMonthIntervalType.cpp                 |   19 -
 types/YearMonthIntervalType.hpp                 |   67 +-
 types/containers/ColumnVector.cpp               |    4 +
 types/containers/ColumnVector.hpp               |   22 +-
 types/operations/CMakeLists.txt                 |   43 +
 types/operations/Operation.hpp                  |   26 +-
 types/operations/Operation.proto                |   59 +-
 types/operations/OperationFactory.cpp           |  357 ++
 types/operations/OperationFactory.hpp           |  203 ++
 types/operations/OperationSignature.cpp         |   91 +
 types/operations/OperationSignature.hpp         |  182 +
 types/operations/OperationUtil.hpp              |  334 ++
 .../binary_operations/AddBinaryOperation.cpp    |  418 ---
 .../ArithmeticBinaryFunctorOverloads.hpp        |  176 +
 .../ArithmeticBinaryOperation.hpp               |  404 ---
 .../ArithmeticBinaryOperations.hpp              |  182 +
 .../ArithmeticBinaryOperators.hpp               |  848 -----
 .../AsciiStringBinaryOperations.hpp             |  130 +
 .../binary_operations/BinaryOperation.cpp       |   29 -
 .../binary_operations/BinaryOperation.hpp       |  300 +-
 .../BinaryOperationFactory.cpp                  |   93 -
 .../BinaryOperationFactory.hpp                  |   79 -
 .../binary_operations/BinaryOperationID.cpp     |   40 -
 .../binary_operations/BinaryOperationID.hpp     |   62 -
 .../BinaryOperationWrapper.hpp                  |  629 ++++
 .../operations/binary_operations/CMakeLists.txt |  187 +-
 .../binary_operations/CMathBinaryOperations.hpp |   78 +
 .../binary_operations/DivideBinaryOperation.cpp |  391 --
 .../binary_operations/DivideBinaryOperation.hpp |   93 -
 .../binary_operations/ModuloBinaryOperation.cpp |  259 --
 .../binary_operations/ModuloBinaryOperation.hpp |   93 -
 .../MultiplyBinaryOperation.cpp                 |  410 ---
 .../SubtractBinaryOperation.cpp                 |  459 ---
 .../SubtractBinaryOperation.hpp                 |   93 -
 types/operations/comparisons/CMakeLists.txt     |    6 +-
 types/operations/comparisons/Comparison.hpp     |    6 +-
 types/operations/comparisons/ComparisonUtil.hpp |   30 +-
 .../comparisons/PatternMatchingComparison.cpp   |   18 +-
 .../ArithmeticUnaryOperations.cpp               |  145 -
 .../ArithmeticUnaryOperations.hpp               |   93 +-
 .../ArithmeticUnaryOperators.hpp                |  169 -
 .../AsciiStringUnaryOperations.hpp              |  122 +
 .../operations/unary_operations/CMakeLists.txt  |  141 +-
 .../unary_operations/CMathUnaryOperations.hpp   |  116 +
 .../unary_operations/CastOperation.cpp          |  298 ++
 .../unary_operations/CastOperation.hpp          |  154 +
 .../unary_operations/DateExtractOperation.cpp   |  596 +---
 .../unary_operations/DateExtractOperation.hpp   |  193 +-
 .../unary_operations/NumericCastOperation.hpp   |  313 --
 .../unary_operations/SubstringOperation.cpp     |  198 +-
 .../unary_operations/SubstringOperation.hpp     |  286 +-
 .../unary_operations/UnaryOperation.cpp         |   20 -
 .../unary_operations/UnaryOperation.hpp         |  180 +-
 .../unary_operations/UnaryOperationFactory.cpp  |  120 -
 .../unary_operations/UnaryOperationFactory.hpp  |   79 -
 .../unary_operations/UnaryOperationID.cpp       |   32 -
 .../unary_operations/UnaryOperationID.hpp       |   63 -
 .../unary_operations/UnaryOperationWrapper.hpp  |  250 ++
 utility/CMakeLists.txt                          |   12 +-
 utility/StringUtil.cpp                          |    6 +
 utility/StringUtil.hpp                          |   12 +-
 utility/TemplateUtil.hpp                        |  198 --
 utility/meta/CMakeLists.txt                     |   41 +
 utility/meta/Common.hpp                         |  143 +
 utility/meta/Dispatchers.hpp                    |  107 +
 utility/meta/TMP.hpp                            |   28 +
 utility/meta/TransitiveClosure.hpp              |   97 +
 utility/meta/TypeList.hpp                       |  124 +
 utility/meta/TypeListMetaFunctions.hpp          |  245 ++
 utility/tests/TemplateUtil_unittest.cpp         |  115 -
 163 files changed, 9387 insertions(+), 11953 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/CMakeLists.txt b/expressions/CMakeLists.txt
index 33606cd..3d2e156 100644
--- a/expressions/CMakeLists.txt
+++ b/expressions/CMakeLists.txt
@@ -56,9 +56,9 @@ target_link_libraries(quickstep_expressions_ExpressionFactories
                       quickstep_expressions_scalar_ScalarUnaryExpression
                       quickstep_types_TypeFactory
                       quickstep_types_TypedValue
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
+                      quickstep_types_operations_OperationFactory
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_comparisons_ComparisonFactory
-                      quickstep_types_operations_unaryoperations_UnaryOperationFactory
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_Expressions_proto
                       quickstep_types_Type_proto

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/ExpressionFactories.cpp
----------------------------------------------------------------------
diff --git a/expressions/ExpressionFactories.cpp b/expressions/ExpressionFactories.cpp
index 871db50..f8913ba 100644
--- a/expressions/ExpressionFactories.cpp
+++ b/expressions/ExpressionFactories.cpp
@@ -43,9 +43,9 @@
 #include "expressions/scalar/ScalarUnaryExpression.hpp"
 #include "types/TypeFactory.hpp"
 #include "types/TypedValue.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
+#include "types/operations/OperationFactory.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
-#include "types/operations/unary_operations/UnaryOperationFactory.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
@@ -168,17 +168,43 @@ Scalar* ScalarFactory::ReconstructFromProto(const serialization::Scalar &proto,
           proto.GetExtension(serialization::ScalarAttribute::attribute_id)));
     }
     case serialization::Scalar::UNARY_EXPRESSION: {
+      std::vector<TypedValue> static_arguments;
+      const int num_static_args =
+          proto.ExtensionSize(serialization::ScalarUnaryExpression::static_arguments);
+      for (int i = 0; i < num_static_args; ++i) {
+        static_arguments.emplace_back(
+            TypedValue::ReconstructFromProto(
+                proto.GetExtension(serialization::ScalarUnaryExpression::static_arguments, i)));
+      }
+      const OperationSignaturePtr op_signature =
+          OperationSignature::ReconstructFromProto(
+              proto.GetExtension(serialization::ScalarUnaryExpression::op_signature));
       return new ScalarUnaryExpression(
-          UnaryOperationFactory::ReconstructFromProto(
-              proto.GetExtension(serialization::ScalarUnaryExpression::operation)),
-          ReconstructFromProto(proto.GetExtension(serialization::ScalarUnaryExpression::operand), database));
+          op_signature,
+          OperationFactory::Instance().getUnaryOperation(op_signature),
+          ReconstructFromProto(proto.GetExtension(serialization::ScalarUnaryExpression::operand), database),
+          std::make_shared<std::vector<TypedValue>>(std::move(static_arguments)));
     }
     case serialization::Scalar::BINARY_EXPRESSION: {
+      std::vector<TypedValue> static_arguments;
+      const int num_static_args =
+          proto.ExtensionSize(serialization::ScalarBinaryExpression::static_arguments);
+      for (int i = 0; i < num_static_args; ++i) {
+        static_arguments.emplace_back(
+            TypedValue::ReconstructFromProto(
+                proto.GetExtension(serialization::ScalarBinaryExpression::static_arguments, i)));
+      }
+      const OperationSignaturePtr op_signature =
+          OperationSignature::ReconstructFromProto(
+              proto.GetExtension(serialization::ScalarBinaryExpression::op_signature));
       return new ScalarBinaryExpression(
-          BinaryOperationFactory::ReconstructFromProto(
-              proto.GetExtension(serialization::ScalarBinaryExpression::operation)),
-          ReconstructFromProto(proto.GetExtension(serialization::ScalarBinaryExpression::left_operand), database),
-          ReconstructFromProto(proto.GetExtension(serialization::ScalarBinaryExpression::right_operand), database));
+          op_signature,
+          OperationFactory::Instance().getBinaryOperation(op_signature),
+          ReconstructFromProto(
+              proto.GetExtension(serialization::ScalarBinaryExpression::left_operand), database),
+          ReconstructFromProto(
+              proto.GetExtension(serialization::ScalarBinaryExpression::right_operand), database),
+          std::make_shared<std::vector<TypedValue>>(std::move(static_arguments)));
     }
     case serialization::Scalar::CASE_EXPRESSION: {
       const Type &result_type = TypeFactory::ReconstructFromProto(
@@ -248,22 +274,13 @@ bool ScalarFactory::ProtoIsValid(const serialization::Scalar &proto,
       break;
     }
     case serialization::Scalar::UNARY_EXPRESSION: {
-      if (proto.HasExtension(serialization::ScalarUnaryExpression::operation)
-          && proto.HasExtension(serialization::ScalarUnaryExpression::operand)) {
-        return UnaryOperationFactory::ProtoIsValid(proto.GetExtension(serialization::ScalarUnaryExpression::operation))
-               && ProtoIsValid(proto.GetExtension(serialization::ScalarUnaryExpression::operand), database);
-      }
+      // TODO
+      return true;
       break;
     }
     case serialization::Scalar::BINARY_EXPRESSION: {
-      if (proto.HasExtension(serialization::ScalarBinaryExpression::operation)
-          && proto.HasExtension(serialization::ScalarBinaryExpression::left_operand)
-          && proto.HasExtension(serialization::ScalarBinaryExpression::right_operand)) {
-        return BinaryOperationFactory::ProtoIsValid(
-                   proto.GetExtension(serialization::ScalarBinaryExpression::operation))
-               && ProtoIsValid(proto.GetExtension(serialization::ScalarBinaryExpression::left_operand), database)
-               && ProtoIsValid(proto.GetExtension(serialization::ScalarBinaryExpression::right_operand), database);
-      }
+      // TODO
+      return true;
       break;
     }
     case serialization::Scalar::CASE_EXPRESSION: {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/Expressions.proto
----------------------------------------------------------------------
diff --git a/expressions/Expressions.proto b/expressions/Expressions.proto
index 8b4611e..3a9e673 100644
--- a/expressions/Expressions.proto
+++ b/expressions/Expressions.proto
@@ -103,16 +103,18 @@ message ScalarAttribute {
 
 message ScalarUnaryExpression {
   extend Scalar {
-    optional UnaryOperation operation = 96;
-    optional Scalar operand = 97;
+    optional OperationSignature op_signature = 97;
+    optional Scalar operand = 98;
+    repeated TypedValue static_arguments = 99;
   }
 }
 
 message ScalarBinaryExpression {
   extend Scalar {
-    optional BinaryOperation operation = 128;
-    optional Scalar left_operand = 129;
-    optional Scalar right_operand = 130;
+    optional OperationSignature op_signature = 129;
+    optional Scalar left_operand = 130;
+    optional Scalar right_operand = 131;
+    repeated TypedValue static_arguments = 132;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/aggregation/AggregateFunctionAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionAvg.cpp b/expressions/aggregation/AggregateFunctionAvg.cpp
index 040d7d9..b2b99c7 100644
--- a/expressions/aggregation/AggregateFunctionAvg.cpp
+++ b/expressions/aggregation/AggregateFunctionAvg.cpp
@@ -25,9 +25,8 @@
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
+#include "types/operations/OperationFactory.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 
 #include "glog/logging.h"
 
@@ -41,10 +40,15 @@ bool AggregateFunctionAvg::canApplyToTypes(
   }
 
   // Argument must be addable and divisible.
-  return BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-             .canApplyToTypes(*argument_types.front(), *argument_types.front())
-         && BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
-             .canApplyToTypes(*argument_types.front(), TypeFactory::GetType(kDouble));
+  const Type &type = *argument_types.front();
+  if (!OperationFactory::Instance()
+          .getBinaryOperation("+", {type.getTypeID(), type.getTypeID()})
+              ->canApplyTo(type, type)) {
+    return false;
+  }
+  return OperationFactory::Instance()
+      .getBinaryOperation("/", {type.getTypeID(), kDouble})
+          ->canApplyTo(type, TypeFactory::GetType(kDouble));
 }
 
 const Type* AggregateFunctionAvg::resultTypeForArgumentTypes(
@@ -67,8 +71,9 @@ const Type* AggregateFunctionAvg::resultTypeForArgumentTypes(
       break;
   }
 
-  return BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
-             .resultTypeForArgumentTypes(*sum_type, TypeFactory::GetType(kDouble));
+  return OperationFactory::Instance()
+      .getBinaryOperation("/", {sum_type->getTypeID(), kDouble})
+          ->getResultType(*sum_type, TypeFactory::GetType(kDouble));
 }
 
 AggregationHandle* AggregateFunctionAvg::createHandle(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/aggregation/AggregateFunctionSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregateFunctionSum.cpp b/expressions/aggregation/AggregateFunctionSum.cpp
index b62660f..11b33c0 100644
--- a/expressions/aggregation/AggregateFunctionSum.cpp
+++ b/expressions/aggregation/AggregateFunctionSum.cpp
@@ -25,9 +25,8 @@
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
+#include "types/operations/OperationFactory.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 
 #include "glog/logging.h"
 
@@ -41,8 +40,10 @@ bool AggregateFunctionSum::canApplyToTypes(
   }
 
   // Argument must be addable.
-  return BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-             .canApplyToTypes(*argument_types.front(), *argument_types.front());
+  const Type &type = *argument_types.front();
+  return OperationFactory::Instance()
+      .getBinaryOperation("+", {type.getTypeID(), type.getTypeID()})
+          ->canApplyTo(type, type);
 }
 
 const Type* AggregateFunctionSum::resultTypeForArgumentTypes(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/aggregation/AggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleAvg.cpp b/expressions/aggregation/AggregationHandleAvg.cpp
index 46bec1e..1324fd8 100644
--- a/expressions/aggregation/AggregationHandleAvg.cpp
+++ b/expressions/aggregation/AggregationHandleAvg.cpp
@@ -32,9 +32,8 @@
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
+#include "types/operations/OperationFactory.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 
 #include "glog/logging.h"
 
@@ -68,25 +67,24 @@ AggregationHandleAvg::AggregationHandleAvg(const Type &type)
 
   // Make operators to do arithmetic:
   // Add operator for summing argument values.
-  fast_add_operator_.reset(
-      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-          .makeUncheckedBinaryOperatorForTypes(sum_type, argument_type_));
+  fast_add_operator_.reset(OperationFactory::Instance()
+      .getBinaryOperation("+", {type_precision_id, argument_type_.getTypeID()})
+          ->makeUncheckedBinaryOperator(sum_type, argument_type_));
   // Add operator for merging states.
-  merge_add_operator_.reset(
-      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-          .makeUncheckedBinaryOperatorForTypes(sum_type, sum_type));
+  merge_add_operator_.reset(OperationFactory::Instance()
+      .getBinaryOperation("+", {type_precision_id, type_precision_id})
+          ->makeUncheckedBinaryOperator(sum_type, sum_type));
   // Divide operator for dividing sum by count to get final average.
-  divide_operator_.reset(
-      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
-          .makeUncheckedBinaryOperatorForTypes(sum_type,
-                                               TypeFactory::GetType(kDouble)));
+  divide_operator_.reset(OperationFactory::Instance()
+      .getBinaryOperation("/", {type_precision_id, kDouble})
+          ->makeUncheckedBinaryOperator(sum_type, TypeFactory::GetType(kDouble)));
 
   // Result is nullable, because AVG() over 0 values (or all NULL values) is
   // NULL.
-  result_type_ =
-      &(BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
-            .resultTypeForArgumentTypes(sum_type, TypeFactory::GetType(kDouble))
-            ->getNullableVersion());
+  result_type_ = &OperationFactory::Instance()
+      .getBinaryOperation("/", {type_precision_id, kDouble})
+          ->getResultType(sum_type, TypeFactory::GetType(kDouble))
+              ->getNullableVersion();
 }
 
 AggregationState* AggregationHandleAvg::accumulateValueAccessor(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/aggregation/AggregationHandleSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/aggregation/AggregationHandleSum.cpp b/expressions/aggregation/AggregationHandleSum.cpp
index 9f5f220..c7ee776 100644
--- a/expressions/aggregation/AggregationHandleSum.cpp
+++ b/expressions/aggregation/AggregationHandleSum.cpp
@@ -32,9 +32,8 @@
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
+#include "types/operations/OperationFactory.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 
 #include "glog/logging.h"
 
@@ -67,13 +66,13 @@ AggregationHandleSum::AggregationHandleSum(const Type &type)
 
   // Make operators to do arithmetic:
   // Add operator for summing argument values.
-  fast_operator_.reset(
-      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-          .makeUncheckedBinaryOperatorForTypes(sum_type, argument_type_));
+  fast_operator_.reset(OperationFactory::Instance()
+      .getBinaryOperation("+", {type_precision_id, argument_type_.getTypeID()})
+          ->makeUncheckedBinaryOperator(sum_type, argument_type_));
   // Add operator for merging states.
-  merge_operator_.reset(
-      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-          .makeUncheckedBinaryOperatorForTypes(sum_type, sum_type));
+  merge_operator_.reset(OperationFactory::Instance()
+      .getBinaryOperation("+", {type_precision_id, type_precision_id})
+          ->makeUncheckedBinaryOperator(sum_type, sum_type));
 
   // Result is nullable, because SUM() over 0 values (or all NULL values) is
   // NULL.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/aggregation/CMakeLists.txt b/expressions/aggregation/CMakeLists.txt
index 4220a8d..29a58c6 100644
--- a/expressions/aggregation/CMakeLists.txt
+++ b/expressions/aggregation/CMakeLists.txt
@@ -84,9 +84,8 @@ target_link_libraries(quickstep_expressions_aggregation_AggregateFunctionAvg
                       quickstep_types_Type
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
+                      quickstep_types_operations_OperationFactory
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregateFunctionCount
                       glog
@@ -135,9 +134,8 @@ target_link_libraries(quickstep_expressions_aggregation_AggregateFunctionSum
                       quickstep_types_Type
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
+                      quickstep_types_operations_OperationFactory
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationConcreteHandle
                       glog
@@ -170,9 +168,8 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleAvg
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
+                      quickstep_types_operations_OperationFactory
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_aggregation_AggregationHandleCount
                       glog
@@ -228,9 +225,8 @@ target_link_libraries(quickstep_expressions_aggregation_AggregationHandleSum
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
+                      quickstep_types_operations_OperationFactory
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_utility_Macros)
 
 # Submodule all-in-one library:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/predicate/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/predicate/CMakeLists.txt b/expressions/predicate/CMakeLists.txt
index 04abfc7..def0bc5 100644
--- a/expressions/predicate/CMakeLists.txt
+++ b/expressions/predicate/CMakeLists.txt
@@ -150,7 +150,6 @@ target_link_libraries(Predicate_unittest
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/scalar/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/scalar/CMakeLists.txt b/expressions/scalar/CMakeLists.txt
index 6b52231..a59635b 100644
--- a/expressions/scalar/CMakeLists.txt
+++ b/expressions/scalar/CMakeLists.txt
@@ -68,9 +68,9 @@ target_link_libraries(quickstep_expressions_scalar_ScalarBinaryExpression
                       quickstep_types_TypeErrors
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_scalar_ScalarCaseExpression
                       quickstep_catalog_CatalogTypedefs
@@ -118,9 +118,9 @@ target_link_libraries(quickstep_expressions_scalar_ScalarUnaryExpression
                       quickstep_types_TypeErrors
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_Macros)
 
 # Submodule all-in-one library:
@@ -161,12 +161,8 @@ target_link_libraries(ScalarCaseExpression_unittest
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorsValueAccessor
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_types_operations_comparisons_ComparisonFactory
-                      quickstep_types_operations_comparisons_ComparisonID
-                      quickstep_types_operations_unaryoperations_UnaryOperationFactory
-                      quickstep_types_operations_unaryoperations_UnaryOperationID)
+                      quickstep_types_operations_comparisons_ComparisonID)
 add_test(ScalarCaseExpression_unittest ScalarCaseExpression_unittest)
 
 add_executable(Scalar_unittest "${CMAKE_CURRENT_SOURCE_DIR}/tests/Scalar_unittest.cpp")
@@ -189,11 +185,7 @@ target_link_libraries(Scalar_unittest
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_types_operations_unaryoperations_NumericCastOperation
+                      quickstep_types_operations_unaryoperations_CastOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationFactory
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_Macros)
 add_test(Scalar_unittest Scalar_unittest)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/scalar/ScalarBinaryExpression.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarBinaryExpression.cpp b/expressions/scalar/ScalarBinaryExpression.cpp
index b3568f8..2f0a0d4 100644
--- a/expressions/scalar/ScalarBinaryExpression.cpp
+++ b/expressions/scalar/ScalarBinaryExpression.cpp
@@ -34,43 +34,71 @@
 #include "types/containers/ColumnVector.hpp"
 #include "types/operations/Operation.pb.h"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
-ScalarBinaryExpression::ScalarBinaryExpression(const BinaryOperation &operation,
-                                               Scalar *left_operand,
-                                               Scalar *right_operand)
-    : Scalar(*operation.resultTypeForArgumentTypes(left_operand->getType(),
-                                                   right_operand->getType())),
+ScalarBinaryExpression::ScalarBinaryExpression(
+    const OperationSignaturePtr &op_signature,
+    const BinaryOperationPtr &operation,
+    Scalar *left_operand,
+    Scalar *right_operand,
+    const std::shared_ptr<const std::vector<TypedValue>> &static_arguments)
+    : Scalar(*operation->getResultType(left_operand->getType(),
+                                       right_operand->getType(),
+                                       *static_arguments)),
+      op_signature_(op_signature),
       operation_(operation),
       left_operand_(left_operand),
-      right_operand_(right_operand) {
-  initHelper(false);
+      right_operand_(right_operand),
+      static_arguments_(static_arguments) {
+  DCHECK(operation_->canApplyTo(left_operand_->getType(),
+                                right_operand_->getType(),
+                                *static_arguments));
+  fast_operator_.reset(
+      operation_->makeUncheckedBinaryOperator(left_operand_->getType(),
+                                              right_operand_->getType(),
+                                              *static_arguments));
+  if (left_operand_->hasStaticValue() && right_operand_->hasStaticValue()) {
+    static_value_.reset(new TypedValue(
+        fast_operator_->applyToTypedValues(left_operand_->getStaticValue(),
+                                           right_operand_->getStaticValue())));
+  }
 }
 
 serialization::Scalar ScalarBinaryExpression::getProto() const {
   serialization::Scalar proto;
   proto.set_data_source(serialization::Scalar::BINARY_EXPRESSION);
-  proto.MutableExtension(serialization::ScalarBinaryExpression::operation)->CopyFrom(operation_.getProto());
-  proto.MutableExtension(serialization::ScalarBinaryExpression::left_operand)->CopyFrom(left_operand_->getProto());
-  proto.MutableExtension(serialization::ScalarBinaryExpression::right_operand)->CopyFrom(right_operand_->getProto());
-
+  proto.MutableExtension(
+      serialization::ScalarBinaryExpression::op_signature)->CopyFrom(
+          op_signature_->getProto());
+  proto.MutableExtension(
+      serialization::ScalarBinaryExpression::left_operand)->CopyFrom(
+          left_operand_->getProto());
+  proto.MutableExtension(
+      serialization::ScalarBinaryExpression::right_operand)->CopyFrom(
+          right_operand_->getProto());
+  for (const TypedValue &value : *static_arguments_) {
+    proto.AddExtension(
+        serialization::ScalarUnaryExpression::static_arguments)->CopyFrom(
+            value.getProto());
+  }
   return proto;
 }
 
 Scalar* ScalarBinaryExpression::clone() const {
-  return new ScalarBinaryExpression(operation_,
+  return new ScalarBinaryExpression(op_signature_,
+                                    operation_,
                                     left_operand_->clone(),
-                                    right_operand_->clone());
+                                    right_operand_->clone(),
+                                    static_arguments_);
 }
 
 TypedValue ScalarBinaryExpression::getValueForSingleTuple(const ValueAccessor &accessor,
                                                           const tuple_id tuple) const {
   if (fast_operator_.get() == nullptr) {
-    return static_value_.makeReferenceToThis();
+    return static_value_->makeReferenceToThis();
   } else {
     return fast_operator_->applyToTypedValues(left_operand_->getValueForSingleTuple(accessor, tuple),
                                               right_operand_->getValueForSingleTuple(accessor, tuple));
@@ -85,7 +113,7 @@ TypedValue ScalarBinaryExpression::getValueForJoinedTuples(
     const relation_id right_relation_id,
     const tuple_id right_tuple_id) const {
   if (fast_operator_.get() == nullptr) {
-    return static_value_.makeReferenceToThis();
+    return static_value_->makeReferenceToThis();
   } else {
     return fast_operator_->applyToTypedValues(
         left_operand_->getValueForJoinedTuples(left_accessor,
@@ -110,7 +138,7 @@ ColumnVectorPtr ScalarBinaryExpression::getAllValues(
   if (fast_operator_.get() == nullptr) {
     return ColumnVectorPtr(
         ColumnVector::MakeVectorOfValue(getType(),
-                                        static_value_,
+                                        *static_value_,
                                         accessor->getNumTuplesVirtual()));
   } else {
     // NOTE(chasseur): We don't check if BOTH operands have a static value,
@@ -208,7 +236,7 @@ ColumnVectorPtr ScalarBinaryExpression::getAllValuesForJoin(
   if (fast_operator_.get() == nullptr) {
     return ColumnVectorPtr(
         ColumnVector::MakeVectorOfValue(getType(),
-                                        static_value_,
+                                        *static_value_,
                                         joined_tuple_ids.size()));
   } else {
     if (left_operand_->hasStaticValue()) {
@@ -380,31 +408,6 @@ ColumnVectorPtr ScalarBinaryExpression::getAllValuesForJoin(
   }
 }
 
-void ScalarBinaryExpression::initHelper(bool own_children) {
-  if (operation_.canApplyToTypes(left_operand_->getType(), right_operand_->getType())) {
-    if (left_operand_->hasStaticValue() && right_operand_->hasStaticValue()) {
-      static_value_ = operation_.applyToChecked(left_operand_->getStaticValue(),
-                                                left_operand_->getType(),
-                                                right_operand_->getStaticValue(),
-                                                right_operand_->getType());
-    } else {
-      fast_operator_.reset(operation_.makeUncheckedBinaryOperatorForTypes(left_operand_->getType(),
-                                                                           right_operand_->getType()));
-    }
-  } else {
-    const Type &left_operand_type = left_operand_->getType();
-    const Type &right_operand_type = right_operand_->getType();
-    if (!own_children) {
-      left_operand_.release();
-      right_operand_.release();
-    }
-    throw OperationInapplicableToType(operation_.getName(),
-                                      2,
-                                      left_operand_type.getName().c_str(),
-                                      right_operand_type.getName().c_str());
-  }
-}
-
 void ScalarBinaryExpression::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,
@@ -419,19 +422,17 @@ void ScalarBinaryExpression::getFieldStringItems(
                               container_child_field_names,
                               container_child_fields);
 
-  if (fast_operator_ == nullptr) {
+  if (static_value_ != nullptr) {
     inline_field_names->emplace_back("static_value");
-    if (static_value_.isNull()) {
+    if (static_value_->isNull()) {
       inline_field_values->emplace_back("NULL");
     } else {
-      inline_field_values->emplace_back(type_.printValueToString(static_value_));
+      inline_field_values->emplace_back(type_.printValueToString(*static_value_));
     }
   }
 
-  inline_field_names->emplace_back("operation");
-  inline_field_values->emplace_back(
-      kBinaryOperationNames[static_cast<std::underlying_type<BinaryOperationID>::type>(
-          operation_.getBinaryOperationID())]);
+  inline_field_names->emplace_back("op_signature");
+  inline_field_values->emplace_back(op_signature_->toString());
 
   non_container_child_field_names->emplace_back("left_operand");
   non_container_child_fields->emplace_back(left_operand_.get());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/scalar/ScalarBinaryExpression.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarBinaryExpression.hpp b/expressions/scalar/ScalarBinaryExpression.hpp
index 4ac1f62..a27c820 100644
--- a/expressions/scalar/ScalarBinaryExpression.hpp
+++ b/expressions/scalar/ScalarBinaryExpression.hpp
@@ -31,6 +31,7 @@
 #include "storage/StorageBlockInfo.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVector.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
 #include "utility/Macros.hpp"
 
@@ -52,18 +53,11 @@ struct SubBlocksReference;
  **/
 class ScalarBinaryExpression : public Scalar {
  public:
-  /**
-   * @brief Constructor.
-   *
-   * @param operation The binary operation to be performed.
-   * @param left_operand The left argument of the operation, which this
-   *        ScalarBinaryExpression takes ownership of.
-   * @param right_operand The right argument of the operation, which this
-   *        ScalarBinaryExpression takes ownership of.
-   **/
-  ScalarBinaryExpression(const BinaryOperation &operation,
+  ScalarBinaryExpression(const OperationSignaturePtr &op_signature,
+                         const BinaryOperationPtr &operation,
                          Scalar *left_operand,
-                         Scalar *right_operand);
+                         Scalar *right_operand,
+                         const std::shared_ptr<const std::vector<TypedValue>> &static_arguments);
 
   /**
    * @brief Destructor
@@ -91,12 +85,12 @@ class ScalarBinaryExpression : public Scalar {
       const tuple_id right_tuple_id) const override;
 
   bool hasStaticValue() const override {
-    return fast_operator_.get() == nullptr;
+    return static_value_ != nullptr;
   }
 
   const TypedValue& getStaticValue() const override {
     DCHECK(hasStaticValue());
-    return static_value_;
+    return *static_value_;
   }
 
   ColumnVectorPtr getAllValues(ValueAccessor *accessor,
@@ -121,13 +115,14 @@ class ScalarBinaryExpression : public Scalar {
       std::vector<std::vector<const Expression*>> *container_child_fields) const override;
 
  private:
-  void initHelper(bool own_children);
+  const OperationSignaturePtr op_signature_;
+  const BinaryOperationPtr operation_;
 
-  const BinaryOperation &operation_;
+  const std::unique_ptr<Scalar> left_operand_;
+  const std::unique_ptr<Scalar> right_operand_;
+  const std::shared_ptr<const std::vector<TypedValue>> static_arguments_;
 
-  std::unique_ptr<Scalar> left_operand_;
-  std::unique_ptr<Scalar> right_operand_;
-  TypedValue static_value_;
+  std::unique_ptr<TypedValue> static_value_;
   std::unique_ptr<UncheckedBinaryOperator> fast_operator_;
 
   friend class PredicateTest;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/scalar/ScalarUnaryExpression.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarUnaryExpression.cpp b/expressions/scalar/ScalarUnaryExpression.cpp
index c51e38f..c2fd931 100644
--- a/expressions/scalar/ScalarUnaryExpression.cpp
+++ b/expressions/scalar/ScalarUnaryExpression.cpp
@@ -34,41 +34,62 @@
 #include "types/containers/ColumnVector.hpp"
 #include "types/operations/Operation.pb.h"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
 #include "glog/logging.h"
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
 
 namespace quickstep {
 
 struct SubBlocksReference;
 
-ScalarUnaryExpression::ScalarUnaryExpression(const UnaryOperation &operation,
-                                             Scalar *operand)
-    : Scalar(*operation.resultTypeForArgumentType(operand->getType())),
+ScalarUnaryExpression::ScalarUnaryExpression(
+    const OperationSignaturePtr &op_signature,
+    const UnaryOperationPtr &operation,
+    Scalar *operand,
+    const std::shared_ptr<const std::vector<TypedValue>> &static_arguments)
+    : Scalar(*operation->getResultType(operand->getType(), *static_arguments)),
+      op_signature_(op_signature),
       operation_(operation),
-      operand_(operand) {
-  initHelper(false);
+      operand_(operand),
+      static_arguments_(static_arguments) {
+  DCHECK(operation_->canApplyTo(operand_->getType(), *static_arguments_));
+
+  fast_operator_.reset(
+      operation_->makeUncheckedUnaryOperator(operand_->getType(),
+                                             *static_arguments_));
+  if (operand_->hasStaticValue()) {
+    static_value_.reset(new TypedValue(
+        fast_operator_->applyToTypedValue(operand_->getStaticValue())));
+  }
 }
 
 serialization::Scalar ScalarUnaryExpression::getProto() const {
   serialization::Scalar proto;
   proto.set_data_source(serialization::Scalar::UNARY_EXPRESSION);
-  proto.MutableExtension(serialization::ScalarUnaryExpression::operation)->CopyFrom(operation_.getProto());
-  proto.MutableExtension(serialization::ScalarUnaryExpression::operand)->CopyFrom(operand_->getProto());
-
+  proto.MutableExtension(
+      serialization::ScalarUnaryExpression::op_signature)->CopyFrom(
+          op_signature_->getProto());
+  proto.MutableExtension(
+      serialization::ScalarUnaryExpression::operand)->CopyFrom(
+          operand_->getProto());
+  for (const TypedValue &value : *static_arguments_) {
+    proto.AddExtension(
+        serialization::ScalarUnaryExpression::static_arguments)->CopyFrom(
+            value.getProto());
+  }
   return proto;
 }
 
 Scalar* ScalarUnaryExpression::clone() const {
-  return new ScalarUnaryExpression(operation_, operand_->clone());
+  return new ScalarUnaryExpression(op_signature_,
+                                   operation_,
+                                   operand_->clone(),
+                                   static_arguments_);
 }
 
 TypedValue ScalarUnaryExpression::getValueForSingleTuple(const ValueAccessor &accessor,
                                                          const tuple_id tuple) const {
   if (fast_operator_.get() == nullptr) {
-    return static_value_.makeReferenceToThis();
+    return static_value_->makeReferenceToThis();
   } else {
     return fast_operator_->applyToTypedValue(operand_->getValueForSingleTuple(accessor, tuple));
   }
@@ -82,7 +103,7 @@ TypedValue ScalarUnaryExpression::getValueForJoinedTuples(
     const relation_id right_relation_id,
     const tuple_id right_tuple_id) const {
   if (fast_operator_.get() == nullptr) {
-    return static_value_.makeReferenceToThis();
+    return static_value_->makeReferenceToThis();
   } else {
     return fast_operator_->applyToTypedValue(operand_->getValueForJoinedTuples(left_accessor,
                                                                                left_relation_id,
@@ -100,7 +121,7 @@ ColumnVectorPtr ScalarUnaryExpression::getAllValues(
   if (fast_operator_.get() == nullptr) {
     return ColumnVectorPtr(
         ColumnVector::MakeVectorOfValue(getType(),
-                                        static_value_,
+                                        *static_value_,
                                         accessor->getNumTuplesVirtual()));
   } else {
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -128,27 +149,9 @@ ColumnVectorPtr ScalarUnaryExpression::getAllValuesForJoin(
   if (fast_operator_.get() == nullptr) {
     return ColumnVectorPtr(
         ColumnVector::MakeVectorOfValue(getType(),
-                                        static_value_,
+                                        *static_value_,
                                         joined_tuple_ids.size()));
   } else {
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-    const attribute_id operand_attr_id = operand_->getAttributeIdForValueAccessor();
-    if (operand_attr_id != -1) {
-      const relation_id operand_relation_id = operand_->getRelationIdForValueAccessor();
-      DCHECK_NE(operand_relation_id, -1);
-      DCHECK((operand_relation_id == left_relation_id)
-             || (operand_relation_id == right_relation_id));
-      const bool using_left_relation = (operand_relation_id == left_relation_id);
-      ValueAccessor *operand_accessor = using_left_relation ? left_accessor
-                                                            : right_accessor;
-      return ColumnVectorPtr(
-          fast_operator_->applyToValueAccessorForJoin(operand_accessor,
-                                                      using_left_relation,
-                                                      operand_attr_id,
-                                                      joined_tuple_ids));
-    }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
     ColumnVectorPtr operand_result(
         operand_->getAllValuesForJoin(left_relation_id,
                                       left_accessor,
@@ -161,23 +164,6 @@ ColumnVectorPtr ScalarUnaryExpression::getAllValuesForJoin(
   }
 }
 
-void ScalarUnaryExpression::initHelper(bool own_children) {
-  if (operation_.canApplyToType(operand_->getType())) {
-    if (operand_->hasStaticValue()) {
-      static_value_ = operation_.applyToChecked(operand_->getStaticValue(),
-                                                operand_->getType());
-    } else {
-      fast_operator_.reset(operation_.makeUncheckedUnaryOperatorForType(operand_->getType()));
-    }
-  } else {
-    const Type &operand_type = operand_->getType();
-    if (!own_children) {
-      operand_.release();
-    }
-    throw OperationInapplicableToType(operation_.getName(), 1, operand_type.getName().c_str());
-  }
-}
-
 void ScalarUnaryExpression::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,
@@ -192,19 +178,17 @@ void ScalarUnaryExpression::getFieldStringItems(
                               container_child_field_names,
                               container_child_fields);
 
-  if (fast_operator_ == nullptr) {
+  if (static_value_ != nullptr) {
     inline_field_names->emplace_back("static_value");
-    if (static_value_.isNull()) {
+    if (static_value_->isNull()) {
       inline_field_values->emplace_back("NULL");
     } else {
-      inline_field_values->emplace_back(type_.printValueToString(static_value_));
+      inline_field_values->emplace_back(type_.printValueToString(*static_value_));
     }
   }
 
-  inline_field_names->emplace_back("operation");
-  inline_field_values->emplace_back(
-      kUnaryOperationNames[static_cast<std::underlying_type<UnaryOperationID>::type>(
-          operation_.getUnaryOperationID())]);
+  inline_field_names->emplace_back("op_signature");
+  inline_field_values->emplace_back(op_signature_->toString());
 
   non_container_child_field_names->emplace_back("operand");
   non_container_child_fields->emplace_back(operand_.get());

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/scalar/ScalarUnaryExpression.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarUnaryExpression.hpp b/expressions/scalar/ScalarUnaryExpression.hpp
index 52edea7..8dc4c30 100644
--- a/expressions/scalar/ScalarUnaryExpression.hpp
+++ b/expressions/scalar/ScalarUnaryExpression.hpp
@@ -30,6 +30,8 @@
 #include "expressions/scalar/Scalar.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
 #include "utility/Macros.hpp"
 
@@ -51,14 +53,10 @@ struct SubBlocksReference;
  **/
 class ScalarUnaryExpression : public Scalar {
  public:
-  /**
-   * @brief Constructor.
-   *
-   * @param operation The unary operation to be performed.
-   * @param operand The argument of the operation, which this
-   *        ScalarUnaryExpression takes ownership of.
-   **/
-  ScalarUnaryExpression(const UnaryOperation &operation, Scalar *operand);
+  ScalarUnaryExpression(const OperationSignaturePtr &op_signature,
+                        const UnaryOperationPtr &operation,
+                        Scalar *operand,
+                        const std::shared_ptr<const std::vector<TypedValue>> &static_arguments);
 
   /**
    * @brief Destructor.
@@ -86,12 +84,12 @@ class ScalarUnaryExpression : public Scalar {
       const tuple_id right_tuple_id) const override;
 
   bool hasStaticValue() const override {
-    return fast_operator_.get() == nullptr;
+    return static_value_ != nullptr;
   }
 
   const TypedValue& getStaticValue() const override {
     DCHECK(hasStaticValue());
-    return static_value_;
+    return *static_value_;
   }
 
   ColumnVectorPtr getAllValues(ValueAccessor *accessor,
@@ -116,12 +114,13 @@ class ScalarUnaryExpression : public Scalar {
       std::vector<std::vector<const Expression*>> *container_child_fields) const override;
 
  private:
-  void initHelper(bool own_children);
+  const OperationSignaturePtr op_signature_;
+  const UnaryOperationPtr operation_;
 
-  const UnaryOperation &operation_;
+  const std::unique_ptr<Scalar> operand_;
+  const std::shared_ptr<const std::vector<TypedValue>> static_arguments_;
 
-  std::unique_ptr<Scalar> operand_;
-  TypedValue static_value_;
+  std::unique_ptr<TypedValue> static_value_;
   std::unique_ptr<UncheckedUnaryOperator> fast_operator_;
 
   friend class PredicateTest;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/window_aggregation/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/CMakeLists.txt b/expressions/window_aggregation/CMakeLists.txt
index b33a401..1b5b743 100644
--- a/expressions/window_aggregation/CMakeLists.txt
+++ b/expressions/window_aggregation/CMakeLists.txt
@@ -72,8 +72,6 @@ target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregateFun
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregateFunctionCount
                       glog
@@ -123,8 +121,6 @@ target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregateFun
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregationHandle
                       glog
@@ -138,9 +134,8 @@ target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregationH
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_types_operations_OperationFactory
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
@@ -156,8 +151,6 @@ target_link_libraries(quickstep_expressions_windowaggregation_WindowAggregationH
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVectorsValueAccessor
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_utility_Macros)
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp b/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp
index 20c296b..a70a8bc 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionAvg.cpp
@@ -26,8 +26,6 @@
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 
 #include "glog/logging.h"
 
@@ -41,10 +39,11 @@ bool WindowAggregateFunctionAvg::canApplyToTypes(
   }
 
   // Argument must be addable and divisible.
-  return BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-             .canApplyToTypes(*argument_types.front(), *argument_types.front()) &&
-         BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
-             .canApplyToTypes(*argument_types.front(), TypeFactory::GetType(kDouble));
+//  return BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
+//             .canApplyTo(*argument_types.front(), *argument_types.front()) &&
+//         BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
+//             .canApplyTo(*argument_types.front(), TypeFactory::GetType(kDouble));
+  return false;
 }
 
 const Type* WindowAggregateFunctionAvg::resultTypeForArgumentTypes(
@@ -67,8 +66,10 @@ const Type* WindowAggregateFunctionAvg::resultTypeForArgumentTypes(
       break;
   }
 
-  return BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
-             .resultTypeForArgumentTypes(*sum_type, TypeFactory::GetType(kDouble));
+// TODO
+//  return BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
+//             .getResultType(*sum_type, TypeFactory::GetType(kDouble));
+  return nullptr;
 }
 
 WindowAggregationHandle* WindowAggregateFunctionAvg::createHandle(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/window_aggregation/WindowAggregateFunctionSum.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregateFunctionSum.cpp b/expressions/window_aggregation/WindowAggregateFunctionSum.cpp
index 14c51d8..e383c63 100644
--- a/expressions/window_aggregation/WindowAggregateFunctionSum.cpp
+++ b/expressions/window_aggregation/WindowAggregateFunctionSum.cpp
@@ -26,8 +26,6 @@
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 
 #include "glog/logging.h"
 
@@ -41,8 +39,10 @@ bool WindowAggregateFunctionSum::canApplyToTypes(
   }
 
   // Argument must be addable.
-  return BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-             .canApplyToTypes(*argument_types.front(), *argument_types.front());
+// TODO
+//  return BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
+//             .canApplyTo(*argument_types.front(), *argument_types.front());
+  return false;
 }
 
 const Type* WindowAggregateFunctionSum::resultTypeForArgumentTypes(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/window_aggregation/WindowAggregationHandle.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregationHandle.cpp b/expressions/window_aggregation/WindowAggregationHandle.cpp
index f26656d..7621726 100644
--- a/expressions/window_aggregation/WindowAggregationHandle.cpp
+++ b/expressions/window_aggregation/WindowAggregationHandle.cpp
@@ -30,9 +30,8 @@
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "types/operations/OperationFactory.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
 
 #include "glog/logging.h"
@@ -83,8 +82,9 @@ WindowAggregationHandle::WindowAggregationHandle(
         TypeFactory::GetUnifyingType(*first_order_key_type, long_type);
 
     range_add_operator_.reset(
-        BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-            .makeUncheckedBinaryOperatorForTypes(*first_order_key_type, long_type));
+        OperationFactory::Instance().getBinaryOperation(
+            "+", {first_order_key_type->getTypeID(), kLong}, 0)
+                ->makeUncheckedBinaryOperator(*first_order_key_type, long_type));
     range_comparator_.reset(
         ComparisonFactory::GetComparison(ComparisonID::kLessOrEqual)
             .makeUncheckedComparatorForTypes(*range_compare_type_, *range_compare_type_));

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/window_aggregation/WindowAggregationHandle.hpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregationHandle.hpp b/expressions/window_aggregation/WindowAggregationHandle.hpp
index 3569123..0d5f5ca 100644
--- a/expressions/window_aggregation/WindowAggregationHandle.hpp
+++ b/expressions/window_aggregation/WindowAggregationHandle.hpp
@@ -37,8 +37,6 @@
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/expressions/window_aggregation/WindowAggregationHandleAvg.cpp
----------------------------------------------------------------------
diff --git a/expressions/window_aggregation/WindowAggregationHandleAvg.cpp b/expressions/window_aggregation/WindowAggregationHandleAvg.cpp
index b1c6e3b..3539d03 100644
--- a/expressions/window_aggregation/WindowAggregationHandleAvg.cpp
+++ b/expressions/window_aggregation/WindowAggregationHandleAvg.cpp
@@ -32,8 +32,6 @@
 #include "types/TypedValue.hpp"
 #include "types/containers/ColumnVectorsValueAccessor.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
 
 #include "glog/logging.h"
@@ -71,28 +69,29 @@ WindowAggregationHandleAvg::WindowAggregationHandleAvg(
 
   sum_type_ = &(TypeFactory::GetType(type_id));
 
-  // Result is nullable, because AVG() over 0 values (or all NULL values) is
-  // NULL.
-  result_type_
-      = &(BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
-              .resultTypeForArgumentTypes(*sum_type_, TypeFactory::GetType(kDouble))
-                  ->getNullableVersion());
-
-  // Make operators to do arithmetic:
-  // Add operator for summing argument values.
-  fast_add_operator_.reset(
-      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
-          .makeUncheckedBinaryOperatorForTypes(*sum_type_, *argument_type));
-
-  // Subtract operator for dropping argument values off the window.
-  fast_subtract_operator_.reset(
-      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kSubtract)
-          .makeUncheckedBinaryOperatorForTypes(*sum_type_, *argument_type));
-
-  // Divide operator for dividing sum by count to get final average.
-  divide_operator_.reset(
-      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
-          .makeUncheckedBinaryOperatorForTypes(*sum_type_, TypeFactory::GetType(kDouble)));
+// TODO
+//  // Result is nullable, because AVG() over 0 values (or all NULL values) is
+//  // NULL.
+//  result_type_
+//      = &(BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
+//              .getResultType(*sum_type_, TypeFactory::GetType(kDouble))
+//                  ->getNullableVersion());
+//
+//  // Make operators to do arithmetic:
+//  // Add operator for summing argument values.
+//  fast_add_operator_.reset(
+//      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kAdd)
+//          .makeUncheckedBinaryOperator(*sum_type_, *argument_type));
+//
+//  // Subtract operator for dropping argument values off the window.
+//  fast_subtract_operator_.reset(
+//      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kSubtract)
+//          .makeUncheckedBinaryOperator(*sum_type_, *argument_type));
+//
+//  // Divide operator for dividing sum by count to get final average.
+//  divide_operator_.reset(
+//      BinaryOperationFactory::GetBinaryOperation(BinaryOperationID::kDivide)
+//          .makeUncheckedBinaryOperator(*sum_type_, TypeFactory::GetType(kDouble)));
 }
 
 ColumnVector* WindowAggregationHandleAvg::calculate(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index b3ddf30..fb7a3cd 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -366,15 +366,9 @@ target_link_libraries(quickstep_parser_SqlParser
                       quickstep_types_Type
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
-                      quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
-                      quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationFactory
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_PtrList
                       quickstep_utility_PtrVector)
 target_link_libraries(quickstep_parser_SqlParserWrapper

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/parser/ParseBasicExpressions.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseBasicExpressions.cpp b/parser/ParseBasicExpressions.cpp
index b0b1247..580cd09 100644
--- a/parser/ParseBasicExpressions.cpp
+++ b/parser/ParseBasicExpressions.cpp
@@ -68,54 +68,6 @@ void ParseAttribute::getFieldStringItems(
   }
 }
 
-std::string ParseUnaryExpression::getName() const {
-  return op_.getName();
-}
-
-string ParseUnaryExpression::generateName() const {
-  string name(op_.getShortName());
-  name.append(operand_->generateName());
-  return name;
-}
-
-void ParseUnaryExpression::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<const ParseTreeNode*> *non_container_child_fields,
-    std::vector<std::string> *container_child_field_names,
-    std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const {
-  non_container_child_field_names->push_back("");
-  non_container_child_fields->push_back(operand_.get());
-}
-
-std::string ParseBinaryExpression::getName() const {
-  return op_.getName();
-}
-
-string ParseBinaryExpression::generateName() const {
-  string name("(");
-  name.append(left_operand_->generateName());
-  name.append(op_.getShortName());
-  name.append(right_operand_->generateName());
-  name.push_back(')');
-  return name;
-}
-
-void ParseBinaryExpression::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<const ParseTreeNode*> *non_container_child_fields,
-    std::vector<std::string> *container_child_field_names,
-    std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const  {
-  non_container_child_field_names->push_back("left_operand");
-  non_container_child_fields->push_back(left_operand_.get());
-
-  non_container_child_field_names->push_back("right_operand");
-  non_container_child_fields->push_back(right_operand_.get());
-}
-
 std::string ParseFunctionCall::generateName() const {
   string name(name_->value());
   name.push_back('(');
@@ -175,59 +127,4 @@ void ParseFunctionCall::getFieldStringItems(
   }
 }
 
-std::string ParseExtractFunction::generateName() const {
-  std::string name;
-  name.append("EXTRACT(");
-  name.append(extract_field_->value());
-  name.append(" FROM ");
-  name.append(date_expression_->generateName());
-  name.push_back(')');
-  return name;
-}
-
-void ParseExtractFunction::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<const ParseTreeNode*> *non_container_child_fields,
-    std::vector<std::string> *container_child_field_names,
-    std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const {
-  inline_field_names->push_back("unit");
-  inline_field_values->push_back(extract_field_->value());
-
-  non_container_child_field_names->push_back("date_expression");
-  non_container_child_fields->push_back(date_expression_.get());
-}
-
-std::string ParseSubstringFunction::generateName() const {
-  std::string name;
-  name.append("SUBSTRING(");
-  name.append(operand_->generateName());
-  name.append(" FROM ");
-  name.append(std::to_string(start_position_));
-  if (length_ != kDefaultLength) {
-    name.append(" FOR ");
-    name.append(std::to_string(length_));
-  }
-  name.push_back(')');
-  return name;
-}
-
-void ParseSubstringFunction::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<const ParseTreeNode*> *non_container_child_fields,
-    std::vector<std::string> *container_child_field_names,
-    std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const {
-  inline_field_names->push_back("start_position");
-  inline_field_values->push_back(std::to_string(start_position_));
-
-  inline_field_names->push_back("length");
-  inline_field_values->push_back(std::to_string(length_));
-
-  non_container_child_field_names->push_back("operand");
-  non_container_child_fields->push_back(operand_.get());
-}
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/parser/ParseBasicExpressions.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseBasicExpressions.hpp b/parser/ParseBasicExpressions.hpp
index d8de669..4572214 100644
--- a/parser/ParseBasicExpressions.hpp
+++ b/parser/ParseBasicExpressions.hpp
@@ -36,9 +36,6 @@
 
 namespace quickstep {
 
-class BinaryOperation;
-class UnaryOperation;
-
 /** \addtogroup Parser
  *  @{
  */
@@ -173,155 +170,6 @@ class ParseAttribute : public ParseExpression {
 
 
 /**
- * @brief The parsed representation of an unary operation applied to an expression.
- **/
-class ParseUnaryExpression : public ParseExpression {
- public:
-  /**
-   * @brief Constructor.
-   *
-   * @param line_number Line number of the first token of this node in the SQL statement.
-   * @param column_number Column number of the first token of this node in the SQL statement.
-   * @param op The UnaryOperation from the quickstep type system to apply.
-   * @param operand The parsed scalar representation of the unary operation's
-   *        argument, which becomes owned by this ParseScalarUnaryExpression.
-   **/
-  ParseUnaryExpression(const int line_number,
-                       const int column_number,
-                       const UnaryOperation &op,
-                       ParseExpression *operand)
-      : ParseExpression(line_number, column_number),
-        op_(op),
-        operand_(operand) {
-  }
-
-  /**
-   * @brief Destructor.
-   */
-  ~ParseUnaryExpression() override {
-  }
-
-  ExpressionType getExpressionType() const override {
-    return kUnaryExpression;
-  }
-
-  std::string getName() const override;
-
-  /**
-   * @return The unary operation.
-   */
-  const UnaryOperation& op() const {
-    return op_;
-  }
-
-  /**
-   * @return The operand expression.
-   */
-  const ParseExpression* operand() const {
-    return operand_.get();
-  }
-
-  std::string generateName() const override;
-
- 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<const ParseTreeNode*> *non_container_child_fields,
-      std::vector<std::string> *container_child_field_names,
-      std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override;
-
- private:
-  const UnaryOperation &op_;
-  std::unique_ptr<ParseExpression> operand_;
-
-  DISALLOW_COPY_AND_ASSIGN(ParseUnaryExpression);
-};
-
-/**
- * @brief The parsed representation of a binary operation applied to two
- *        expressions.
- **/
-class ParseBinaryExpression : public ParseExpression {
- public:
-  /**
-   * @brief Constructor.
-   *
-   * @param line_number Line number of the binary operator token in the SQL statement.
-   * @param column_number Column number of the binary operator token in the SQL statement.
-   * @param op The BinaryOperation from the quickstep type system to apply.
-   * @param left_operand The parsed scalar representation of the binary
-   *        operation's left argument, which becomes owned by this
-   *        ParseScalarBinaryExpression.
-   * @param right_operand The parsed scalar representation of the binary
-   *        operation's right argument, which becomes owned by this
-   *        ParseScalarBinaryExpression.
-   **/
-  ParseBinaryExpression(const int line_number,
-                        const int column_number,
-                        const BinaryOperation &op,
-                        ParseExpression *left_operand,
-                        ParseExpression *right_operand)
-      : ParseExpression(line_number, column_number),
-        op_(op),
-        left_operand_(left_operand),
-        right_operand_(right_operand) {
-  }
-
-  /**
-   * @brief Destructor.
-   */
-  ~ParseBinaryExpression() override {
-  }
-
-  ExpressionType getExpressionType() const override {
-    return kBinaryExpression;
-  }
-
-  std::string getName() const override;
-
-  /**
-   * @return The binary operation.
-   */
-  const BinaryOperation& op() const {
-    return op_;
-  }
-
-  /**
-   * @return The left operand expression.
-   */
-  const ParseExpression* left_operand() const {
-    return left_operand_.get();
-  }
-
-  /**
-   * @return The right operand expression.
-   */
-  const ParseExpression* right_operand() const {
-    return right_operand_.get();
-  }
-
-  std::string generateName() const override;
-
- 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<const ParseTreeNode*> *non_container_child_fields,
-      std::vector<std::string> *container_child_field_names,
-      std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override;
-
- private:
-  const BinaryOperation &op_;
-  std::unique_ptr<ParseExpression> left_operand_;
-  std::unique_ptr<ParseExpression> right_operand_;
-
-  DISALLOW_COPY_AND_ASSIGN(ParseBinaryExpression);
-};
-
-/**
  * @brief The parsed representation of '*' as a function argument.
  */
 class ParseStar : public ParseTreeNode {
@@ -346,6 +194,7 @@ class ParseStar : public ParseTreeNode {
   DISALLOW_COPY_AND_ASSIGN(ParseStar);
 };
 
+
 /**
  * @brief Parsed function call in the form of a name with a list of arguments in parentheses.
  */
@@ -495,144 +344,6 @@ class ParseFunctionCall : public ParseExpression {
   DISALLOW_COPY_AND_ASSIGN(ParseFunctionCall);
 };
 
-
-/**
- * @brief Parsed representation of EXTRACT(unit FROM date).
- */
-class ParseExtractFunction : public ParseExpression {
- public:
-  /**
-   * @brief Constructor.
-   *
-   * @param line_number The line number of the token "extract" in the statement.
-   * @param column_number The column number of the token "extract in the statement.
-   * @param extract_field The field to extract.
-   * @param source_expression The expression to extract a field from.
-   */
-  ParseExtractFunction(const int line_number,
-                       const int column_number,
-                       ParseString *extract_field,
-                       ParseExpression *date_expression)
-      : ParseExpression(line_number, column_number),
-        extract_field_(extract_field),
-        date_expression_(date_expression) {
-  }
-
-  ExpressionType getExpressionType() const override {
-    return kExtract;
-  }
-
-  std::string getName() const override {
-    return "Extract";
-  }
-
-  /**
-   * @return The field to extract.
-   */
-  const ParseString* extract_field() const {
-    return extract_field_.get();
-  }
-
-  /**
-   * @return The expression to extract a field from.
-   */
-  const ParseExpression* date_expression() const {
-    return date_expression_.get();
-  }
-
-  std::string generateName() const override;
-
- 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<const ParseTreeNode*> *non_container_child_fields,
-      std::vector<std::string> *container_child_field_names,
-      std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override;
-
- private:
-  std::unique_ptr<ParseString> extract_field_;
-  std::unique_ptr<ParseExpression> date_expression_;
-
-  DISALLOW_COPY_AND_ASSIGN(ParseExtractFunction);
-};
-
-
-/**
- * @brief Parsed representation of the substring function.
- */
-class ParseSubstringFunction : public ParseExpression {
- public:
-  static constexpr std::size_t kDefaultLength = std::numeric_limits<std::size_t>::max();
-
-  /**
-   * @brief Constructor.
-   *
-   * @param line_number The line number of the first token of the function call.
-   * @param column_number The column number of the first token of the function call.
-   * @param operand The operand of the substring.
-   * @param start_position The 1-based starting position of the substring.
-   * @param length Optional substring length.
-   */
-  ParseSubstringFunction(const int line_number,
-                         const int column_number,
-                         ParseExpression *operand,
-                         const std::size_t start_position,
-                         const std::size_t length = kDefaultLength)
-      : ParseExpression(line_number, column_number),
-        operand_(operand),
-        start_position_(start_position),
-        length_(length) {}
-
-  ExpressionType getExpressionType() const override {
-    return kSubstring;
-  }
-
-  std::string getName() const override {
-    return "Substring";
-  }
-
-  /**
-   * @return The operand of the substring.
-   */
-  const ParseExpression* operand() const {
-    return operand_.get();
-  }
-
-  /**
-   * @return The 1-based starting position of the substring.
-   */
-  std::size_t start_position() const {
-    return start_position_;
-  }
-
-  /**
-   * @return Then substring length.
-   */
-  std::size_t length() const {
-    return length_;
-  }
-
-  std::string generateName() const override;
-
- 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<const ParseTreeNode*> *non_container_child_fields,
-      std::vector<std::string> *container_child_field_names,
-      std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override;
-
- private:
-  std::unique_ptr<ParseExpression> operand_;
-  const std::size_t start_position_;
-  const std::size_t length_;
-
-  DISALLOW_COPY_AND_ASSIGN(ParseSubstringFunction);
-};
-
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/parser/ParseExpression.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseExpression.hpp b/parser/ParseExpression.hpp
index 1b9ade4..94b4487 100644
--- a/parser/ParseExpression.hpp
+++ b/parser/ParseExpression.hpp
@@ -39,14 +39,11 @@ class ParseExpression : public ParseTreeNode {
   enum ExpressionType {
     kAttribute,
     kBinaryExpression,
-    kExtract,
     kFunctionCall,
     kScalarLiteral,
     kSearchedCaseExpression,
     kSimpleCaseExpression,
     kSubqueryExpression,
-    kSubstring,
-    kUnaryExpression,
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/parser/SqlLexer.lpp
----------------------------------------------------------------------
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index c6402e0..a3f3238 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -184,6 +184,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "blocksample"      return TOKEN_BLOCKSAMPLE;
   "bloomfilter"      return TOKEN_BLOOM_FILTER;
   "case"             return TOKEN_CASE;
+  "cast"             return TOKEN_CAST;
   "csbtree"          return TOKEN_CSB_TREE;
   "by"               return TOKEN_BY;
   "char"             return TOKEN_CHARACTER;


[13/32] incubator-quickstep git commit: Adds NetworkIO to CLI

Posted by ji...@apache.org.
Adds NetworkIO to CLI

Main changes:
-IOHandle represents a single interaction with quickstep
-CMAKE flag to enable, off by default
-2 flags for ip, port of network IO
-new executable for network client
-Tests


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

Branch: refs/heads/new-op
Commit: 99198ef2995001cef5ca5287b2feb861fbbbf9fa
Parents: 4eb1dee
Author: cramja <ma...@gmail.com>
Authored: Mon Apr 17 17:58:26 2017 -0500
Committer: cramja <ma...@gmail.com>
Committed: Sat Apr 29 15:28:36 2017 -0500

----------------------------------------------------------------------
 CMakeLists.txt                            |  18 +-
 cli/CMakeLists.txt                        |  83 ++++++++
 cli/CliConfig.h.in                        |   1 +
 cli/IOInterface.hpp                       |  84 ++++++++
 cli/LineReaderBuffered.cpp                |  39 ++++
 cli/LineReaderBuffered.hpp                |  75 +++++++
 cli/LocalIO.hpp                           |  89 ++++++++
 cli/NetworkCli.proto                      |  33 +++
 cli/NetworkCliClient.hpp                  |  94 +++++++++
 cli/NetworkCliClientMain.cpp              |  61 ++++++
 cli/NetworkIO.cpp                         |  58 +++++
 cli/NetworkIO.hpp                         | 282 +++++++++++++++++++++++++
 cli/QuickstepCli.cpp                      |  69 +++---
 cli/tests/CMakeLists.txt                  |  35 +++
 cli/tests/LineReaderBuffered_unittest.cpp |  70 ++++++
 cli/tests/NetworkIO_unittest.cpp          | 187 ++++++++++++++++
 validate_cmakelists.py                    |   1 +
 17 files changed, 1253 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index a4bce6f..ed80fab 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -145,6 +145,7 @@ if (ENABLE_VECTOR_PREDICATE_SHORT_CIRCUIT)
   )
 endif()
 
+option(ENABLE_NETWORK_CLI "Allows use of the network cli" OFF)
 option(ENABLE_DISTRIBUTED "Use the distributed version of Quickstep" OFF)
 
 macro (set_gflags_lib_name)
@@ -730,6 +731,7 @@ target_link_libraries(quickstep_cli_shell
                       quickstep_cli_Flags
                       quickstep_cli_InputParserUtil
                       quickstep_cli_LineReader
+                      quickstep_cli_LocalIO
                       quickstep_cli_PrintToScreen
                       quickstep_parser_ParseStatement
                       quickstep_parser_SqlParserWrapper
@@ -755,7 +757,10 @@ if (ENABLE_HDFS)
   target_link_libraries(quickstep_cli_shell
                         quickstep_storage_FileManagerHdfs)
 endif()
-
+if (ENABLE_NETWORK_CLI)
+  target_link_libraries(quickstep_cli_shell
+                        quickstep_cli_NetworkIO)
+endif()
 # Link against other required system and third-party libraries.
 target_link_libraries(quickstep_cli_shell ${LIBS})
 
@@ -792,3 +797,14 @@ if (ENABLE_DISTRIBUTED)
                         ${GFLAGS_LIB_NAME}
                         ${GRPCPLUSPLUS_LIBRARIES})
 endif(ENABLE_DISTRIBUTED)
+
+if (ENABLE_NETWORK_CLI)
+  add_executable (quickstep_client cli/NetworkCliClientMain.cpp)
+  target_link_libraries(quickstep_client
+                        ${GFLAGS_LIB_NAME}
+                        ${GRPCPLUSPLUS_LIBRARIES}
+                        glog
+                        quickstep_cli_LineReaderBuffered
+                        quickstep_cli_NetworkCliClient
+                        quickstep_cli_NetworkIO)
+endif()

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/CMakeLists.txt b/cli/CMakeLists.txt
index 4562fe6..e802e8d 100644
--- a/cli/CMakeLists.txt
+++ b/cli/CMakeLists.txt
@@ -16,6 +16,7 @@
 # under the License.
 
 include_directories(${CMAKE_CURRENT_BINARY_DIR})
+
 if (ENABLE_DISTRIBUTED)
   add_subdirectory(distributed)
 endif(ENABLE_DISTRIBUTED)
@@ -39,11 +40,31 @@ if (ENABLE_GOOGLE_PROFILER)
   set(QUICKSTEP_ENABLE_GOOGLE_PROFILER TRUE)
 endif()
 
+if (ENABLE_NETWORK_CLI)
+  set(QUICKSTEP_ENABLE_NETWORK_CLI TRUE)
+endif()
+
 configure_file (
   "${CMAKE_CURRENT_SOURCE_DIR}/CliConfig.h.in"
   "${CMAKE_CURRENT_BINARY_DIR}/CliConfig.h"
 )
 
+# Compile the protos for Single Node Server mode.
+if (ENABLE_NETWORK_CLI)
+  # We will need some of the TMBs libraries
+  set(CMAKE_MODULE_PATH
+      ${CMAKE_MODULE_PATH}
+      "${PROJECT_SOURCE_DIR}/third_party/src/tmb/cmake")
+
+  find_package(Grpc++ REQUIRED)
+  include_directories(${GRPCPLUSPLUS_INCLUDE_DIRS})
+
+  GRPC_GENERATE_CPP(cli_NetworkCli_proto_srcs
+                    cli_NetworkCli_proto_hdrs
+                    .
+                    NetworkCli.proto)
+endif()
+
 # Declare micro-libs and link dependencies:
 add_library(quickstep_cli_CommandExecutor CommandExecutor.cpp CommandExecutor.hpp)
 add_library(quickstep_cli_CommandExecutorUtil CommandExecutorUtil.cpp CommandExecutorUtil.hpp)
@@ -51,6 +72,7 @@ add_library(quickstep_cli_Constants ../empty_src.cpp Constants.hpp)
 add_library(quickstep_cli_DefaultsConfigurator DefaultsConfigurator.cpp DefaultsConfigurator.hpp)
 add_library(quickstep_cli_DropRelation DropRelation.cpp DropRelation.hpp)
 add_library(quickstep_cli_Flags Flags.cpp Flags.hpp)
+add_library(quickstep_cli_IOInterface ../empty_src.cpp IOInterface.hpp)
 add_library(quickstep_cli_InputParserUtil InputParserUtil.cpp InputParserUtil.hpp)
 
 if(USE_LINENOISE)
@@ -67,6 +89,25 @@ else()
               LineReaderDumb.hpp)
 endif()
 
+add_library(quickstep_cli_LineReaderBuffered
+            LineReaderBuffered.cpp
+            LineReaderBuffered.hpp)
+add_library(quickstep_cli_LocalIO
+            ../empty_src.cpp
+            LocalIO.hpp)
+
+if (ENABLE_NETWORK_CLI)
+  add_library(quickstep_cli_NetworkCli_proto
+              ${cli_NetworkCli_proto_srcs}
+              ${cli_NetworkCli_proto_hdrs})
+  add_library(quickstep_cli_NetworkCliClient
+              ../empty_src.cpp
+              NetworkCliClient.hpp)
+  add_library(quickstep_cli_NetworkIO
+              NetworkIO.cpp
+              NetworkIO.hpp)
+endif()
+
 add_library(quickstep_cli_PrintToScreen PrintToScreen.cpp PrintToScreen.hpp)
 
 # Link dependencies:
@@ -135,6 +176,8 @@ target_link_libraries(quickstep_cli_Flags
                       quickstep_cli_DefaultsConfigurator
                       quickstep_storage_StorageConstants
                       ${GFLAGS_LIB_NAME})
+target_link_libraries(quickstep_cli_IOInterface
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_cli_InputParserUtil
                       glog
                       quickstep_utility_Macros
@@ -151,6 +194,37 @@ else()
   target_link_libraries(quickstep_cli_LineReader
                         quickstep_utility_Macros)
 endif()
+target_link_libraries(quickstep_cli_LineReaderBuffered
+                      quickstep_cli_LineReader
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_cli_LocalIO
+                      quickstep_cli_LineReader
+                      quickstep_cli_IOInterface
+                      quickstep_utility_Macros)
+if (ENABLE_NETWORK_CLI)
+  target_link_libraries(quickstep_cli_NetworkCli_proto
+                        ${GRPCPLUSPLUS_LIBRARIES}
+                        ${PROTOBUF3_LIBRARY})
+  target_link_libraries(quickstep_cli_NetworkCliClient
+                        ${GFLAGS_LIB_NAME}
+                        ${GRPCPLUSPLUS_LIBRARIES}
+                        ${PROTOBUF3_LIBRARIES}
+                        glog
+                        quickstep_cli_NetworkCli_proto
+                        quickstep_utility_Macros)
+  target_link_libraries(quickstep_cli_NetworkIO
+                        ${GFLAGS_LIB_NAME}
+                        ${GRPCPLUSPLUS_LIBRARIES}
+                        ${PROTOBUF3_LIBRARIES}
+                        glog
+                        quickstep_cli_IOInterface
+                        quickstep_cli_NetworkCli_proto
+                        quickstep_threading_ConditionVariable
+                        quickstep_threading_SpinSharedMutex
+                        quickstep_utility_Macros
+                        quickstep_utility_MemStream
+                        quickstep_utility_ThreadSafeQueue)
+endif()
 target_link_libraries(quickstep_cli_PrintToScreen
                       ${GFLAGS_LIB_NAME}
                       quickstep_catalog_CatalogAttribute
@@ -175,6 +249,15 @@ target_link_libraries(quickstep_cli
                       quickstep_cli_DefaultsConfigurator
                       quickstep_cli_DropRelation
                       quickstep_cli_Flags
+                      quickstep_cli_IOInterface
                       quickstep_cli_InputParserUtil
                       quickstep_cli_LineReader
+                      quickstep_cli_LineReaderBuffered
+                      quickstep_cli_LocalIO
                       quickstep_cli_PrintToScreen)
+if (ENABLE_NETWORK_CLI)
+  target_link_libraries(quickstep_cli
+                        quickstep_cli_NetworkCli_proto
+                        quickstep_cli_NetworkCliClient
+                        quickstep_cli_NetworkIO)
+endif()

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/CliConfig.h.in
----------------------------------------------------------------------
diff --git a/cli/CliConfig.h.in b/cli/CliConfig.h.in
index 2508f3a..aa04367 100644
--- a/cli/CliConfig.h.in
+++ b/cli/CliConfig.h.in
@@ -20,3 +20,4 @@
 #cmakedefine QUICKSTEP_USE_LINENOISE
 #cmakedefine QUICKSTEP_OS_WINDOWS
 #cmakedefine QUICKSTEP_ENABLE_GOOGLE_PROFILER
+#cmakedefine QUICKSTEP_ENABLE_NETWORK_CLI

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/IOInterface.hpp
----------------------------------------------------------------------
diff --git a/cli/IOInterface.hpp b/cli/IOInterface.hpp
new file mode 100644
index 0000000..815596e
--- /dev/null
+++ b/cli/IOInterface.hpp
@@ -0,0 +1,84 @@
+/**
+ * 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_CLI_IO_INTERFACE_HPP_
+#define QUICKSTEP_CLI_IO_INTERFACE_HPP_
+
+#include <cstdio>
+#include <string>
+
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+/**
+ * An individual IO interaction with Quickstep.
+ */
+class IOHandle {
+ public:
+  IOHandle() {}
+
+  /**
+   * @note Destructor should handle clean up of any IO state.
+   */
+  virtual ~IOHandle() {}
+
+  /**
+   * @return A file handle for standard output.
+   */
+  virtual FILE *out() = 0;
+
+  /**
+   * @return A file handle for error output.
+   */
+  virtual FILE *err() = 0;
+
+  virtual std::string getCommand() const = 0;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(IOHandle);
+};
+
+/**
+ * Virtual base defines a generic, file-based interface around IO. One IO interaction (eg a SQL query) will be assigned
+ * an IOHandle. On destruction of the IOHandle, the IO interaction has finished.
+ */
+class IOInterface {
+ public:
+  /**
+   * @note Destructing the IOInterface should close any outstanding IO state (eg an open port).
+   */
+  virtual ~IOInterface() {}
+
+  /**
+   * @brief Retrieves the next IOHandle. Blocks if no IO ready.
+   * @return An IOHandle.
+   */
+  virtual IOHandle* getNextIOHandle() = 0;
+
+ protected:
+  IOInterface() {}
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(IOInterface);
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_CLI_IO_INTERFACE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/LineReaderBuffered.cpp
----------------------------------------------------------------------
diff --git a/cli/LineReaderBuffered.cpp b/cli/LineReaderBuffered.cpp
new file mode 100644
index 0000000..35b9144
--- /dev/null
+++ b/cli/LineReaderBuffered.cpp
@@ -0,0 +1,39 @@
+/**
+ * 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 "cli/LineReaderBuffered.hpp"
+
+#include <string>
+
+namespace quickstep {
+
+LineReaderBuffered::LineReaderBuffered(const std::string &default_prompt,
+                                       const std::string &continue_prompt)
+    : LineReader(default_prompt, continue_prompt),
+      buffer_empty_(true) {}
+
+LineReaderBuffered::LineReaderBuffered() : LineReader("", ""), buffer_empty_(true) {}
+
+std::string LineReaderBuffered::getLineInternal(const bool continuing) {
+  // This method is called when the leftover_ string is depleted.
+  buffer_empty_ = true;
+  return "";
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/LineReaderBuffered.hpp
----------------------------------------------------------------------
diff --git a/cli/LineReaderBuffered.hpp b/cli/LineReaderBuffered.hpp
new file mode 100644
index 0000000..f1d99bf
--- /dev/null
+++ b/cli/LineReaderBuffered.hpp
@@ -0,0 +1,75 @@
+/**
+ * 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_CLI_LINE_READER_BUFFERED_HPP_
+#define QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_
+
+#include <string>
+
+#include "cli/LineReader.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class LineReaderBuffered : public LineReader {
+ public:
+  /**
+   * @brief A line reader which accepts a string buffer.
+   * Other line readers are meant to support some form of user interaction. This linereader does not and is intended for
+   * programmer use- it does not print anything to stdout. Therefore it ignores any prompt strings given to the
+   * inherited constructor.
+   * @param default_prompt Not used by this line reader, but required by interface.
+   * @param continue_prompt Not used by this line reader, but required by interface.
+   */
+  LineReaderBuffered(const std::string &default_prompt,
+                     const std::string &continue_prompt);
+
+  LineReaderBuffered();
+
+  ~LineReaderBuffered() override {}
+
+  /**
+   * @brief Replaces the current buffer contents with new contents.
+   * @param buffer Replacement text.
+   */
+  void setBuffer(std::string buffer) {
+    leftover_ = buffer;
+    buffer_empty_ = false;
+  }
+
+  /**
+   * @brief This is set to true after getNextCommand is called and runs out of input to process.
+   * @return True if the buffer has been consumed.
+   */
+  bool bufferEmpty() const {
+    return buffer_empty_;
+  }
+
+ protected:
+  std::string getLineInternal(const bool continuing) override;
+
+ private:
+  bool buffer_empty_;
+
+  DISALLOW_COPY_AND_ASSIGN(LineReaderBuffered);
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_CLI_LINE_READER_BUFFERED_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/LocalIO.hpp
----------------------------------------------------------------------
diff --git a/cli/LocalIO.hpp b/cli/LocalIO.hpp
new file mode 100644
index 0000000..4dc3b3f
--- /dev/null
+++ b/cli/LocalIO.hpp
@@ -0,0 +1,89 @@
+/**
+ * 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_CLI_LOCAL_IO_HPP_
+#define QUICKSTEP_CLI_LOCAL_IO_HPP_
+
+#include <cstdio>
+#include <string>
+
+#include "cli/CliConfig.h"
+#include "cli/IOInterface.hpp"
+#include "cli/LineReader.hpp"
+#ifdef QUICKSTEP_USE_LINENOISE
+#include "cli/LineReaderLineNoise.hpp"
+typedef quickstep::LineReaderLineNoise LineReaderImpl;
+#else
+#include "cli/LineReaderDumb.hpp"
+typedef quickstep::LineReaderDumb LineReaderImpl;
+#endif
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class LocalIOHandle final : public IOHandle {
+ public:
+  LocalIOHandle() {}
+
+  explicit LocalIOHandle(const std::string &command)
+      : command_(command) {}
+
+  ~LocalIOHandle() override {}
+
+  FILE *out() override {
+    return stdout;
+  }
+
+  FILE *err() override {
+    return stderr;
+  }
+
+  std::string getCommand() const override {
+    return command_;
+  }
+
+ private:
+  const std::string command_;
+
+  DISALLOW_COPY_AND_ASSIGN(LocalIOHandle);
+};
+
+/**
+ * IO class for getting commands from stdin via an interactive line reader.
+ */
+class LocalIO final : public IOInterface {
+ public:
+  LocalIO() : line_reader_("quickstep> ",
+                           "      ...> ") {}
+
+  ~LocalIO() override {}
+
+  IOHandle* getNextIOHandle() override {
+    return new LocalIOHandle(line_reader_.getNextCommand());
+  }
+
+ private:
+  LineReaderImpl line_reader_;
+
+  DISALLOW_COPY_AND_ASSIGN(LocalIO);
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_CLI_LOCAL_IO_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/NetworkCli.proto
----------------------------------------------------------------------
diff --git a/cli/NetworkCli.proto b/cli/NetworkCli.proto
new file mode 100644
index 0000000..e065c7c
--- /dev/null
+++ b/cli/NetworkCli.proto
@@ -0,0 +1,33 @@
+// 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.
+
+syntax = "proto3";
+
+package quickstep;
+
+service NetworkCli {
+  rpc SendQuery (QueryRequest) returns (QueryResponse) {}
+}
+
+message QueryRequest {
+  string query = 1;
+}
+
+message QueryResponse {
+  string query_result = 1;
+  string error_result = 2;
+}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/NetworkCliClient.hpp
----------------------------------------------------------------------
diff --git a/cli/NetworkCliClient.hpp b/cli/NetworkCliClient.hpp
new file mode 100644
index 0000000..7affeff
--- /dev/null
+++ b/cli/NetworkCliClient.hpp
@@ -0,0 +1,94 @@
+/**
+ * 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_CLI_SINGLE_NODE_CLIENT_HPP_
+#define QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_
+
+#include <grpc++/grpc++.h>
+
+#include <iostream>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "cli/NetworkCli.grpc.pb.h"
+#include "cli/NetworkCli.pb.h"
+#include "utility/Macros.hpp"
+
+#include "gflags/gflags.h"
+#include "glog/logging.h"
+
+using grpc::Channel;
+using grpc::ClientContext;
+using grpc::Status;
+
+namespace quickstep {
+
+/**
+ * A simple wrapper class used to do CLI interactions with QuickstepCLI via the gRPC interface.
+ */
+class NetworkCliClient {
+ public:
+  explicit NetworkCliClient(const std::shared_ptr<Channel> &channel)
+      : stub_(NetworkCli::NewStub(channel)) {}
+
+  /**
+   * Assembles the client's payload, sends it and presents the response back from the server.
+   * @param user_query A SQL statement or command to be executed on the server.
+   * @return The text of the server's response.
+   */
+  std::string Query(const std::string &user_query) {
+    QueryRequest request;
+    request.set_query(user_query);
+    QueryResponse response;
+
+    Status status = SendQuery(request, &response);
+
+    if (status.ok()) {
+      return HandleQueryResponse(response);
+    } else {
+      LOG(WARNING) << "RPC call failed with code " << status.error_code()
+                   << " and message: " << status.error_message();
+      return "RPC failed";
+    }
+  }
+
+  Status SendQuery(const QueryRequest& request, QueryResponse* response) {
+    ClientContext context;
+    return stub_->SendQuery(&context, request, response);
+  }
+
+ private:
+  /**
+   * Handle a valid response from the server.
+   * @param response A valid query response.
+   * @return The response string.
+   */
+  std::string HandleQueryResponse(QueryResponse const &response) const {
+    return response.query_result() + response.error_result();
+  }
+
+  std::unique_ptr<NetworkCli::Stub> stub_;
+
+  DISALLOW_COPY_AND_ASSIGN(NetworkCliClient);
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_CLI_SINGLE_NODE_CLIENT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/NetworkCliClientMain.cpp
----------------------------------------------------------------------
diff --git a/cli/NetworkCliClientMain.cpp b/cli/NetworkCliClientMain.cpp
new file mode 100644
index 0000000..862941c
--- /dev/null
+++ b/cli/NetworkCliClientMain.cpp
@@ -0,0 +1,61 @@
+/**
+ * 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 <grpc++/grpc++.h>
+
+#include <iostream>
+#include <istream>
+#include <memory>
+#include <string>
+
+#include "cli/LineReaderBuffered.hpp"
+#include "cli/NetworkCliClient.hpp"
+#include "cli/NetworkIO.hpp"
+
+#include "gflags/gflags.h"
+
+using quickstep::LineReaderBuffered;
+using quickstep::NetworkCliClient;
+
+int main(int argc, char **argv) {
+  google::InitGoogleLogging(argv[0]);
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
+  grpc_init();
+
+  // Attempts to send a single query retrieved from stdin to the Quickstep Server.
+  NetworkCliClient qs_client(
+    grpc::CreateChannel(quickstep::NetworkIO::GetAddress(),
+                        grpc::InsecureChannelCredentials()));
+
+  // Read stdin until EOF, then we use a Line reader to divide query into parts.
+  std::cin >> std::noskipws;
+  std::istream_iterator<char> it(std::cin), end;
+  std::string user_queries(it, end);
+
+  LineReaderBuffered linereader;
+  linereader.setBuffer(user_queries);
+  while (!linereader.bufferEmpty()) {
+    std::string query = linereader.getNextCommand();
+    if (!query.empty()) {
+      std::cout << query << std::endl;
+      std::cout << qs_client.Query(query) << std::endl;
+    }
+  }
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/NetworkIO.cpp
----------------------------------------------------------------------
diff --git a/cli/NetworkIO.cpp b/cli/NetworkIO.cpp
new file mode 100644
index 0000000..f0acc65
--- /dev/null
+++ b/cli/NetworkIO.cpp
@@ -0,0 +1,58 @@
+/**
+ * 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 "cli/NetworkIO.hpp"
+
+#include <grpc++/grpc++.h>
+
+#include "gflags/gflags.h"
+#include "glog/logging.h"
+
+namespace quickstep {
+
+static bool ValidatePort(const char *flagname, std::int32_t value) {
+  int const min = 0, max = 65536;
+  if (value > min && value < max) {
+    return true;
+  }
+  std::cout << "Invalid value for --" << flagname << ": " << value
+            << "\nUse ports between " << min << " and "
+            << max << std::endl;
+  return false;
+}
+
+DEFINE_int32(cli_network_port, 3000,
+             "Listens for TCP connections on this port when network mode is enabled. "
+               "This is only used if the cli is set to use the network mode (--mode=network).");
+DEFINE_validator(cli_network_port, &ValidatePort);
+
+DEFINE_string(cli_network_ip, "0.0.0.0",
+              "The ip address which the cli should open a connection on. This is only used "
+                "if the cli is set to use the network mode (--mode=network). Defaults to "
+                "the address of localhost.");
+
+NetworkIO::NetworkIO() {
+  grpc::ServerBuilder builder;
+  builder.AddListeningPort(GetAddress(), grpc::InsecureServerCredentials());
+  builder.RegisterService(&service_);
+  server_ = builder.BuildAndStart();
+  LOG(INFO) << "Listening on " << GetAddress();
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/NetworkIO.hpp
----------------------------------------------------------------------
diff --git a/cli/NetworkIO.hpp b/cli/NetworkIO.hpp
new file mode 100644
index 0000000..3693f88
--- /dev/null
+++ b/cli/NetworkIO.hpp
@@ -0,0 +1,282 @@
+/**
+ * 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_CLI_NETWORK_IO_HPP_
+#define QUICKSTEP_CLI_NETWORK_IO_HPP_
+
+#include <grpc++/grpc++.h>
+
+#include <cstdio>
+#include <iostream>
+#include <memory>
+#include <string>
+
+#include "cli/IOInterface.hpp"
+#include "cli/NetworkCli.grpc.pb.h"
+#include "cli/NetworkCli.pb.h"
+#include "threading/ConditionVariable.hpp"
+#include "threading/SpinSharedMutex.hpp"
+#include "utility/Macros.hpp"
+#include "utility/MemStream.hpp"
+#include "utility/ThreadSafeQueue.hpp"
+
+#include "gflags/gflags.h"
+#include "glog/logging.h"
+
+using grpc::Channel;
+using grpc::ClientContext;
+using grpc::Server;
+using grpc::Status;
+
+namespace quickstep {
+DECLARE_int32(cli_network_port);
+DECLARE_string(cli_network_ip);
+namespace networkio_internal {
+
+/**
+ * Contains state and helper methods for managing interactions between a producer/consumer thread. A producer thread
+ * will wait on a condition variable. When the consumer thread returns, it will notify the producer.
+ */
+class RequestState {
+ public:
+  explicit RequestState(const QueryRequest& request)
+      : response_ready_(false),
+        canceled_(false),
+        request_(request),
+        condition_(mutex_.createConditionVariable()) {}
+
+  /**
+   * @brief Notifies that the consumer has finished consuming and that a response is ready.
+   * To be called after the consumer has executed.
+   * @param stdout_str Stdout from Quickstep.
+   * @param stderr_str Stderr from Quickstep.
+   */
+  void responseReady(const std::string& stdout_str, const std::string& stderr_str) {
+    std::unique_lock<Mutex> lock(mutex_);
+    response_message_.set_query_result(stdout_str);
+    response_message_.set_error_result(stderr_str);
+    response_ready_ = true;
+    condition_->signalOne();
+  }
+
+  /**
+   * @brief The producer thread blocks until Quickstep signals that it has finished.
+   * @note Quickstep may either produce a query response or cancel. Both these actions must notify the condition.
+   */
+  void waitForResponse() {
+    while (!response_ready_)
+      condition_->await();
+  }
+
+  /**
+   * @brief Notifies the producer that its request will not be served by Quickstep.
+   */
+  void cancel() {
+    std::unique_lock<Mutex> lock(mutex_);
+    canceled_ = true;
+    response_ready_ = true;
+    condition_->signalOne();
+  }
+
+  /**
+   * @return The producer's query for Quickstep to process.
+   */
+  std::string getRequest() const {
+    return request_.query();
+  }
+
+  /**
+   * @return The response message from Quickstep.
+   */
+  QueryResponse getResponse() const {
+    DCHECK(response_ready_);
+    return response_message_;
+  }
+
+  /**
+   * @return True if query was canceled.
+   */
+  bool getCanceled() const {
+    DCHECK(response_ready_);
+    return canceled_;
+  }
+
+ private:
+  bool response_ready_;
+  bool canceled_;
+  const QueryRequest& request_;
+  QueryResponse response_message_;
+  Mutex mutex_;
+  ConditionVariable *condition_;  // note: owned by the mutex.
+
+  DISALLOW_COPY_AND_ASSIGN(RequestState);
+};
+
+}  // namespace networkio_internal
+
+using networkio_internal::RequestState;
+
+/**
+ * @brief Contains the callback methods which the gRPC service defines.
+ * When a request is made of gRPC, a gRPC worker thread is spun up and enters one of the callback methods
+ * (eg SendQuery). This thread keeps the network connection open while Quickstep processes the query. Concurrency
+ * control between the gRPC worker thread, and the Quickstep thread is controlled by a RequestState object which is
+ * created for each interaction.
+ */
+class NetworkCliServiceImpl final : public NetworkCli::Service {
+ public:
+  NetworkCliServiceImpl()
+      : running_(true) {}
+
+  /**
+   * @brief Handles gRPC request.
+   * Sets the buffer in the RequestState, places the request on a queue, then waits for a response. The response shall
+   * be triggered by the Quickstep system.
+   */
+  Status SendQuery(grpc::ServerContext *context,
+                   const QueryRequest *request,
+                   QueryResponse *response) override {
+    std::unique_ptr<RequestState> request_state(new RequestState(*request));
+    // Check to see if the gRPC service has been shutdown.
+    {
+      SpinSharedMutexSharedLock<true> lock(service_mtx_);
+
+      if (!running_) {
+        return Status::CANCELLED;
+      }
+      // While we have a service lock, we add to the Queue. Note that we keep the service lock to protect ourselves from
+      // a race condition in the kill() method.
+
+      // Pushing to the queue will notify consumers.
+      request_queue_.push(request_state.get());
+    }
+
+    DCHECK(request_state);
+
+    // We have pushed to the request queue, so all there is to do now is wait for Quickstep to process the request.
+    request_state->waitForResponse();
+    if (request_state->getCanceled()) {
+      return Status::CANCELLED;
+    }
+    *response = request_state->getResponse();
+    return Status::OK;
+  }
+
+  /**
+   * @brief The consumer thread waits for a request to materialize.
+   * @return A non-owned RequestState.
+   */
+  RequestState* waitForRequest() {
+    return request_queue_.popOne();
+  }
+
+  /**
+   * @brief Stops accepting further requests and cancels all pending requests.
+   */
+  void kill() {
+    {
+      // This action guarantees that no further requests are added to the queue.
+      SpinSharedMutexExclusiveLock<true> lock(service_mtx_);
+      running_ = false;
+    }
+    // Go through each pending request, and cancel them.
+    while (!request_queue_.empty()) {
+      request_queue_.popOne()->cancel();
+    }
+  }
+
+ private:
+  SpinSharedMutex<true> service_mtx_;
+  bool running_;
+  ThreadSafeQueue<RequestState*> request_queue_;
+
+  DISALLOW_COPY_AND_ASSIGN(NetworkCliServiceImpl);
+};
+
+class NetworkIOHandle final : public IOHandle {
+ public:
+  explicit NetworkIOHandle(RequestState* state)
+      : request_state_(state) {}
+
+  ~NetworkIOHandle() override {
+      // All the commands from the last network interaction have completed, return our response.
+      // This signals to the producer thread that the interaction is complete.
+      request_state_->responseReady(out_stream_.str(), err_stream_.str());
+  }
+
+  FILE* out() override {
+    return out_stream_.file();
+  }
+
+  FILE* err() override {
+    return err_stream_.file();
+  }
+
+  std::string getCommand() const override {
+    return request_state_->getRequest();
+  }
+
+ private:
+  MemStream out_stream_, err_stream_;
+  RequestState *request_state_;
+
+  DISALLOW_COPY_AND_ASSIGN(NetworkIOHandle);
+};
+
+/**
+ * A network interface that uses gRPC to accept commands.
+ */
+class NetworkIO final : public IOInterface {
+ public:
+  NetworkIO();
+
+  ~NetworkIO() override {
+    service_.kill();
+    server_->Shutdown();
+    server_->Wait();
+  }
+
+  IOHandle* getNextIOHandle() override {
+    return new NetworkIOHandle(service_.waitForRequest());
+  }
+
+  /**
+   * @brief Kills the underlying gRPC service.
+   */
+  void killService() {
+    service_.kill();
+  }
+
+  /**
+   * @return IP address and port of the network address specified by the user flags.
+   */
+  static std::string GetAddress() {
+    return FLAGS_cli_network_ip + ":" + std::to_string(FLAGS_cli_network_port);
+  }
+
+ private:
+  std::unique_ptr<Server> server_;
+  NetworkCliServiceImpl service_;
+
+  DISALLOW_COPY_AND_ASSIGN(NetworkIO);
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_CLI_NETWORK_IO_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index c2634bc..d8e3c5c 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -28,19 +28,11 @@
 #include <utility>
 #include <vector>
 
-#include "cli/CliConfig.h"  // For QUICKSTEP_USE_LINENOISE, QUICKSTEP_ENABLE_GOOGLE_PROFILER.
+#include "cli/CliConfig.h"  // For QUICKSTEP_ENABLE_NETWORK_CLI, QUICKSTEP_ENABLE_GOOGLE_PROFILER.
 
 #include "cli/CommandExecutor.hpp"
 #include "cli/DropRelation.hpp"
 
-#ifdef QUICKSTEP_USE_LINENOISE
-#include "cli/LineReaderLineNoise.hpp"
-typedef quickstep::LineReaderLineNoise LineReaderImpl;
-#else
-#include "cli/LineReaderDumb.hpp"
-typedef quickstep::LineReaderDumb LineReaderImpl;
-#endif
-
 #ifdef QUICKSTEP_ENABLE_GOOGLE_PROFILER
 #include <gperftools/profiler.h>
 #endif
@@ -48,6 +40,11 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 #include "cli/DefaultsConfigurator.hpp"
 #include "cli/Flags.hpp"
 #include "cli/InputParserUtil.hpp"
+#include "cli/IOInterface.hpp"
+#include "cli/LocalIO.hpp"
+#ifdef QUICKSTEP_ENABLE_NETWORK_CLI
+#include "cli/NetworkIO.hpp"
+#endif
 #include "cli/PrintToScreen.hpp"
 #include "parser/ParseStatement.hpp"
 #include "parser/SqlParserWrapper.hpp"
@@ -99,6 +96,7 @@ using quickstep::FLAGS_num_workers;
 using quickstep::FLAGS_storage_path;
 using quickstep::ForemanSingleNode;
 using quickstep::InputParserUtil;
+using quickstep::IOInterface;
 using quickstep::MessageBusImpl;
 using quickstep::ParseResult;
 using quickstep::ParseStatement;
@@ -144,6 +142,13 @@ DEFINE_string(profile_file_name, "",
               // times more expensive than the average run. That means the query needs to be
               // run at least a hundred times to make the impact of the first run small (< 5 %).
 
+DEFINE_string(mode, "local",
+              "Defines which interaction mode to use. Options are either 'local' which "
+              "uses a simple SQL command line interface via stdin (default). The other "
+              "option is to use 'network' which will open a port and accept connections "
+              "via an rpc interface. SQL queries will be accepted and processed in the "
+              "same manner as with the local cli.");
+
 DECLARE_bool(profile_and_report_workorder_perf);
 DECLARE_bool(visualize_execution_dag);
 
@@ -152,6 +157,9 @@ DECLARE_bool(visualize_execution_dag);
 int main(int argc, char* argv[]) {
   google::InitGoogleLogging(argv[0]);
   gflags::ParseCommandLineFlags(&argc, &argv, true);
+#ifdef QUICKSTEP_CLI_NETWORK_IO_HPP_
+  grpc_init();
+#endif
 
   printf("Starting Quickstep with %d worker thread(s) and a %.2f GB buffer pool.\n",
          FLAGS_num_workers,
@@ -273,8 +281,19 @@ int main(int argc, char* argv[]) {
 
   foreman.start();
 
-  LineReaderImpl line_reader("quickstep> ",
-                             "      ...> ");
+  std::unique_ptr<IOInterface> io;
+  if (quickstep::FLAGS_mode == "network") {
+#ifdef QUICKSTEP_ENABLE_NETWORK_CLI
+    io.reset(new quickstep::NetworkIO);
+#else
+    LOG(FATAL) << "Quickstep must be compiled with '-D ENABLE_NETWORK_CLI=true' to use this feature.";
+#endif
+  } else if (quickstep::FLAGS_mode == "local") {
+    io.reset(new quickstep::LocalIO);
+  } else {
+    LOG(FATAL) << "unknown flag --mode value: " << quickstep::FLAGS_mode;
+  }
+
   std::unique_ptr<SqlParserWrapper> parser_wrapper(new SqlParserWrapper());
   std::chrono::time_point<std::chrono::steady_clock> start, end;
 
@@ -283,14 +302,16 @@ int main(int argc, char* argv[]) {
 #endif
   for (;;) {
     string *command_string = new string();
-    *command_string = line_reader.getNextCommand();
+    std::unique_ptr<quickstep::IOHandle> io_handle(io->getNextIOHandle());
+    *command_string = io_handle->getCommand();
+    LOG(INFO) << "Command received: " << *command_string;
     if (command_string->size() == 0) {
       delete command_string;
       break;
     }
 
     if (quickstep::FLAGS_print_query) {
-      printf("\n%s\n", command_string->c_str());
+      fprintf(io_handle->out(), "\n%s\n", command_string->c_str());
     }
 
     parser_wrapper->feedNextBuffer(command_string);
@@ -306,9 +327,7 @@ int main(int argc, char* argv[]) {
         if (statement.getStatementType() == ParseStatement::kQuit) {
           quitting = true;
           break;
-        }
-
-        if (statement.getStatementType() == ParseStatement::kCommand) {
+        } else if (statement.getStatementType() == ParseStatement::kCommand) {
           try {
             quickstep::cli::executeCommand(
                 statement,
@@ -318,16 +337,16 @@ int main(int argc, char* argv[]) {
                 &bus,
                 &storage_manager,
                 query_processor.get(),
-                stdout);
+                io_handle->out());
           } catch (const quickstep::SqlError &sql_error) {
-            fprintf(stderr, "%s",
+            fprintf(io_handle->err(), "%s",
                     sql_error.formatMessage(*command_string).c_str());
             reset_parser = true;
             break;
           }
           continue;
         }
-
+        // Here the statement is presumed to be a query.
         const std::size_t query_id = query_processor->query_id();
         const CatalogRelation *query_result_relation = nullptr;
         std::unique_ptr<quickstep::ExecutionDAGVisualizer> dag_visualizer;
@@ -353,7 +372,7 @@ int main(int argc, char* argv[]) {
               query_handle.release(),
               &bus);
         } catch (const quickstep::SqlError &sql_error) {
-          fprintf(stderr, "%s", sql_error.formatMessage(*command_string).c_str());
+          fprintf(io_handle->err(), "%s", sql_error.formatMessage(*command_string).c_str());
           reset_parser = true;
           break;
         }
@@ -366,11 +385,11 @@ int main(int argc, char* argv[]) {
           if (query_result_relation) {
             PrintToScreen::PrintRelation(*query_result_relation,
                                          &storage_manager,
-                                         stdout);
+                                         io_handle->out());
             PrintToScreen::PrintOutputSize(
                 *query_result_relation,
                 &storage_manager,
-                stdout);
+                io_handle->err());
 
             DropRelation::Drop(*query_result_relation,
                                query_processor->getDefaultDatabase(),
@@ -379,7 +398,7 @@ int main(int argc, char* argv[]) {
 
           query_processor->saveCatalog();
           std::chrono::duration<double, std::milli> time_ms = end - start;
-          printf("Time: %s ms\n",
+          fprintf(io_handle->out(), "Time: %s ms\n",
                  quickstep::DoubleToStringWithSignificantDigits(
                      time_ms.count(), 3).c_str());
           if (quickstep::FLAGS_profile_and_report_workorder_perf) {
@@ -393,12 +412,12 @@ int main(int argc, char* argv[]) {
             std::cerr << "\n" << dag_visualizer->toDOT() << "\n";
           }
         } catch (const std::exception &e) {
-          fprintf(stderr, "QUERY EXECUTION ERROR: %s\n", e.what());
+          fprintf(io_handle->err(), "QUERY EXECUTION ERROR: %s\n", e.what());
           break;
         }
       } else {
         if (result.condition == ParseResult::kError) {
-          fprintf(stderr, "%s", result.error_message.c_str());
+          fprintf(io_handle->err(), "%s", result.error_message.c_str());
         }
         reset_parser = true;
         break;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/tests/CMakeLists.txt b/cli/tests/CMakeLists.txt
index 7f8150f..f402ac0 100644
--- a/cli/tests/CMakeLists.txt
+++ b/cli/tests/CMakeLists.txt
@@ -31,6 +31,18 @@ if (ENABLE_DISTRIBUTED)
                  "${PROJECT_SOURCE_DIR}/utility/textbased_test/TextBasedTest.cpp"
                  "${PROJECT_SOURCE_DIR}/utility/textbased_test/TextBasedTest.hpp")
 endif(ENABLE_DISTRIBUTED)
+add_executable(LineReaderBuffered_unittest
+               LineReaderBuffered_unittest.cpp)
+if (ENABLE_NETWORK_CLI)
+  set(CMAKE_MODULE_PATH
+      ${CMAKE_MODULE_PATH}
+      "${PROJECT_SOURCE_DIR}/third_party/src/tmb/cmake")
+
+  find_package(Grpc++ REQUIRED)
+
+  add_executable(NetworkIO_unittest
+                 NetworkIO_unittest.cpp)
+endif()
 
 target_link_libraries(quickstep_cli_tests_CommandExecutorTest
                       glog
@@ -90,3 +102,26 @@ if (ENABLE_DISTRIBUTED)
                         ${GFLAGS_LIB_NAME}
                         ${LIBS})
 endif(ENABLE_DISTRIBUTED)
+
+target_link_libraries(LineReaderBuffered_unittest
+                      ${GFLAGS_LIB_NAME}
+                      glog
+                      gtest
+                      quickstep_cli_LineReaderBuffered)
+add_test(LineReaderBuffered_unittest LineReaderBuffered_unittest)
+
+if (ENABLE_NETWORK_CLI)
+  target_link_libraries(NetworkIO_unittest
+                        ${GFLAGS_LIB_NAME}
+                        ${GRPCPLUSPLUS_LIBRARIES}
+                        glog
+                        gtest
+                        quickstep_cli_Flags
+                        quickstep_cli_IOInterface
+                        quickstep_cli_LineReaderBuffered
+                        quickstep_cli_NetworkCliClient
+                        quickstep_cli_NetworkIO
+                        quickstep_threading_Thread
+                        quickstep_utility_Macros)
+  add_test(NetworkIO_unittest NetworkIO_unittest)
+endif()

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/tests/LineReaderBuffered_unittest.cpp
----------------------------------------------------------------------
diff --git a/cli/tests/LineReaderBuffered_unittest.cpp b/cli/tests/LineReaderBuffered_unittest.cpp
new file mode 100644
index 0000000..97d5a68
--- /dev/null
+++ b/cli/tests/LineReaderBuffered_unittest.cpp
@@ -0,0 +1,70 @@
+/**
+ * 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 <numeric>
+#include <string>
+#include <vector>
+
+#include "cli/LineReaderBuffered.hpp"
+
+#include "gflags/gflags.h"
+#include "glog/logging.h"
+#include "gtest/gtest.h"
+
+namespace quickstep {
+
+/**
+ * Create a string of several SQL statements and expect that they are individually parsed out.
+ */
+TEST(NetworkIOTest, TestLineReaderBuffered) {
+  LineReaderBuffered linereader;
+  EXPECT_TRUE(linereader.bufferEmpty());
+
+  std::vector<std::string> statements_orig = {"select * from foo;", "select 1;", "select 2;", "quit;"};
+  std::string statements_str;
+  statements_str = accumulate(statements_orig.begin(), statements_orig.end(), statements_str);
+  linereader.setBuffer(statements_str);
+  ASSERT_FALSE(linereader.bufferEmpty());
+
+  std::vector<std::string> statements_parsed;
+  std::size_t parsed_commands;
+  for (parsed_commands = 0;
+       parsed_commands < statements_orig.size() + 1 && !linereader.bufferEmpty();
+       parsed_commands++) {
+    std::string command = linereader.getNextCommand();
+    if (!command.empty()) {
+      statements_parsed.push_back(command);
+    }
+  }
+
+  ASSERT_EQ(statements_parsed.size(), statements_orig.size());
+  for (std::size_t i = 0; i < statements_parsed.size(); ++i) {
+    EXPECT_EQ(statements_orig[i], statements_parsed[i]);
+  }
+  EXPECT_TRUE(linereader.bufferEmpty());
+}
+
+}  // namespace quickstep
+
+int main(int argc, char** argv) {
+  google::InitGoogleLogging(argv[0]);
+  ::testing::InitGoogleTest(&argc, argv);
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
+  return RUN_ALL_TESTS();
+}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/cli/tests/NetworkIO_unittest.cpp
----------------------------------------------------------------------
diff --git a/cli/tests/NetworkIO_unittest.cpp b/cli/tests/NetworkIO_unittest.cpp
new file mode 100644
index 0000000..9dbd63e
--- /dev/null
+++ b/cli/tests/NetworkIO_unittest.cpp
@@ -0,0 +1,187 @@
+/**
+ * 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 <cstddef>
+#include <cstdio>
+#include <functional>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "cli/LineReaderBuffered.hpp"
+#include "cli/NetworkCliClient.hpp"
+#include "cli/NetworkIO.hpp"
+#include "threading/Thread.hpp"
+#include "utility/Macros.hpp"
+
+#include "gflags/gflags.h"
+#include "glog/logging.h"
+#include "gtest/gtest.h"
+
+namespace quickstep {
+
+using networkio_internal::RequestState;
+
+static std::string const kQueryRequest = "O Captain! My Captain!";
+static std::string const kQueryResponse = "Our fearful trip is done,";
+
+/**
+ * Contains a server instance for testing.
+ */
+class TestNetworkIO {
+ public:
+  TestNetworkIO() : server_address_(NetworkIO::GetAddress()) {
+    grpc::ServerBuilder builder;
+    builder.AddListeningPort(server_address_, grpc::InsecureServerCredentials());
+    builder.RegisterService(&service_);
+    server_ = builder.BuildAndStart();
+    CHECK(server_) << "Failed to start server";
+    LOG(INFO) << "TestSingleNodeServer listening on " << server_address_;
+  }
+
+  ~TestNetworkIO() {
+    service_.kill();
+    server_->Shutdown();
+    server_->Wait();
+  }
+
+  /**
+   * @brief Waits on the service for a sent message.
+   */
+  std::string getSentMessage() {
+    CHECK(current_request_ == nullptr);
+    current_request_ = service_.waitForRequest();
+    EXPECT_EQ(current_request_->getCanceled(), false);
+    return current_request_->getRequest();
+  }
+
+  /**
+   * @brief Sets the response message of the Service worker. Alerts it that the request is ready.
+   */
+  void setResponse(std::string response) {
+    CHECK_NOTNULL(current_request_);
+    current_request_->responseReady(response, std::string(""));
+    current_request_ = nullptr;
+  }
+
+  NetworkCliServiceImpl& getService() {
+    return service_;
+  }
+
+ private:
+  NetworkCliServiceImpl service_;
+  std::string server_address_;
+  std::unique_ptr<grpc::Server> server_;
+  RequestState* current_request_;
+
+  DISALLOW_COPY_AND_ASSIGN(TestNetworkIO);
+};
+
+/**
+ * We will pass this thread a lambda based on the desired server interactions.
+ */
+class HelperThread : public Thread {
+ public:
+  explicit HelperThread(std::function<void(void)> function) : function_(function) {}
+
+ protected:
+  void run() override {
+    function_();
+  }
+
+ private:
+  std::function<void(void)> function_;
+
+  DISALLOW_COPY_AND_ASSIGN(HelperThread);
+};
+
+/**
+ * Tests a simple call and response to the Service.
+ */
+TEST(NetworkIOTest, TestNetworkIOCommandInteraction) {
+  NetworkIO networkIO;
+
+  // This thread will handle the response from the client in a similar way as the quickstep cli will.
+  HelperThread server_handler([&networkIO]() {
+    std::unique_ptr<IOHandle> command(networkIO.getNextIOHandle());
+    EXPECT_EQ(command->getCommand(), kQueryRequest);
+
+    // Set some output for the main test thread, destruction of the handle will return the request.
+    fprintf(command->out(), "%s", kQueryResponse.c_str());
+  });
+  server_handler.start();
+
+  NetworkCliClient client(
+    grpc::CreateChannel(NetworkIO::GetAddress(),
+                        grpc::InsecureChannelCredentials()));
+  QueryRequest request;
+  request.set_query(kQueryRequest);
+  QueryResponse response;
+  Status status = client.SendQuery(request, &response);
+  ASSERT_TRUE(status.ok());
+  EXPECT_EQ(kQueryResponse, response.query_result());
+  EXPECT_TRUE(response.error_result().empty());
+
+  server_handler.join();
+}
+
+/**
+ * Tests that killing the service will cancel requests.
+ */
+TEST(NetworkIOTest, TestShutdown) {
+  // Start a server:
+  NetworkIO networkIO;
+
+  std::function<void(void)> send_request_fn([]() {
+    // Create a request, and, on return it should be canceled.
+    NetworkCliClient client(grpc::CreateChannel(NetworkIO::GetAddress(),
+                            grpc::InsecureChannelCredentials()));
+    QueryRequest request;
+    request.set_query(kQueryRequest);
+    QueryResponse response;
+    Status status = client.SendQuery(request, &response);
+    EXPECT_EQ(grpc::OK, status.error_code());
+
+    // Server will kill the next request.
+    status = client.SendQuery(request, &response);
+    EXPECT_EQ(grpc::CANCELLED, status.error_code());
+  });
+
+  HelperThread client_thread(send_request_fn);
+  client_thread.start();
+
+  {
+    std::unique_ptr<IOHandle> ioHandle(networkIO.getNextIOHandle());
+    EXPECT_EQ(ioHandle->getCommand(), kQueryRequest);
+    // Killing the service should cause the response the client thread receives to be canceled.
+    networkIO.killService();
+  }
+
+  client_thread.join();
+}
+
+}  // namespace quickstep
+
+int main(int argc, char** argv) {
+  google::InitGoogleLogging(argv[0]);
+  ::testing::InitGoogleTest(&argc, argv);
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
+  grpc_init();
+  return RUN_ALL_TESTS();
+}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/99198ef2/validate_cmakelists.py
----------------------------------------------------------------------
diff --git a/validate_cmakelists.py b/validate_cmakelists.py
index 9d1f530..0b2e79d 100755
--- a/validate_cmakelists.py
+++ b/validate_cmakelists.py
@@ -48,6 +48,7 @@ EXCLUDED_TOP_LEVEL_DIRS = ["build", "third_party"]
 IGNORED_DEPENDENCIES = frozenset(
     ["quickstep_cli_LineReaderDumb",
      "quickstep_cli_LineReaderLineNoise",
+     "quickstep_cli_NetworkCli.grpc_proto",
      "quickstep_storage_DataExchange.grpc_proto",
      "quickstep_threading_WinThreadsAPI",
      "quickstep_utility_textbasedtest_TextBasedTest",


[09/32] incubator-quickstep git commit: Add ThreadPrivateCompactKeyHashTable as a fast path data structure for aggregation.

Posted by ji...@apache.org.
Add ThreadPrivateCompactKeyHashTable as a fast path data structure for aggregation.


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

Branch: refs/heads/new-op
Commit: d6a01e7c867354ca05545595644a62b03de56b81
Parents: 8169306
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Fri Apr 21 23:23:13 2017 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Mon Apr 24 13:40:17 2017 -0500

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp          |  14 +-
 .../cost_model/StarSchemaSimpleCostModel.cpp    |  71 +++-
 .../cost_model/StarSchemaSimpleCostModel.hpp    |  12 +
 storage/AggregationOperationState.cpp           |  68 ++-
 storage/AggregationOperationState.hpp           |   6 +-
 storage/CMakeLists.txt                          |  24 ++
 storage/CollisionFreeVectorTable.hpp            |   4 +
 storage/HashTable.proto                         |   1 +
 storage/HashTableBase.hpp                       |  18 +-
 storage/HashTableFactory.hpp                    |  13 +-
 storage/HashTablePool.hpp                       |   9 +
 storage/PackedPayloadHashTable.hpp              |   4 +
 storage/ThreadPrivateCompactKeyHashTable.cpp    | 421 +++++++++++++++++++
 storage/ThreadPrivateCompactKeyHashTable.hpp    | 230 ++++++++++
 14 files changed, 870 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 3e0f647..9625a91 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -1580,14 +1580,22 @@ void ExecutionGenerator::convertAggregate(
             ->canUseCollisionFreeAggregation(physical_plan,
                                              estimated_num_groups,
                                              &max_num_groups)) {
+      // First option: use array-based aggregation if applicable.
       aggr_state_proto->set_hash_table_impl_type(
           serialization::HashTableImplType::COLLISION_FREE_VECTOR);
       aggr_state_proto->set_estimated_num_entries(max_num_groups);
       use_parallel_initialization = true;
     } else {
-      // Otherwise, use SeparateChaining.
-      aggr_state_proto->set_hash_table_impl_type(
-          serialization::HashTableImplType::SEPARATE_CHAINING);
+      if (cost_model_for_aggregation_->canUseTwoPhaseCompactKeyAggregation(
+              physical_plan, estimated_num_groups)) {
+        // Second option: use thread-private compact-key aggregation if applicable.
+        aggr_state_proto->set_hash_table_impl_type(
+            serialization::HashTableImplType::THREAD_PRIVATE_COMPACT_KEY);
+      } else {
+        // Otherwise, use SeparateChaining.
+        aggr_state_proto->set_hash_table_impl_type(
+            serialization::HashTableImplType::SEPARATE_CHAINING);
+      }
       aggr_state_proto->set_estimated_num_entries(std::max(16uL, estimated_num_groups));
     }
   } else {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index b17fac0..e0e3dff 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -71,8 +71,8 @@ namespace optimizer {
 namespace cost {
 
 DEFINE_int64(collision_free_vector_table_max_size, 1000000000,
-              "The maximum allowed key range (number of entries) for using a "
-              "CollisionFreeVectorTable.");
+             "The maximum allowed key range (number of entries) for using a "
+             "CollisionFreeVectorTable.");
 
 namespace E = ::quickstep::optimizer::expressions;
 namespace P = ::quickstep::optimizer::physical;
@@ -700,6 +700,73 @@ bool StarSchemaSimpleCostModel::canUseCollisionFreeAggregation(
   return true;
 }
 
+bool StarSchemaSimpleCostModel::canUseTwoPhaseCompactKeyAggregation(
+    const physical::AggregatePtr &aggregate,
+    const std::size_t estimated_num_groups) {
+  // Require estimated number of groups to be below the specified threshold.
+  //
+  // TODO(jianqiao): It is good to have the threshold to be the same as
+  // FLAGS_partition_aggregation_num_groups_threshold which is defined in
+  // AggregationOperationState.cpp. However, there seems to be no sound place
+  // to put that flag so that it can be shared by the two cpp files (optimizer
+  // vs backend). So here we hardcode the threshold and leave it to be solved
+  // later.
+  if (estimated_num_groups >= 10000u) {
+    return false;
+  }
+
+  // Require fix-length non-nullable keys that can be packed into a 64-bit QWORD.
+  std::size_t total_key_size = 0;
+  for (const auto &key_expr : aggregate->grouping_expressions()) {
+    const Type &type = key_expr->getValueType();
+    if (type.isVariableLength() || type.isNullable()) {
+      return false;
+    }
+    total_key_size += type.maximumByteLength();
+  }
+
+  if (total_key_size > sizeof(std::uint64_t)) {
+    return false;
+  }
+
+  // Check aggregate arguments.
+  for (const auto &agg_alias : aggregate->aggregate_expressions()) {
+    const E::AggregateFunctionPtr agg_expr =
+        std::static_pointer_cast<const E::AggregateFunction>(agg_alias->expression());
+
+    // Not supporting DISTINCT aggregation.
+    if (agg_expr->is_distinct()) {
+      return false;
+    }
+
+    // Currently we do not handle NULL values.
+    const auto &arguments = agg_expr->getArguments();
+    for (const auto &arg : arguments) {
+      if (arg->getValueType().isNullable()) {
+        return false;
+      }
+    }
+
+    // Restricted to COUNT/SUM with INT/LONG/FLOAT/DOUBLE arguments.
+    switch (agg_expr->getAggregate().getAggregationID()) {
+      case AggregationID::kCount:
+        break;
+      case AggregationID::kSum: {
+        DCHECK_EQ(1u, arguments.size());
+        if (!QUICKSTEP_EQUALS_ANY_CONSTANT(arguments.front()->getValueType().getTypeID(),
+                                           kInt, kLong, kFloat, kDouble)) {
+          return false;
+        }
+        break;
+      }
+      default:
+        return false;
+    }
+  }
+
+  return true;
+}
+
 }  // namespace cost
 }  // namespace optimizer
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
index 0461077..99518cf 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
@@ -184,6 +184,18 @@ class StarSchemaSimpleCostModel : public CostModel {
                                       const std::size_t estimated_num_groups,
                                       std::size_t *max_num_groups);
 
+  /**
+   * @brief Checks whether an aggregate node can be efficiently evaluated with
+   *        the two-phase compact key aggregation fast path.
+   *
+   * @param aggregate The physical aggregate node to be checked.
+   * @param estimated_num_groups The estimated number of groups for the aggregate.
+   * @return A bool value indicating whether two-phase compact key aggregation
+   *         can be used to evaluate \p aggregate.
+   */
+  bool canUseTwoPhaseCompactKeyAggregation(const physical::AggregatePtr &aggregate,
+                                           const std::size_t estimated_num_groups);
+
  private:
   std::size_t estimateCardinalityForAggregate(
       const physical::AggregatePtr &physical_plan);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/storage/AggregationOperationState.cpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.cpp b/storage/AggregationOperationState.cpp
index e5dc93e..0f4a105 100644
--- a/storage/AggregationOperationState.cpp
+++ b/storage/AggregationOperationState.cpp
@@ -48,6 +48,7 @@
 #include "storage/StorageBlockInfo.hpp"
 #include "storage/StorageManager.hpp"
 #include "storage/SubBlocksReference.hpp"
+#include "storage/ThreadPrivateCompactKeyHashTable.hpp"
 #include "storage/TupleIdSequence.hpp"
 #include "storage/TupleStorageSubBlock.hpp"
 #include "storage/ValueAccessor.hpp"
@@ -69,10 +70,10 @@ namespace quickstep {
 DEFINE_int32(num_aggregation_partitions,
              41,
              "The number of partitions used for performing the aggregation");
-DEFINE_int32(partition_aggregation_num_groups_threshold,
-             500000,
-             "The threshold used for deciding whether the aggregation is done "
-             "in a partitioned way or not");
+DEFINE_uint64(partition_aggregation_num_groups_threshold,
+              100000,
+              "The threshold used for deciding whether the aggregation is done "
+              "in a partitioned way or not");
 
 AggregationOperationState::AggregationOperationState(
     const CatalogRelationSchema &input_relation,
@@ -94,11 +95,16 @@ AggregationOperationState::AggregationOperationState(
                                     !is_distinct_.empty(), std::logical_and<bool>())),
       storage_manager_(storage_manager) {
   if (!group_by.empty()) {
-    if (hash_table_impl_type == HashTableImplType::kCollisionFreeVector) {
-      is_aggregate_collision_free_ = true;
-    } else {
-      is_aggregate_partitioned_ = checkAggregatePartitioned(
-          estimated_num_entries, is_distinct_, group_by, aggregate_functions);
+    switch (hash_table_impl_type) {
+      case HashTableImplType::kCollisionFreeVector:
+        is_aggregate_collision_free_ = true;
+        break;
+      case HashTableImplType::kThreadPrivateCompactKey:
+        is_aggregate_partitioned_ = false;
+        break;
+      default:
+        is_aggregate_partitioned_ = checkAggregatePartitioned(
+            estimated_num_entries, is_distinct_, group_by, aggregate_functions);
     }
   }
 
@@ -420,9 +426,7 @@ bool AggregationOperationState::checkAggregatePartitioned(
 
   // There are GROUP BYs without DISTINCT. Check if the estimated number of
   // groups is large enough to warrant a partitioned aggregation.
-  return estimated_num_groups >=
-         static_cast<std::size_t>(
-             FLAGS_partition_aggregation_num_groups_threshold);
+  return estimated_num_groups >= FLAGS_partition_aggregation_num_groups_threshold;
 }
 
 std::size_t AggregationOperationState::getNumInitializationPartitions() const {
@@ -715,7 +719,18 @@ void AggregationOperationState::finalizeHashTable(
     finalizeHashTableImplPartitioned(partition_id, output_destination);
   } else {
     DCHECK_EQ(0u, partition_id);
-    finalizeHashTableImplThreadPrivate(output_destination);
+    DCHECK(group_by_hashtable_pool_ != nullptr);
+    switch (group_by_hashtable_pool_->getHashTableImplType()) {
+      case HashTableImplType::kSeparateChaining:
+        finalizeHashTableImplThreadPrivatePackedPayload(output_destination);
+        break;
+      case HashTableImplType::kThreadPrivateCompactKey:
+        finalizeHashTableImplThreadPrivateCompactKey(output_destination);
+        break;
+      default:
+        LOG(FATAL) << "Unexpected hash table type in "
+                   << "AggregationOperationState::finalizeHashTable()";
+    }
   }
 }
 
@@ -840,7 +855,7 @@ void AggregationOperationState::finalizeHashTableImplPartitioned(
   output_destination->bulkInsertTuples(&complete_result);
 }
 
-void AggregationOperationState::finalizeHashTableImplThreadPrivate(
+void AggregationOperationState::finalizeHashTableImplThreadPrivatePackedPayload(
     InsertDestination *output_destination) {
   // TODO(harshad) - The merge phase may be slower when each hash table contains
   // large number of entries. We should find ways in which we can perform a
@@ -948,6 +963,31 @@ void AggregationOperationState::finalizeHashTableImplThreadPrivate(
   output_destination->bulkInsertTuples(&complete_result);
 }
 
+void AggregationOperationState::finalizeHashTableImplThreadPrivateCompactKey(
+    InsertDestination *output_destination) {
+  auto *hash_tables = group_by_hashtable_pool_->getAllHashTables();
+  DCHECK(hash_tables != nullptr);
+  if (hash_tables->empty()) {
+    return;
+  }
+
+  // Merge all hash tables into one.
+  std::unique_ptr<ThreadPrivateCompactKeyHashTable> final_hash_table(
+      static_cast<ThreadPrivateCompactKeyHashTable*>(hash_tables->back().release()));
+  for (std::size_t i = 0; i < hash_tables->size() - 1; ++i) {
+    std::unique_ptr<AggregationStateHashTableBase> hash_table(
+        hash_tables->at(i).release());
+    final_hash_table->mergeFrom(
+        static_cast<const ThreadPrivateCompactKeyHashTable&>(*hash_table));
+  }
+
+  ColumnVectorsValueAccessor complete_result;
+  final_hash_table->finalize(&complete_result);
+
+  // Bulk-insert the complete result.
+  output_destination->bulkInsertTuples(&complete_result);
+}
+
 std::size_t AggregationOperationState::getMemoryConsumptionBytes() const {
   std::size_t memory = getMemoryConsumptionBytesHelper(distinctify_hashtables_);
   memory += getMemoryConsumptionBytesHelper(group_by_hashtables_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/storage/AggregationOperationState.hpp
----------------------------------------------------------------------
diff --git a/storage/AggregationOperationState.hpp b/storage/AggregationOperationState.hpp
index e6af494..207c4f0 100644
--- a/storage/AggregationOperationState.hpp
+++ b/storage/AggregationOperationState.hpp
@@ -256,7 +256,11 @@ class AggregationOperationState {
   void finalizeHashTableImplPartitioned(const std::size_t partition_id,
                                         InsertDestination *output_destination);
 
-  void finalizeHashTableImplThreadPrivate(InsertDestination *output_destination);
+  void finalizeHashTableImplThreadPrivatePackedPayload(
+      InsertDestination *output_destination);
+
+  void finalizeHashTableImplThreadPrivateCompactKey(
+      InsertDestination *output_destination);
 
   std::size_t getMemoryConsumptionBytesHelper(
       const std::vector<std::unique_ptr<AggregationStateHashTableBase>>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/storage/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/storage/CMakeLists.txt b/storage/CMakeLists.txt
index c3db584..4296ba0 100644
--- a/storage/CMakeLists.txt
+++ b/storage/CMakeLists.txt
@@ -250,6 +250,9 @@ add_library(quickstep_storage_StorageManager StorageManager.cpp StorageManager.h
 add_library(quickstep_storage_SubBlockTypeRegistry SubBlockTypeRegistry.cpp SubBlockTypeRegistry.hpp)
 add_library(quickstep_storage_SubBlockTypeRegistryMacros ../empty_src.cpp SubBlockTypeRegistryMacros.hpp)
 add_library(quickstep_storage_SubBlocksReference ../empty_src.cpp SubBlocksReference.hpp)
+add_library(quickstep_storage_ThreadPrivateCompactKeyHashTable
+            ThreadPrivateCompactKeyHashTable.cpp
+            ThreadPrivateCompactKeyHashTable.hpp)
 add_library(quickstep_storage_TupleIdSequence ../empty_src.cpp TupleIdSequence.hpp)
 add_library(quickstep_storage_TupleReference ../empty_src.cpp TupleReference.hpp)
 add_library(quickstep_storage_TupleStorageSubBlock TupleStorageSubBlock.cpp TupleStorageSubBlock.hpp)
@@ -288,6 +291,7 @@ target_link_libraries(quickstep_storage_AggregationOperationState
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_StorageManager
                       quickstep_storage_SubBlocksReference
+                      quickstep_storage_ThreadPrivateCompactKeyHashTable
                       quickstep_storage_TupleIdSequence
                       quickstep_storage_TupleStorageSubBlock
                       quickstep_storage_ValueAccessor
@@ -724,6 +728,7 @@ target_link_libraries(quickstep_storage_HashTableFactory
                       quickstep_storage_PackedPayloadHashTable
                       quickstep_storage_SeparateChainingHashTable
                       quickstep_storage_SimpleScalarSeparateChainingHashTable
+                      quickstep_storage_ThreadPrivateCompactKeyHashTable
                       quickstep_storage_TupleReference
                       quickstep_types_Type
                       quickstep_types_TypeFactory
@@ -1039,6 +1044,24 @@ target_link_libraries(quickstep_storage_SubBlockTypeRegistry
 target_link_libraries(quickstep_storage_SubBlocksReference
                       glog
                       quickstep_utility_PtrVector)
+target_link_libraries(quickstep_storage_ThreadPrivateCompactKeyHashTable
+                      glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_aggregation_AggregationHandle
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_storage_HashTableBase
+                      quickstep_storage_StorageBlob
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_StorageConstants
+                      quickstep_storage_StorageManager
+                      quickstep_storage_ValueAccessorMultiplexer
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_types_Type
+                      quickstep_types_TypeID
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_containers_ColumnVectorsValueAccessor
+                      quickstep_utility_Macros
+                      quickstep_utility_ScopedBuffer)
 target_link_libraries(quickstep_storage_TupleIdSequence
                       quickstep_storage_StorageBlockInfo
                       quickstep_utility_BitVector
@@ -1164,6 +1187,7 @@ target_link_libraries(quickstep_storage
                       quickstep_storage_SubBlockTypeRegistry
                       quickstep_storage_SubBlockTypeRegistryMacros
                       quickstep_storage_SubBlocksReference
+                      quickstep_storage_ThreadPrivateCompactKeyHashTable
                       quickstep_storage_TupleIdSequence
                       quickstep_storage_TupleReference
                       quickstep_storage_TupleStorageSubBlock

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/storage/CollisionFreeVectorTable.hpp
----------------------------------------------------------------------
diff --git a/storage/CollisionFreeVectorTable.hpp b/storage/CollisionFreeVectorTable.hpp
index 490a5cc..221a221 100644
--- a/storage/CollisionFreeVectorTable.hpp
+++ b/storage/CollisionFreeVectorTable.hpp
@@ -70,6 +70,10 @@ class CollisionFreeVectorTable : public AggregationStateHashTableBase {
 
   ~CollisionFreeVectorTable() override;
 
+  HashTableImplType getImplType() const override {
+    return HashTableImplType::kCollisionFreeVector;
+  }
+
   void destroyPayload() override;
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/storage/HashTable.proto
----------------------------------------------------------------------
diff --git a/storage/HashTable.proto b/storage/HashTable.proto
index 6839ebc..ed383df 100644
--- a/storage/HashTable.proto
+++ b/storage/HashTable.proto
@@ -26,6 +26,7 @@ enum HashTableImplType {
   LINEAR_OPEN_ADDRESSING = 1;
   SEPARATE_CHAINING = 2;
   SIMPLE_SCALAR_SEPARATE_CHAINING = 3;
+  THREAD_PRIVATE_COMPACT_KEY = 4;
 }
 
 // NOTE(chasseur): This proto describes the run-time parameters for a resizable

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/storage/HashTableBase.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableBase.hpp b/storage/HashTableBase.hpp
index 8be388a..4d9310c 100644
--- a/storage/HashTableBase.hpp
+++ b/storage/HashTableBase.hpp
@@ -44,7 +44,8 @@ enum class HashTableImplType {
   kCollisionFreeVector,
   kLinearOpenAddressing,
   kSeparateChaining,
-  kSimpleScalarSeparateChaining
+  kSimpleScalarSeparateChaining,
+  kThreadPrivateCompactKey
 };
 
 /**
@@ -113,8 +114,23 @@ class AggregationStateHashTableBase {
       const std::vector<MultiSourceAttributeId> &key_attr_ids,
       const ValueAccessorMultiplexer &accessor_mux) = 0;
 
+  /**
+   * @brief Destroy hash table payloads.
+   */
   virtual void destroyPayload() = 0;
 
+  /**
+   * @brief Get the implementation type of this aggregation hash table.
+   *
+   * @return The implementation type of this aggregation hash table.
+   */
+  virtual HashTableImplType getImplType() const = 0;
+
+  /**
+   * @brief Get the estimated memory consumption of this hash table in bytes.
+   *
+   * @return The estimated memory consumption of this hash table in bytes.
+   */
   virtual std::size_t getMemoryConsumptionBytes() const = 0;
 
  protected:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/storage/HashTableFactory.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTableFactory.hpp b/storage/HashTableFactory.hpp
index 9686429..cb1f16f 100644
--- a/storage/HashTableFactory.hpp
+++ b/storage/HashTableFactory.hpp
@@ -32,6 +32,7 @@
 #include "storage/PackedPayloadHashTable.hpp"
 #include "storage/SeparateChainingHashTable.hpp"
 #include "storage/SimpleScalarSeparateChainingHashTable.hpp"
+#include "storage/ThreadPrivateCompactKeyHashTable.hpp"
 #include "storage/TupleReference.hpp"
 #include "types/TypeFactory.hpp"
 #include "utility/BloomFilter.hpp"
@@ -123,6 +124,8 @@ inline HashTableImplType HashTableImplTypeFromProto(
       return HashTableImplType::kSeparateChaining;
     case serialization::HashTableImplType::SIMPLE_SCALAR_SEPARATE_CHAINING:
       return HashTableImplType::kSimpleScalarSeparateChaining;
+    case serialization::HashTableImplType::THREAD_PRIVATE_COMPACT_KEY:
+      return HashTableImplType::kThreadPrivateCompactKey;
     default: {
       LOG(FATAL) << "Unrecognized serialization::HashTableImplType\n";
     }
@@ -355,7 +358,6 @@ class AggregationStateHashTableFactory {
    *        hash table constructor.
    * @return A new aggregation state hash table.
    **/
-
   static AggregationStateHashTableBase* CreateResizable(
       const HashTableImplType hash_table_type,
       const std::vector<const Type*> &key_types,
@@ -363,13 +365,16 @@ class AggregationStateHashTableFactory {
       const std::vector<AggregationHandle *> &handles,
       StorageManager *storage_manager) {
     switch (hash_table_type) {
-      case HashTableImplType::kSeparateChaining:
-        return new PackedPayloadHashTable(
-            key_types, num_entries, handles, storage_manager);
       case HashTableImplType::kCollisionFreeVector:
         DCHECK_EQ(1u, key_types.size());
         return new CollisionFreeVectorTable(
             key_types.front(), num_entries, handles, storage_manager);
+      case HashTableImplType::kSeparateChaining:
+        return new PackedPayloadHashTable(
+            key_types, num_entries, handles, storage_manager);
+      case HashTableImplType::kThreadPrivateCompactKey:
+        return new ThreadPrivateCompactKeyHashTable(
+            key_types, num_entries, handles, storage_manager);
       default: {
         LOG(FATAL) << "Unrecognized HashTableImplType in "
                    << "AggregationStateHashTableFactory::createResizable()";

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/storage/HashTablePool.hpp
----------------------------------------------------------------------
diff --git a/storage/HashTablePool.hpp b/storage/HashTablePool.hpp
index 6dbd7f9..f3abddb 100644
--- a/storage/HashTablePool.hpp
+++ b/storage/HashTablePool.hpp
@@ -76,6 +76,15 @@ class HashTablePool {
         storage_manager_(DCHECK_NOTNULL(storage_manager)) {}
 
   /**
+   * @brief Get the type of hash table implementation for this pool.
+   *
+   * @return The type of hash table implementation for this pool.
+   */
+  HashTableImplType getHashTableImplType() const {
+    return hash_table_impl_type_;
+  }
+
+  /**
    * @brief Check out a hash table for insertion.
    *
    * @note This method is relevant for specialized (for aggregation)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/storage/PackedPayloadHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/PackedPayloadHashTable.hpp b/storage/PackedPayloadHashTable.hpp
index 960d5a7..3e89aab 100644
--- a/storage/PackedPayloadHashTable.hpp
+++ b/storage/PackedPayloadHashTable.hpp
@@ -88,6 +88,10 @@ class PackedPayloadHashTable : public AggregationStateHashTableBase {
 
   ~PackedPayloadHashTable() override;
 
+  HashTableImplType getImplType() const override {
+    return HashTableImplType::kSeparateChaining;
+  }
+
   /**
    * @brief Erase all entries in this hash table.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/storage/ThreadPrivateCompactKeyHashTable.cpp
----------------------------------------------------------------------
diff --git a/storage/ThreadPrivateCompactKeyHashTable.cpp b/storage/ThreadPrivateCompactKeyHashTable.cpp
new file mode 100644
index 0000000..fb68940
--- /dev/null
+++ b/storage/ThreadPrivateCompactKeyHashTable.cpp
@@ -0,0 +1,421 @@
+/**
+ * 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/ThreadPrivateCompactKeyHashTable.hpp"
+
+#include <algorithm>
+#include <cstddef>
+#include <cstdint>
+#include <type_traits>
+#include <vector>
+
+#include "expressions/aggregation/AggregationHandle.hpp"
+#include "expressions/aggregation/AggregationID.hpp"
+#include "storage/StorageBlob.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "storage/StorageManager.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
+#include "types/containers/ColumnVectorsValueAccessor.hpp"
+#include "utility/ScopedBuffer.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+namespace {
+
+#define CASE_KEY_SIZE(value) \
+  case value: return functor(std::integral_constant<std::size_t, value>())
+
+template <typename FunctorT>
+auto InvokeOnKeySize(const std::size_t key_size, const FunctorT &functor) {
+  switch (key_size) {
+    CASE_KEY_SIZE(1);
+    CASE_KEY_SIZE(2);
+    CASE_KEY_SIZE(3);
+    CASE_KEY_SIZE(4);
+    CASE_KEY_SIZE(5);
+    CASE_KEY_SIZE(6);
+    CASE_KEY_SIZE(7);
+    CASE_KEY_SIZE(8);
+    default:
+      break;
+  }
+  LOG(FATAL) << "Unexpected key size: " << key_size;
+}
+
+#undef CASE_KEY_SIZE
+
+}  // namespace
+
+constexpr std::size_t ThreadPrivateCompactKeyHashTable::kKeyCodeSize;
+
+ThreadPrivateCompactKeyHashTable::ThreadPrivateCompactKeyHashTable(
+    const std::vector<const Type*> &key_types,
+    const std::size_t num_entries,
+    const std::vector<AggregationHandle*> &handles,
+    StorageManager *storage_manager)
+    : key_types_(key_types),
+      handles_(handles),
+      total_state_size_(0),
+      num_buckets_(0),
+      buckets_allocated_(0),
+      storage_manager_(storage_manager) {
+  // Cache key sizes.
+  for (const Type *key_type : key_types) {
+    DCHECK(!key_type->isVariableLength());
+    DCHECK(!key_type->isNullable());
+    key_sizes_.emplace_back(key_type->maximumByteLength());
+  }
+
+  for (const AggregationHandle *handle : handles) {
+    const std::vector<const Type*> arg_types = handle->getArgumentTypes();
+    DCHECK_LE(arg_types.size(), 1u);
+    DCHECK(arg_types.empty() || !arg_types.front()->isNullable());
+
+    // Figure out state size.
+    std::size_t state_size = 0;
+    switch (handle->getAggregationID()) {
+      case AggregationID::kCount: {
+        state_size = sizeof(std::int64_t);
+        break;
+      }
+      case AggregationID::kSum: {
+        DCHECK_EQ(1u, arg_types.size());
+        switch (arg_types.front()->getTypeID()) {
+          case TypeID::kInt:  // Fall through
+          case TypeID::kLong:
+            state_size = sizeof(std::int64_t);
+            break;
+          case TypeID::kFloat:  // Fall through
+          case TypeID::kDouble:
+            state_size = sizeof(double);
+            break;
+          default:
+            LOG(FATAL) << "Unexpected argument type";
+        }
+        break;
+      }
+      default:
+        LOG(FATAL) << "Unexpected AggregationID";
+    }
+    state_sizes_.emplace_back(state_size);
+    total_state_size_ += state_size;
+  }
+
+  // Calculate required memory size for keys and states.
+  const std::size_t required_memory =
+      num_entries * (kKeyCodeSize + total_state_size_);
+  const std::size_t num_storage_slots =
+      storage_manager_->SlotsNeededForBytes(required_memory);
+
+  // Use storage manager to allocate memory.
+  const block_id blob_id = storage_manager->createBlob(num_storage_slots);
+  blob_ = storage_manager->getBlobMutable(blob_id);
+
+  num_buckets_ = blob_->size() / (kKeyCodeSize + total_state_size_);
+  void *memory = blob_->getMemoryMutable();
+
+  // Calculate the memory locations of state vectors.
+  keys_ = static_cast<KeyCode*>(memory);
+  char *state_memory = static_cast<char*>(memory) + num_buckets_ * kKeyCodeSize;
+  std::memset(state_memory, 0, num_buckets_ * total_state_size_);
+
+  for (std::size_t i = 0; i < state_sizes_.size(); ++i) {
+    state_vecs_.emplace_back(state_memory);
+    state_memory += num_buckets_ * state_sizes_[i];
+  }
+}
+
+ThreadPrivateCompactKeyHashTable::~ThreadPrivateCompactKeyHashTable() {
+  // Release the blob.
+  if (blob_.valid()) {
+    const block_id blob_id = blob_->getID();
+    blob_.release();
+    storage_manager_->deleteBlockOrBlobFile(blob_id);
+  }
+}
+
+void ThreadPrivateCompactKeyHashTable::resize() {
+  DCHECK_EQ(buckets_allocated_, num_buckets_);
+
+  const std::size_t resized_memory_size =
+      num_buckets_ * 2 * (kKeyCodeSize + total_state_size_);
+  const std::size_t resized_num_slots =
+      storage_manager_->SlotsNeededForBytes(resized_memory_size);
+
+  const block_id resized_blob_id =
+      storage_manager_->createBlob(resized_num_slots);
+  MutableBlobReference resized_blob =
+      storage_manager_->getBlobMutable(resized_blob_id);
+
+  const std::size_t resized_num_buckets =
+      resized_blob->size() / (kKeyCodeSize + total_state_size_);
+  void *resized_memory = resized_blob->getMemoryMutable();
+
+  KeyCode *resized_keys = static_cast<KeyCode*>(resized_memory);
+  std::memcpy(resized_keys, keys_, buckets_allocated_ * kKeyCodeSize);
+  keys_ = resized_keys;
+
+  char *resized_state_memory =
+      static_cast<char*>(resized_memory) + resized_num_buckets * kKeyCodeSize;
+  for (std::size_t i = 0; i < state_sizes_.size(); ++i) {
+    const std::size_t vec_size = buckets_allocated_ * state_sizes_[i];
+    const std::size_t resized_vec_size = resized_num_buckets * state_sizes_[i];
+
+    std::memcpy(resized_state_memory, state_vecs_[i], vec_size);
+    std::memset(resized_state_memory + vec_size,
+                0,
+                resized_vec_size - vec_size);
+
+    state_vecs_[i] = resized_state_memory;
+    resized_state_memory += resized_vec_size;
+  }
+
+  std::swap(blob_, resized_blob);
+  num_buckets_ = resized_num_buckets;
+
+  const block_id blob_id_to_delete = resized_blob->getID();
+  resized_blob.release();
+  storage_manager_->deleteBlockOrBlobFile(blob_id_to_delete);
+}
+
+bool ThreadPrivateCompactKeyHashTable::upsertValueAccessorCompositeKey(
+    const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+    const std::vector<MultiSourceAttributeId> &key_attr_ids,
+    const ValueAccessorMultiplexer &accessor_mux) {
+  ValueAccessor *base_accessor = accessor_mux.getBaseAccessor();
+  ValueAccessor *derived_accessor = accessor_mux.getDerivedAccessor();
+
+  DCHECK(base_accessor != nullptr);
+  const std::size_t num_tuples = base_accessor->getNumTuplesVirtual();
+
+  ScopedBuffer buffer(num_tuples * kKeyCodeSize);
+  KeyCode *key_codes = static_cast<KeyCode*>(buffer.get());
+  std::size_t key_code_offset = 0;
+  for (std::size_t i = 0; i < key_attr_ids.size(); ++i) {
+    const auto &key_attr_id = key_attr_ids[i];
+    ValueAccessor *accessor =
+        key_attr_id.source == ValueAccessorSource::kBase
+            ? base_accessor
+            : derived_accessor;
+    DCHECK(accessor != nullptr);
+
+    // Pack the key component into the 64-bit code (with proper offset).
+    InvokeOnKeySize(
+        key_sizes_[i],
+        [&](auto key_size) -> void {  // NOLINT(build/c++11)
+      ConstructKeyCode<decltype(key_size)::value>(
+          key_code_offset, key_attr_id.attr_id, accessor, key_codes);
+    });
+    key_code_offset += key_sizes_[i];
+  }
+
+  std::vector<BucketIndex> bucket_indices(num_tuples);
+  for (std::size_t i = 0; i < num_tuples; ++i) {
+    const std::size_t code = key_codes[i];
+    const auto index_it = index_.find(code);
+    if (index_it == index_.end()) {
+      if (buckets_allocated_ >= num_buckets_) {
+        resize();
+      }
+      index_.emplace(code, buckets_allocated_);
+      bucket_indices[i] = buckets_allocated_;
+      keys_[buckets_allocated_] = code;
+      ++buckets_allocated_;
+    } else {
+      bucket_indices[i] = index_it->second;
+    }
+  }
+
+  // Dispatch on AggregationID and argument type.
+  // TODO(jianqiao): refactor type system and aggregation facilities to eliminate
+  // this type of ad-hoc switch statements.
+  for (std::size_t i = 0; i < handles_.size(); ++i) {
+    const AggregationHandle *handle = handles_[i];
+    switch (handle->getAggregationID()) {
+      case AggregationID::kCount: {
+        upsertValueAccessorCount(bucket_indices, state_vecs_[i]);
+        break;
+      }
+      case AggregationID::kSum: {
+        DCHECK_EQ(1u, argument_ids[i].size());
+        const auto &argument_id = argument_ids[i].front();
+        ValueAccessor *accessor =
+            argument_id.source == ValueAccessorSource::kBase
+                ? base_accessor
+                : derived_accessor;
+        DCHECK(accessor != nullptr);
+
+        DCHECK_EQ(1u, handle->getArgumentTypes().size());
+        const Type *argument_type = handle->getArgumentTypes().front();
+        switch (argument_type->getTypeID()) {
+          case kInt: {
+            upsertValueAccessorSum<int, std::int64_t>(
+                bucket_indices, argument_id.attr_id, accessor, state_vecs_[i]);
+            break;
+          }
+          case kLong: {
+            upsertValueAccessorSum<std::int64_t, std::int64_t>(
+                bucket_indices, argument_id.attr_id, accessor, state_vecs_[i]);
+            break;
+          }
+          case kFloat: {
+            upsertValueAccessorSum<float, double>(
+                bucket_indices, argument_id.attr_id, accessor, state_vecs_[i]);
+            break;
+          }
+          case kDouble: {
+            upsertValueAccessorSum<double, double>(
+                bucket_indices, argument_id.attr_id, accessor, state_vecs_[i]);
+            break;
+          }
+          default:
+            LOG(FATAL) << "Unexpected argument type";
+        }
+        break;
+      }
+      default:
+        LOG(FATAL) << "Unexpected AggregationID";
+    }
+  }
+
+  return true;
+}
+
+void ThreadPrivateCompactKeyHashTable::mergeFrom(
+    const ThreadPrivateCompactKeyHashTable &source) {
+  // First merge keys and generate location mappings. That is, source hash
+  // table's bucket *i* should be merged into destination hash table's bucket
+  // *dst_bucket_indices[i]*.
+  std::vector<BucketIndex> dst_bucket_indices(source.buckets_allocated_);
+  const KeyCode *src_keys = source.keys_;
+  for (std::size_t i = 0; i < source.buckets_allocated_; ++i) {
+    const KeyCode code = src_keys[i];
+    const auto index_it = index_.find(code);
+
+    if (index_it == index_.end()) {
+      if (buckets_allocated_ >= num_buckets_) {
+        resize();
+      }
+      index_.emplace(code, buckets_allocated_);
+      dst_bucket_indices[i] = buckets_allocated_;
+      keys_[buckets_allocated_] = code;
+      ++buckets_allocated_;
+    } else {
+      dst_bucket_indices[i] = index_it->second;
+    }
+  }
+
+  // Then merge states in a column-wise way based on dst_bucket_indices.
+  for (std::size_t i = 0; i < handles_.size(); ++i) {
+    const AggregationHandle *handle = handles_[i];
+    switch (handle->getAggregationID()) {
+      case AggregationID::kCount: {
+        mergeStateSum<std::int64_t>(
+            dst_bucket_indices, source.state_vecs_[i], state_vecs_[i]);
+        break;
+      }
+      case AggregationID::kSum: {
+        const Type *argument_type = handle->getArgumentTypes().front();
+        switch (argument_type->getTypeID()) {
+          case kInt:  // Fall through
+          case kLong: {
+            mergeStateSum<std::int64_t>(
+                dst_bucket_indices, source.state_vecs_[i], state_vecs_[i]);
+            break;
+          }
+          case kFloat:  // Fall through
+          case kDouble: {
+            mergeStateSum<double>(
+                dst_bucket_indices, source.state_vecs_[i], state_vecs_[i]);
+            break;
+          }
+          default:
+            LOG(FATAL) << "Unexpected argument type";
+        }
+        break;
+      }
+      default:
+        LOG(FATAL) << "Unexpected AggregationID";
+    }
+  }
+}
+
+void ThreadPrivateCompactKeyHashTable::finalize(
+    ColumnVectorsValueAccessor *output) const {
+  // First finalize keys.
+  std::size_t key_offset = 0;
+  for (std::size_t i = 0; i < key_types_.size(); ++i) {
+    const Type &key_type = *key_types_[i];
+    std::unique_ptr<NativeColumnVector> native_cv(
+        std::make_unique<NativeColumnVector>(key_type, buckets_allocated_));
+
+    InvokeOnKeySize(
+        key_sizes_[i],
+        [&](auto key_size) -> void {  // NOLINT(build/c++11)
+      this->finalizeKey<decltype(key_size)::value>(key_offset, native_cv.get());
+    });
+    output->addColumn(native_cv.release());
+    key_offset += key_sizes_[i];
+  }
+
+  // Then finalize states.
+  for (std::size_t i = 0; i < handles_.size(); ++i) {
+    const AggregationHandle *handle = handles_[i];
+    const Type &result_type = *handle->getResultType();
+    std::unique_ptr<NativeColumnVector> native_cv(
+        std::make_unique<NativeColumnVector>(result_type, buckets_allocated_));
+
+    switch (handle->getAggregationID()) {
+      case AggregationID::kCount: {
+        finalizeStateSum<std::int64_t, std::int64_t>(
+            state_vecs_[i], native_cv.get());
+        break;
+      }
+      case AggregationID::kSum: {
+        const Type *argument_type = handle->getArgumentTypes().front();
+        switch (argument_type->getTypeID()) {
+          case kInt:  // Fall through
+          case kLong: {
+            finalizeStateSum<std::int64_t, std::int64_t>(
+                state_vecs_[i], native_cv.get());
+            break;
+          }
+          case kFloat:  // Fall through
+          case kDouble: {
+            finalizeStateSum<double, double>(
+                state_vecs_[i], native_cv.get());
+            break;
+          }
+          default:
+            LOG(FATAL) << "Unexpected argument type";
+        }
+        break;
+      }
+      default:
+        LOG(FATAL) << "Unexpected AggregationID";
+    }
+    output->addColumn(native_cv.release());
+  }
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d6a01e7c/storage/ThreadPrivateCompactKeyHashTable.hpp
----------------------------------------------------------------------
diff --git a/storage/ThreadPrivateCompactKeyHashTable.hpp b/storage/ThreadPrivateCompactKeyHashTable.hpp
new file mode 100644
index 0000000..277e2e5
--- /dev/null
+++ b/storage/ThreadPrivateCompactKeyHashTable.hpp
@@ -0,0 +1,230 @@
+/**
+ * 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_THREAD_PRIVATE_COMPACT_KEY_HASH_TABLE_HPP_
+#define QUICKSTEP_STORAGE_THREAD_PRIVATE_COMPACT_KEY_HASH_TABLE_HPP_
+
+#include <algorithm>
+#include <cstddef>
+#include <cstdint>
+#include <unordered_map>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/HashTableBase.hpp"
+#include "storage/StorageBlob.hpp"
+#include "storage/StorageConstants.hpp"
+#include "storage/ValueAccessorMultiplexer.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class AggregationHandle;
+class StorageManager;
+class Type;
+
+/**
+ * @brief Specialized aggregation hash table that is preferable for two-phase
+ *        aggregation with small-cardinality group-by keys. To use this hash
+ *        table, it also requires that the group-by keys have fixed-length types
+ *        with total byte size no greater than 8 (so that the keys can be packed
+ *        into a 64-bit QWORD).
+ */
+class ThreadPrivateCompactKeyHashTable : public AggregationStateHashTableBase {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param key_types A vector of one or more types (>1 indicates a composite
+   *        key).
+   * @param num_entries The estimated number of entries this hash table will
+   *        hold.
+   * @param handles The aggregation handles.
+   * @param storage_manager The StorageManager to use (a StorageBlob will be
+   *        allocated to hold this hash table's contents).
+   **/
+  ThreadPrivateCompactKeyHashTable(
+      const std::vector<const Type*> &key_types,
+      const std::size_t num_entries,
+      const std::vector<AggregationHandle*> &handles,
+      StorageManager *storage_manager);
+
+  ~ThreadPrivateCompactKeyHashTable() override;
+
+  HashTableImplType getImplType() const override {
+    return HashTableImplType::kThreadPrivateCompactKey;
+  }
+
+  void destroyPayload() override {}
+
+  std::size_t getMemoryConsumptionBytes() const override {
+    return blob_->size();
+  }
+
+  /**
+   * @return The number of entries in this HashTable.
+   **/
+  inline std::size_t numEntries() const {
+    return buckets_allocated_;
+  }
+
+  bool upsertValueAccessorCompositeKey(
+      const std::vector<std::vector<MultiSourceAttributeId>> &argument_ids,
+      const std::vector<MultiSourceAttributeId> &key_attr_ids,
+      const ValueAccessorMultiplexer &accessor_mux) override;
+
+  /**
+   * @brief Merge the states of \p source into this hash table.
+   *
+   * @param source The source hash table from which the states are to be merged
+   *        into this hash table.
+   */
+  void mergeFrom(const ThreadPrivateCompactKeyHashTable &source);
+
+  /**
+   * @brief Finalize all the aggregation state vectors and add the result column
+   *        vectors into the output ColumnVectorsValueAccessor.
+   *
+   * @param output The ColumnVectorsValueAccessor to add all the result column
+   *        vectors into.
+   */
+  void finalize(ColumnVectorsValueAccessor *output) const;
+
+ private:
+  // Compact key as a 64-bit QWORD.
+  using KeyCode = std::uint64_t;
+  static constexpr std::size_t kKeyCodeSize = sizeof(KeyCode);
+
+  using BucketIndex = std::uint32_t;
+
+  inline static std::size_t CacheLineAlignedBytes(const std::size_t actual_bytes) {
+    return (actual_bytes + kCacheLineBytes - 1) / kCacheLineBytes * kCacheLineBytes;
+  }
+
+  // Grow the size of this hash table by a factor of 2.
+  void resize();
+
+  template <std::size_t key_size>
+  inline static void ConstructKeyCode(const std::size_t offset,
+                                      const attribute_id attr_id,
+                                      ValueAccessor *accessor,
+                                      void *key_code_start) {
+    InvokeOnAnyValueAccessor(
+        accessor,
+        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+      char *key_code_ptr = static_cast<char*>(key_code_start) + offset;
+      accessor->beginIteration();
+      while (accessor->next()) {
+        std::memcpy(key_code_ptr,
+                    accessor->template getUntypedValue<false>(attr_id),
+                    key_size);
+        key_code_ptr += kKeyCodeSize;
+      }
+    });
+  }
+
+  inline void upsertValueAccessorCount(const std::vector<BucketIndex> &bucket_indices,
+                                       void *state_vec) {
+    std::int64_t *states = static_cast<std::int64_t*>(state_vec);
+    for (const BucketIndex idx : bucket_indices) {
+      states[idx] += 1;
+    }
+  }
+
+  template <typename ArgumentT, typename StateT>
+  inline void upsertValueAccessorSum(const std::vector<BucketIndex> &bucket_indices,
+                                     const attribute_id attr_id,
+                                     ValueAccessor *accessor,
+                                     void *state_vec) {
+    InvokeOnAnyValueAccessor(
+        accessor,
+        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
+      accessor->beginIteration();
+
+      StateT *states = static_cast<StateT*>(state_vec);
+      for (const BucketIndex idx : bucket_indices) {
+        accessor->next();
+        states[idx] += *static_cast<const ArgumentT*>(
+            accessor->template getUntypedValue<false>(attr_id));
+      }
+    });
+  }
+
+  template <typename StateT>
+  inline void mergeStateSum(const std::vector<BucketIndex> &dst_bucket_indices,
+                            const void *src_state_vec,
+                            void *dst_state_vec) {
+    StateT *dst_states = static_cast<StateT*>(dst_state_vec);
+    const StateT* src_states = static_cast<const StateT*>(src_state_vec);
+    for (std::size_t i = 0; i < dst_bucket_indices.size(); ++i) {
+      dst_states[dst_bucket_indices[i]] += src_states[i];
+    }
+  }
+
+  template <std::size_t key_size>
+  inline void finalizeKey(const std::size_t offset,
+                          NativeColumnVector *output_cv) const {
+    const char *key_ptr = reinterpret_cast<const char*>(keys_) + offset;
+    for (std::size_t i = 0; i < buckets_allocated_; ++i) {
+      std::memcpy(output_cv->getPtrForDirectWrite(),
+                  key_ptr,
+                  key_size);
+      key_ptr += kKeyCodeSize;
+    }
+  }
+
+  template <typename StateT, typename ResultT>
+  inline void finalizeStateSum(const void *state_vec,
+                               NativeColumnVector *output_cv) const {
+    const StateT *states = static_cast<const StateT*>(state_vec);
+    for (std::size_t i = 0; i < buckets_allocated_; ++i) {
+      *static_cast<ResultT*>(output_cv->getPtrForDirectWrite()) = states[i];
+    }
+  }
+
+  const std::vector<const Type*> key_types_;
+  const std::vector<AggregationHandle *> handles_;
+
+  std::vector<std::size_t> key_sizes_;
+  std::vector<std::size_t> state_sizes_;
+  std::size_t total_state_size_;
+
+  std::size_t num_buckets_;
+  std::size_t buckets_allocated_;
+
+  // Maps a compact-key to its bucket location.
+  std::unordered_map<KeyCode, BucketIndex> index_;
+
+  // Compact-key array where keys_[i] holds the compact-key for bucket i.
+  KeyCode *keys_;
+
+  // Use a column-wise layout for aggregation states.
+  std::vector<void*> state_vecs_;
+
+  StorageManager *storage_manager_;
+  MutableBlobReference blob_;
+
+  DISALLOW_COPY_AND_ASSIGN(ThreadPrivateCompactKeyHashTable);
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_STORAGE_THREAD_PRIVATE_COMPACT_KEY_HASH_TABLE_HPP_


[31/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index 5db2171..d51ae48 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -102,15 +102,9 @@ typedef struct YYLTYPE {
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
-#include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
-#include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationFactory.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
 #include "utility/PtrList.hpp"
 #include "utility/PtrVector.hpp"
 
@@ -190,8 +184,8 @@ typedef void* yyscan_t;
   quickstep::ParseStatementQuit *quit_statement_;
 
   const quickstep::Comparison *comparison_;
-  const quickstep::UnaryOperation *unary_operation_;
-  const quickstep::BinaryOperation *binary_operation_;
+  quickstep::ParseString *unary_operation_;
+  quickstep::ParseString *binary_operation_;
 
   quickstep::ParseFunctionCall *function_call_;
   quickstep::PtrList<quickstep::ParseExpression> *expression_list_;
@@ -262,6 +256,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_CSB_TREE;
 %token TOKEN_BY;
 %token TOKEN_CASE;
+%token TOKEN_CAST;
 %token TOKEN_CHARACTER;
 %token TOKEN_CHECK;
 %token TOKEN_COLUMN;
@@ -393,6 +388,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
   add_expression
   case_expression
   opt_else_clause
+  cast_function
   extract_function
   substr_function
 
@@ -623,8 +619,6 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 
 %destructor { } <boolean_value_>
 %destructor { } <comparison_>
-%destructor { } <unary_operation_>
-%destructor { } <binary_operation_>
 %destructor { } <join_type_>
 
 %destructor {
@@ -1655,7 +1649,10 @@ predicate_expression_base:
 /* Scalars */
 add_expression:
   add_expression add_operation multiply_expression {
-    $$ = new quickstep::ParseBinaryExpression(@2.first_line, @2.first_column, *$2, $1, $3);
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($1);
+    arguments->push_back($3);
+    $$ = new quickstep::ParseFunctionCall(@1.first_line, @1.first_column, false, $2, arguments);
   }
   | multiply_expression {
     $$ = $1;
@@ -1663,7 +1660,10 @@ add_expression:
 
 multiply_expression:
   multiply_expression multiply_operation unary_expression {
-    $$ = new quickstep::ParseBinaryExpression(@2.first_line, @2.first_column, *$2, $1, $3);
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($1);
+    arguments->push_back($3);
+    $$ = new quickstep::ParseFunctionCall(@1.first_line, @1.first_column, false, $2, arguments);
   }
   | unary_expression {
     $$ = $1;
@@ -1671,7 +1671,9 @@ multiply_expression:
 
 unary_expression:
   unary_operation expression_base {
-    $$ = new quickstep::ParseUnaryExpression(@1.first_line, @1.first_column, *$1, $2);
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($2);
+    $$ = new quickstep::ParseFunctionCall(@1.first_line, @1.first_column, false, $1, arguments);
   }
   | expression_base {
     $$ = $1;
@@ -1695,6 +1697,9 @@ expression_base:
     $1->setWindow($4);
     $$ = $1;
   }
+  | cast_function {
+    $$ = $1;
+  }
   | extract_function {
     $$ = $1;
   }
@@ -1727,19 +1732,59 @@ function_call:
     $$ = new quickstep::ParseFunctionCall(@1.first_line, @1.first_column, true, $1, $4);
   };
 
+cast_function:
+  TOKEN_CAST '(' add_expression TOKEN_AS data_type ')' {
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($3);
+    arguments->push_back(new quickstep::ParseScalarLiteral(
+        new quickstep::StringParseLiteralValue(
+            new quickstep::ParseString(@5.first_line,
+                                       @5.first_column,
+                                       $5->getType().getName()),
+            nullptr)));
+    delete $5;
+    auto *name = new quickstep::ParseString(@1.first_line, @1.first_column, "cast");
+    $$ = new quickstep::ParseFunctionCall(
+        @1.first_line, @1.first_column, false, name, arguments);
+  }
+  | TOKEN_CAST '(' add_expression TOKEN_AS any_name ')' {
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($3);
+    arguments->push_back(new quickstep::ParseScalarLiteral(
+        new quickstep::StringParseLiteralValue($5, nullptr)));
+    auto *name = new quickstep::ParseString(@1.first_line, @1.first_column, "cast");
+    $$ = new quickstep::ParseFunctionCall(
+        @1.first_line, @1.first_column, false, name, arguments);
+  };
+
 extract_function:
   TOKEN_EXTRACT '(' datetime_unit TOKEN_FROM add_expression ')' {
-    $$ = new quickstep::ParseExtractFunction(@1.first_line, @1.first_column, $3, $5);
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($5);
+    arguments->push_back(new quickstep::ParseScalarLiteral(
+        new quickstep::StringParseLiteralValue($3, nullptr)));
+    auto *name = new quickstep::ParseString(@1.first_line, @1.first_column, "extract");
+    $$ = new quickstep::ParseFunctionCall(
+        @1.first_line, @1.first_column, false, name, arguments);
   };
 
 substr_function:
   TOKEN_SUBSTRING '(' add_expression TOKEN_FROM TOKEN_UNSIGNED_NUMVAL ')' {
-    $$ = new quickstep::ParseSubstringFunction(
-        @1.first_line, @1.first_column, $3, $5->long_value());
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($3);
+    arguments->push_back(new quickstep::ParseScalarLiteral($5));
+    auto *name = new quickstep::ParseString(@1.first_line, @1.first_column, "substring");
+    $$ = new quickstep::ParseFunctionCall(
+        @1.first_line, @1.first_column, false, name, arguments);
   }
   | TOKEN_SUBSTRING '(' add_expression TOKEN_FROM TOKEN_UNSIGNED_NUMVAL TOKEN_FOR TOKEN_UNSIGNED_NUMVAL ')' {
-    $$ = new quickstep::ParseSubstringFunction(
-        @1.first_line, @1.first_column, $3, $5->long_value(), $7->long_value());
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($3);
+    arguments->push_back(new quickstep::ParseScalarLiteral($5));
+    arguments->push_back(new quickstep::ParseScalarLiteral($7));
+    auto *name = new quickstep::ParseString(@1.first_line, @1.first_column, "substring");
+    $$ = new quickstep::ParseFunctionCall(
+        @1.first_line, @1.first_column, false, name, arguments);
   };
 
 case_expression:
@@ -1959,26 +2004,26 @@ unary_operation:
      * to shift rather than reduce, the case in 'literal_value' has precedence
      * over this one.
      **/
-    $$ = &quickstep::UnaryOperationFactory::GetUnaryOperation(quickstep::UnaryOperationID::kNegate);
+    $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("-"));
   };
 
 add_operation:
   '+' {
-    $$ = &quickstep::BinaryOperationFactory::GetBinaryOperation(quickstep::BinaryOperationID::kAdd);
+    $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("+"));
   }
   | '-' {
-    $$ = &quickstep::BinaryOperationFactory::GetBinaryOperation(quickstep::BinaryOperationID::kSubtract);
+    $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("-"));
   };
 
 multiply_operation:
   '%' {
-    $$ = &quickstep::BinaryOperationFactory::GetBinaryOperation(quickstep::BinaryOperationID::kModulo);
+    $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("%"));
   }
   | '*' {
-    $$ = &quickstep::BinaryOperationFactory::GetBinaryOperation(quickstep::BinaryOperationID::kMultiply);
+    $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("*"));
   }
   | '/' {
-    $$ = &quickstep::BinaryOperationFactory::GetBinaryOperation(quickstep::BinaryOperationID::kDivide);
+    $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("/"));
   };
 
 /* General Utility Stuff */


[04/32] incubator-quickstep git commit: WorkOrder proto clean-up.

Posted by ji...@apache.org.
WorkOrder proto clean-up.


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

Branch: refs/heads/new-op
Commit: b19bb94fd5b8f2aac3bb2508d750091df36d324a
Parents: 758f07a
Author: Zuyu Zhang <zu...@apache.org>
Authored: Thu Apr 20 15:21:22 2017 -0700
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Thu Apr 20 15:21:22 2017 -0700

----------------------------------------------------------------------
 relational_operators/WorkOrder.proto | 69 ++++++++++++++++---------------
 1 file changed, 35 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/b19bb94f/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index 12a65ca..7231c84 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -21,13 +21,16 @@ package quickstep.serialization;
 
 import "relational_operators/SortMergeRunOperator.proto";
 
+// Next tag: 25.
 enum WorkOrderType {
   AGGREGATION = 1;
+  BUILD_AGGREGATION_EXISTENCE_MAP = 23;
   BUILD_HASH = 2;
   BUILD_LIP_FILTER = 3;
   CREATE_INDEX = 4;  // Placeholder.
   CREATE_TABLE = 5;  // Placeholder.
   DELETE = 6;
+  DESTROY_AGGREGATION_STATE = 22;
   DESTROY_HASH = 7;
   DROP_TABLE = 8;
   FINALIZE_AGGREGATION = 9;
@@ -41,11 +44,9 @@ enum WorkOrderType {
   SORT_RUN_GENERATION = 17;
   TABLE_GENERATOR = 18;
   TEXT_SCAN = 19;
+  UNION_ALL = 24;
   UPDATE = 20;
   WINDOW_AGGREGATION = 21;
-  DESTROY_AGGREGATION_STATE = 22;
-  BUILD_AGGREGATION_EXISTENCE_MAP = 23;
-  UNION_ALL = 24;
 }
 
 message WorkOrder {
@@ -53,7 +54,7 @@ message WorkOrder {
   required uint64 query_id = 2;
 
   // The convention for extension numbering is that extensions for a particular
-  // WorkOrderID should begin from (operator_type + 1) * 16.
+  // WorkOrderID should begin from (operator_type * 16).
   extensions 16 to max;
 }
 
@@ -66,6 +67,15 @@ message AggregationWorkOrder {
   }
 }
 
+message BuildAggregationExistenceMapWorkOrder {
+  extend WorkOrder {
+    optional int32 relation_id = 368;
+    optional fixed64 build_block_id = 369;
+    optional int32 build_attribute = 370;
+    optional uint32 aggr_state_index = 371;
+  }
+}
+
 // Next tag: 39.
 message BuildHashWorkOrder {
   extend WorkOrder {
@@ -100,6 +110,12 @@ message DeleteWorkOrder {
   }
 }
 
+message DestroyAggregationStateWorkOrder {
+  extend WorkOrder {
+    optional uint32 aggr_state_index = 352;
+  }
+}
+
 message DestroyHashWorkOrder {
   extend WorkOrder {
     // All required.
@@ -248,12 +264,21 @@ message TableGeneratorWorkOrder {
 message TextScanWorkOrder {
   extend WorkOrder {
     // All required.
-    optional string filename = 301;
-    optional uint64 text_offset = 302;
-    optional uint64 text_segment_size = 303;
-    optional uint32 field_terminator = 304;  // For one-byte char.
-    optional bool process_escape_sequences = 305;
-    optional int32 insert_destination_index = 306;
+    optional string filename = 304;
+    optional uint64 text_offset = 305;
+    optional uint64 text_segment_size = 306;
+    optional uint32 field_terminator = 307;  // For one-byte char.
+    optional bool process_escape_sequences = 308;
+    optional int32 insert_destination_index = 309;
+  }
+}
+
+message UnionAllWorkOrder {
+  extend WorkOrder {
+    optional int32 relation_id = 384;
+    optional int32 insert_destination_index = 385;
+    optional fixed64 block_id = 386;
+    repeated int32 select_attribute_id = 387;
   }
 }
 
@@ -277,27 +302,3 @@ message WindowAggregationWorkOrder {
     optional int32 insert_destination_index = 338;
   }
 }
-
-message DestroyAggregationStateWorkOrder {
-  extend WorkOrder {
-    optional uint32 aggr_state_index = 352;
-  }
-}
-
-message BuildAggregationExistenceMapWorkOrder {
-  extend WorkOrder {
-    optional int32 relation_id = 368;
-    optional fixed64 build_block_id = 369;
-    optional int32 build_attribute = 370;
-    optional uint32 aggr_state_index = 371;
-  }
-}
-
-message UnionAllWorkOrder {
-  extend WorkOrder {
-    optional int32 relation_id = 384;
-    optional int32 insert_destination_index = 385;
-    optional fixed64 block_id = 386;
-    repeated int32 select_attribute_id = 387;
-  }
-}


[07/32] incubator-quickstep git commit: Add common-subexpression support.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/ScalarLiteral.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarLiteral.cpp b/expressions/scalar/ScalarLiteral.cpp
index 48b5574..808953d 100644
--- a/expressions/scalar/ScalarLiteral.cpp
+++ b/expressions/scalar/ScalarLiteral.cpp
@@ -19,6 +19,7 @@
 
 #include "expressions/scalar/ScalarLiteral.hpp"
 
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -47,24 +48,49 @@ Scalar* ScalarLiteral::clone() const {
   return new ScalarLiteral(internal_literal_, type_);
 }
 
-ColumnVector* ScalarLiteral::getAllValues(
+ColumnVectorPtr ScalarLiteral::getAllValues(
     ValueAccessor *accessor,
-    const SubBlocksReference *sub_blocks_ref) const {
-  return ColumnVector::MakeVectorOfValue(
-      type_,
-      internal_literal_,
-      accessor->getNumTuplesVirtual());
+    const SubBlocksReference *sub_blocks_ref,
+    ColumnVectorCache *cv_cache) const {
+  return ColumnVectorPtr(
+      ColumnVector::MakeVectorOfValue(type_,
+                                      internal_literal_,
+                                      accessor->getNumTuplesVirtual()));
 }
 
-ColumnVector* ScalarLiteral::getAllValuesForJoin(
+ColumnVectorPtr ScalarLiteral::getAllValuesForJoin(
     const relation_id left_relation_id,
     ValueAccessor *left_accessor,
     const relation_id right_relation_id,
     ValueAccessor *right_accessor,
-    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
-  return ColumnVector::MakeVectorOfValue(type_,
-                                         internal_literal_,
-                                         joined_tuple_ids.size());
+    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+    ColumnVectorCache *cv_cache) const {
+  return ColumnVectorPtr(
+      ColumnVector::MakeVectorOfValue(type_,
+                                      internal_literal_,
+                                      joined_tuple_ids.size()));
+}
+
+void ScalarLiteral::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<const Expression*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const Expression*>> *container_child_fields) const {
+  Scalar::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->emplace_back("internal_literal");
+  if (internal_literal_.isNull()) {
+    inline_field_values->emplace_back("NULL");
+  } else {
+    inline_field_values->emplace_back(type_.printValueToString(internal_literal_));
+  }
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/ScalarLiteral.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarLiteral.hpp b/expressions/scalar/ScalarLiteral.hpp
index c7f5ceb..2a4c396 100644
--- a/expressions/scalar/ScalarLiteral.hpp
+++ b/expressions/scalar/ScalarLiteral.hpp
@@ -20,6 +20,7 @@
 #ifndef QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_LITERAL_HPP_
 #define QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_LITERAL_HPP_
 
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -28,11 +29,12 @@
 #include "expressions/scalar/Scalar.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
-class ColumnVector;
+class ColumnVectorCache;
 class Type;
 class ValueAccessor;
 
@@ -101,15 +103,26 @@ class ScalarLiteral : public Scalar {
     return internal_literal_;
   }
 
-  ColumnVector* getAllValues(ValueAccessor *accessor,
-                             const SubBlocksReference *sub_blocks_ref) const override;
+  ColumnVectorPtr getAllValues(ValueAccessor *accessor,
+                               const SubBlocksReference *sub_blocks_ref,
+                               ColumnVectorCache *cv_cache) const override;
 
-  ColumnVector* getAllValuesForJoin(
+  ColumnVectorPtr getAllValuesForJoin(
       const relation_id left_relation_id,
       ValueAccessor *left_accessor,
       const relation_id right_relation_id,
       ValueAccessor *right_accessor,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override;
+      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+      ColumnVectorCache *cv_cache) const override;
+
+ 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<const Expression*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const Expression*>> *container_child_fields) const override;
 
  private:
   TypedValue internal_literal_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/ScalarSharedExpression.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarSharedExpression.cpp b/expressions/scalar/ScalarSharedExpression.cpp
new file mode 100644
index 0000000..f97c60b
--- /dev/null
+++ b/expressions/scalar/ScalarSharedExpression.cpp
@@ -0,0 +1,141 @@
+/**
+ * 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 "expressions/scalar/ScalarSharedExpression.hpp"
+
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/Expressions.pb.h"
+#include "storage/ValueAccessor.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "utility/ColumnVectorCache.hpp"
+
+namespace quickstep {
+
+struct SubBlocksReference;
+
+ScalarSharedExpression::ScalarSharedExpression(const int share_id,
+                                               Scalar *operand)
+    : Scalar(operand->getType()),
+      share_id_(share_id),
+      operand_(operand) {
+}
+
+serialization::Scalar ScalarSharedExpression::getProto() const {
+  serialization::Scalar proto;
+  proto.set_data_source(serialization::Scalar::SHARED_EXPRESSION);
+  proto.SetExtension(serialization::ScalarSharedExpression::share_id, share_id_);
+  proto.MutableExtension(serialization::ScalarSharedExpression::operand)
+      ->CopyFrom(operand_->getProto());
+
+  return proto;
+}
+
+Scalar* ScalarSharedExpression::clone() const {
+  return new ScalarSharedExpression(share_id_, operand_->clone());
+}
+
+TypedValue ScalarSharedExpression::getValueForSingleTuple(const ValueAccessor &accessor,
+                                                          const tuple_id tuple) const {
+  return operand_->getValueForSingleTuple(accessor, tuple);
+}
+
+TypedValue ScalarSharedExpression::getValueForJoinedTuples(
+    const ValueAccessor &left_accessor,
+    const relation_id left_relation_id,
+    const tuple_id left_tuple_id,
+    const ValueAccessor &right_accessor,
+    const relation_id right_relation_id,
+    const tuple_id right_tuple_id) const {
+  return operand_->getValueForJoinedTuples(left_accessor,
+                                           left_relation_id,
+                                           left_tuple_id,
+                                           right_accessor,
+                                           right_relation_id,
+                                           right_tuple_id);
+}
+
+ColumnVectorPtr ScalarSharedExpression::getAllValues(
+    ValueAccessor *accessor,
+    const SubBlocksReference *sub_blocks_ref,
+    ColumnVectorCache *cv_cache) const {
+  if (cv_cache == nullptr) {
+    return operand_->getAllValues(accessor, sub_blocks_ref, cv_cache);
+  } else {
+    ColumnVectorPtr result;
+    if (cv_cache->contains(share_id_)) {
+      result = cv_cache->get(share_id_);
+    } else {
+      result = operand_->getAllValues(accessor, sub_blocks_ref, cv_cache);
+      cv_cache->set(share_id_, result);
+    }
+    return result;
+  }
+}
+
+ColumnVectorPtr ScalarSharedExpression::getAllValuesForJoin(
+    const relation_id left_relation_id,
+    ValueAccessor *left_accessor,
+    const relation_id right_relation_id,
+    ValueAccessor *right_accessor,
+    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+    ColumnVectorCache *cv_cache) const {
+  if (cv_cache == nullptr) {
+    return operand_->getAllValuesForJoin(left_relation_id,
+                                         left_accessor,
+                                         right_relation_id,
+                                         right_accessor,
+                                         joined_tuple_ids,
+                                         cv_cache);
+  } else {
+    ColumnVectorPtr result;
+    if (cv_cache->contains(share_id_)) {
+      result = cv_cache->get(share_id_);
+    } else {
+      result = operand_->getAllValuesForJoin(left_relation_id,
+                                             left_accessor,
+                                             right_relation_id,
+                                             right_accessor,
+                                             joined_tuple_ids,
+                                             cv_cache);
+      cv_cache->set(share_id_, result);
+    }
+    return result;
+  }
+}
+
+void ScalarSharedExpression::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<const Expression*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const Expression*>> *container_child_fields) const {
+  inline_field_names->emplace_back("share_id");
+  inline_field_values->emplace_back(std::to_string(share_id_));
+
+  non_container_child_field_names->emplace_back("operand");
+  non_container_child_fields->emplace_back(operand_.get());
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/ScalarSharedExpression.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarSharedExpression.hpp b/expressions/scalar/ScalarSharedExpression.hpp
new file mode 100644
index 0000000..d5dddbc
--- /dev/null
+++ b/expressions/scalar/ScalarSharedExpression.hpp
@@ -0,0 +1,127 @@
+/**
+ * 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_EXPRESSIONS_SCALAR_SCALAR_SHARED_EXPRESSION_HPP_
+#define QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_SHARED_EXPRESSION_HPP_
+
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "expressions/Expressions.pb.h"
+#include "expressions/scalar/Scalar.hpp"
+#include "storage/StorageBlockInfo.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class ColumnVectorCache;
+class ValueAccessor;
+
+struct SubBlocksReference;
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief Scalars that represent common subexpressions whose results are cached
+ *        and shared.
+ **/
+class ScalarSharedExpression : public Scalar {
+ public:
+  /**
+   * @brief Constructor.
+   *
+   * @param share_id The unique integer identifier for each equivalence class of
+   *        common subexpressions.
+   * @param operand The underlying scalar subexpression.
+   **/
+  ScalarSharedExpression(const int share_id, Scalar *operand);
+
+  /**
+   * @brief Destructor.
+   **/
+  ~ScalarSharedExpression() override {
+  }
+
+  serialization::Scalar getProto() const override;
+
+  Scalar* clone() const override;
+
+  ScalarDataSource getDataSource() const override {
+    return kSharedExpression;
+  }
+
+  TypedValue getValueForSingleTuple(const ValueAccessor &accessor,
+                                    const tuple_id tuple) const override;
+
+  TypedValue getValueForJoinedTuples(
+      const ValueAccessor &left_accessor,
+      const relation_id left_relation_id,
+      const tuple_id left_tuple_id,
+      const ValueAccessor &right_accessor,
+      const relation_id right_relation_id,
+      const tuple_id right_tuple_id) const override;
+
+  bool hasStaticValue() const override {
+    return operand_->hasStaticValue();
+  }
+
+  const TypedValue& getStaticValue() const override {
+    return operand_->getStaticValue();
+  }
+
+  ColumnVectorPtr getAllValues(ValueAccessor *accessor,
+                               const SubBlocksReference *sub_blocks_ref,
+                               ColumnVectorCache *cv_cache) const override;
+
+  ColumnVectorPtr getAllValuesForJoin(
+      const relation_id left_relation_id,
+      ValueAccessor *left_accessor,
+      const relation_id right_relation_id,
+      ValueAccessor *right_accessor,
+      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+      ColumnVectorCache *cv_cache) const override;
+
+ 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<const Expression*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const Expression*>> *container_child_fields) const override;
+
+ private:
+  const int share_id_;
+  std::unique_ptr<Scalar> operand_;
+
+  DISALLOW_COPY_AND_ASSIGN(ScalarSharedExpression);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_SHARED_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/ScalarUnaryExpression.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarUnaryExpression.cpp b/expressions/scalar/ScalarUnaryExpression.cpp
index 72fdbe1..c51e38f 100644
--- a/expressions/scalar/ScalarUnaryExpression.cpp
+++ b/expressions/scalar/ScalarUnaryExpression.cpp
@@ -21,6 +21,7 @@
 
 #include <memory>
 #include <string>
+#include <type_traits>
 #include <utility>
 #include <vector>
 
@@ -33,6 +34,7 @@
 #include "types/containers/ColumnVector.hpp"
 #include "types/operations/Operation.pb.h"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
+#include "types/operations/unary_operations/UnaryOperationID.hpp"
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
 #include "glog/logging.h"
@@ -91,36 +93,43 @@ TypedValue ScalarUnaryExpression::getValueForJoinedTuples(
   }
 }
 
-ColumnVector* ScalarUnaryExpression::getAllValues(
+ColumnVectorPtr ScalarUnaryExpression::getAllValues(
     ValueAccessor *accessor,
-    const SubBlocksReference *sub_blocks_ref) const {
+    const SubBlocksReference *sub_blocks_ref,
+    ColumnVectorCache *cv_cache) const {
   if (fast_operator_.get() == nullptr) {
-    return ColumnVector::MakeVectorOfValue(getType(),
-                                           static_value_,
-                                           accessor->getNumTuplesVirtual());
+    return ColumnVectorPtr(
+        ColumnVector::MakeVectorOfValue(getType(),
+                                        static_value_,
+                                        accessor->getNumTuplesVirtual()));
   } else {
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
     const attribute_id operand_attr_id = operand_->getAttributeIdForValueAccessor();
     if (operand_attr_id != -1) {
-      return fast_operator_->applyToValueAccessor(accessor, operand_attr_id);
+      return ColumnVectorPtr(
+          fast_operator_->applyToValueAccessor(accessor, operand_attr_id));
     }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
-    std::unique_ptr<ColumnVector> operand_result(operand_->getAllValues(accessor, sub_blocks_ref));
-    return fast_operator_->applyToColumnVector(*operand_result);
+    ColumnVectorPtr operand_result(
+        operand_->getAllValues(accessor, sub_blocks_ref, cv_cache));
+    return ColumnVectorPtr(
+        fast_operator_->applyToColumnVector(*operand_result));
   }
 }
 
-ColumnVector* ScalarUnaryExpression::getAllValuesForJoin(
+ColumnVectorPtr ScalarUnaryExpression::getAllValuesForJoin(
     const relation_id left_relation_id,
     ValueAccessor *left_accessor,
     const relation_id right_relation_id,
     ValueAccessor *right_accessor,
-    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
+    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+    ColumnVectorCache *cv_cache) const {
   if (fast_operator_.get() == nullptr) {
-    return ColumnVector::MakeVectorOfValue(getType(),
-                                           static_value_,
-                                           joined_tuple_ids.size());
+    return ColumnVectorPtr(
+        ColumnVector::MakeVectorOfValue(getType(),
+                                        static_value_,
+                                        joined_tuple_ids.size()));
   } else {
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
     const attribute_id operand_attr_id = operand_->getAttributeIdForValueAccessor();
@@ -132,20 +141,23 @@ ColumnVector* ScalarUnaryExpression::getAllValuesForJoin(
       const bool using_left_relation = (operand_relation_id == left_relation_id);
       ValueAccessor *operand_accessor = using_left_relation ? left_accessor
                                                             : right_accessor;
-      return fast_operator_->applyToValueAccessorForJoin(operand_accessor,
-                                                         using_left_relation,
-                                                         operand_attr_id,
-                                                         joined_tuple_ids);
+      return ColumnVectorPtr(
+          fast_operator_->applyToValueAccessorForJoin(operand_accessor,
+                                                      using_left_relation,
+                                                      operand_attr_id,
+                                                      joined_tuple_ids));
     }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
 
-    std::unique_ptr<ColumnVector> operand_result(
+    ColumnVectorPtr operand_result(
         operand_->getAllValuesForJoin(left_relation_id,
                                       left_accessor,
                                       right_relation_id,
                                       right_accessor,
-                                      joined_tuple_ids));
-    return fast_operator_->applyToColumnVector(*operand_result);
+                                      joined_tuple_ids,
+                                      cv_cache));
+    return ColumnVectorPtr(
+        fast_operator_->applyToColumnVector(*operand_result));
   }
 }
 
@@ -166,4 +178,36 @@ void ScalarUnaryExpression::initHelper(bool own_children) {
   }
 }
 
+void ScalarUnaryExpression::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<const Expression*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const Expression*>> *container_child_fields) const {
+  Scalar::getFieldStringItems(inline_field_names,
+                              inline_field_values,
+                              non_container_child_field_names,
+                              non_container_child_fields,
+                              container_child_field_names,
+                              container_child_fields);
+
+  if (fast_operator_ == nullptr) {
+    inline_field_names->emplace_back("static_value");
+    if (static_value_.isNull()) {
+      inline_field_values->emplace_back("NULL");
+    } else {
+      inline_field_values->emplace_back(type_.printValueToString(static_value_));
+    }
+  }
+
+  inline_field_names->emplace_back("operation");
+  inline_field_values->emplace_back(
+      kUnaryOperationNames[static_cast<std::underlying_type<UnaryOperationID>::type>(
+          operation_.getUnaryOperationID())]);
+
+  non_container_child_field_names->emplace_back("operand");
+  non_container_child_fields->emplace_back(operand_.get());
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/ScalarUnaryExpression.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarUnaryExpression.hpp b/expressions/scalar/ScalarUnaryExpression.hpp
index 608a842..52edea7 100644
--- a/expressions/scalar/ScalarUnaryExpression.hpp
+++ b/expressions/scalar/ScalarUnaryExpression.hpp
@@ -21,6 +21,7 @@
 #define QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_UNARY_EXPRESSION_HPP_
 
 #include <memory>
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -36,7 +37,7 @@
 
 namespace quickstep {
 
-class ColumnVector;
+class ColumnVectorCache;
 class ValueAccessor;
 
 struct SubBlocksReference;
@@ -93,15 +94,26 @@ class ScalarUnaryExpression : public Scalar {
     return static_value_;
   }
 
-  ColumnVector* getAllValues(ValueAccessor *accessor,
-                             const SubBlocksReference *sub_blocks_ref) const override;
+  ColumnVectorPtr getAllValues(ValueAccessor *accessor,
+                               const SubBlocksReference *sub_blocks_ref,
+                               ColumnVectorCache *cv_cache) const override;
 
-  ColumnVector* getAllValuesForJoin(
+  ColumnVectorPtr getAllValuesForJoin(
       const relation_id left_relation_id,
       ValueAccessor *left_accessor,
       const relation_id right_relation_id,
       ValueAccessor *right_accessor,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override;
+      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+      ColumnVectorCache *cv_cache) const override;
+
+ 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<const Expression*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const Expression*>> *container_child_fields) const override;
 
  private:
   void initHelper(bool own_children);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp b/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp
index 2d1064b..2de9e84 100644
--- a/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp
+++ b/expressions/scalar/tests/ScalarCaseExpression_unittest.cpp
@@ -223,8 +223,8 @@ TEST_F(ScalarCaseExpressionTest, BasicComparisonAndLiteralTest) {
       new ScalarLiteral(TypedValue(kVarChar, kThirdLawString, std::strlen(kThirdLawString) + 1),
                         varchar_type));
 
-  std::unique_ptr<ColumnVector> result_cv(
-      case_expr.getAllValues(&sample_data_value_accessor_, nullptr));
+  ColumnVectorPtr result_cv(
+      case_expr.getAllValues(&sample_data_value_accessor_, nullptr, nullptr));
   ASSERT_FALSE(result_cv->isNative());
   const IndirectColumnVector &indirect_result_cv
       = static_cast<const IndirectColumnVector&>(*result_cv);
@@ -308,8 +308,8 @@ TEST_F(ScalarCaseExpressionTest, BasicComparisonAndLiteralWithFilteredInputTest)
       new ScalarLiteral(TypedValue(kVarChar, kThirdLawString, std::strlen(kThirdLawString) + 1),
                         varchar_type));
 
-  std::unique_ptr<ColumnVector> result_cv(
-      case_expr.getAllValues(filtered_accessor.get(), nullptr));
+  ColumnVectorPtr result_cv(
+      case_expr.getAllValues(filtered_accessor.get(), nullptr, nullptr));
   ASSERT_FALSE(result_cv->isNative());
   const IndirectColumnVector &indirect_result_cv
       = static_cast<const IndirectColumnVector&>(*result_cv);
@@ -380,8 +380,8 @@ TEST_F(ScalarCaseExpressionTest, WhenClauseOrderTest) {
       new ScalarLiteral(TypedValue(kVarChar, kThirdLawString, std::strlen(kThirdLawString) + 1),
                         varchar_type));
 
-  std::unique_ptr<ColumnVector> result_cv(
-      case_expr.getAllValues(&sample_data_value_accessor_, nullptr));
+  ColumnVectorPtr result_cv(
+      case_expr.getAllValues(&sample_data_value_accessor_, nullptr, nullptr));
   ASSERT_FALSE(result_cv->isNative());
   const IndirectColumnVector &indirect_result_cv
       = static_cast<const IndirectColumnVector&>(*result_cv);
@@ -474,8 +474,8 @@ TEST_F(ScalarCaseExpressionTest, ComplexConjunctionAndCalculatedExpressionTest)
           new ScalarLiteral(TypedValue(5), int_type),
           new ScalarAttribute(*sample_relation_->getAttributeById(0))));
 
-  std::unique_ptr<ColumnVector> result_cv(
-      case_expr.getAllValues(&sample_data_value_accessor_, nullptr));
+  ColumnVectorPtr result_cv(
+      case_expr.getAllValues(&sample_data_value_accessor_, nullptr, nullptr));
   ASSERT_TRUE(result_cv->isNative());
   const NativeColumnVector &native_result_cv
       = static_cast<const NativeColumnVector&>(*result_cv);
@@ -597,8 +597,8 @@ TEST_F(ScalarCaseExpressionTest,
           new ScalarLiteral(TypedValue(5), int_type),
           new ScalarAttribute(*sample_relation_->getAttributeById(0))));
 
-  std::unique_ptr<ColumnVector> result_cv(
-      case_expr.getAllValues(filtered_accessor.get(), nullptr));
+  ColumnVectorPtr result_cv(
+      case_expr.getAllValues(filtered_accessor.get(), nullptr, nullptr));
   ASSERT_TRUE(result_cv->isNative());
   const NativeColumnVector &native_result_cv
       = static_cast<const NativeColumnVector&>(*result_cv);
@@ -707,8 +707,8 @@ TEST_F(ScalarCaseExpressionTest, ComplexDisjunctionAndCalculatedExpressionTest)
           new ScalarLiteral(TypedValue(5), int_type),
           new ScalarAttribute(*sample_relation_->getAttributeById(0))));
 
-  std::unique_ptr<ColumnVector> result_cv(
-      case_expr.getAllValues(&sample_data_value_accessor_, nullptr));
+  ColumnVectorPtr result_cv(
+      case_expr.getAllValues(&sample_data_value_accessor_, nullptr, nullptr));
   ASSERT_TRUE(result_cv->isNative());
   const NativeColumnVector &native_result_cv
       = static_cast<const NativeColumnVector&>(*result_cv);
@@ -827,8 +827,8 @@ TEST_F(ScalarCaseExpressionTest,
           new ScalarLiteral(TypedValue(5), int_type),
           new ScalarAttribute(*sample_relation_->getAttributeById(0))));
 
-  std::unique_ptr<ColumnVector> result_cv(
-      case_expr.getAllValues(filtered_accessor.get(), nullptr));
+  ColumnVectorPtr result_cv(
+      case_expr.getAllValues(filtered_accessor.get(), nullptr, nullptr));
   ASSERT_TRUE(result_cv->isNative());
   const NativeColumnVector &native_result_cv
       = static_cast<const NativeColumnVector&>(*result_cv);
@@ -929,12 +929,13 @@ TEST_F(ScalarCaseExpressionTest, JoinTest) {
     joined_tuple_ids.emplace_back(tuple_num, 1);
   }
 
-  std::unique_ptr<ColumnVector> result_cv(case_expr.getAllValuesForJoin(
+  ColumnVectorPtr result_cv(case_expr.getAllValuesForJoin(
       0,
       &sample_data_value_accessor_,
       1,
       &other_accessor,
-      joined_tuple_ids));
+      joined_tuple_ids,
+      nullptr));
   ASSERT_TRUE(result_cv->isNative());
   const NativeColumnVector &native_result_cv
       = static_cast<const NativeColumnVector&>(*result_cv);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 08b6467..c969f16 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -214,12 +214,15 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_logical_Logical
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_rules_AttachLIPFilters
+                      quickstep_queryoptimizer_rules_CollapseSelection
+                      quickstep_queryoptimizer_rules_ExtractCommonSubexpression
                       quickstep_queryoptimizer_rules_FuseAggregateJoin
                       quickstep_queryoptimizer_rules_InjectJoinFilters
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
                       quickstep_queryoptimizer_rules_ReduceGroupByAttributes
                       quickstep_queryoptimizer_rules_ReorderColumns
+                      quickstep_queryoptimizer_rules_ReuseAggregateExpressions
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_rules_SwapProbeBuild
                       quickstep_queryoptimizer_strategy_Aggregate

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index ac51c31..5f3b1b7 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -27,12 +27,15 @@
 #include "query_optimizer/logical/Logical.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/rules/AttachLIPFilters.hpp"
+#include "query_optimizer/rules/CollapseSelection.hpp"
+#include "query_optimizer/rules/ExtractCommonSubexpression.hpp"
 #include "query_optimizer/rules/FuseAggregateJoin.hpp"
 #include "query_optimizer/rules/InjectJoinFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
 #include "query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp"
 #include "query_optimizer/rules/ReduceGroupByAttributes.hpp"
 #include "query_optimizer/rules/ReorderColumns.hpp"
+#include "query_optimizer/rules/ReuseAggregateExpressions.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/rules/SwapProbeBuild.hpp"
 #include "query_optimizer/strategy/Aggregate.hpp"
@@ -121,12 +124,6 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   std::vector<std::unique_ptr<Rule<P::Physical>>> rules;
   rules.emplace_back(new PruneColumns());
 
-  // TODO(jianqiao): It is possible for PushDownLowCostDisjunctivePredicate to
-  // generate two chaining Selection nodes that can actually be fused into one.
-  // Note that currently it is okay to have the two Selections because they are
-  // applied to a small cardinality stored relation, which is very light-weight.
-  // However it is better to have a FuseSelection optimization (or even a more
-  // general FusePhysical optimization) in the future.
   rules.emplace_back(new PushDownLowCostDisjunctivePredicate());
 
   rules.emplace_back(new ReduceGroupByAttributes(optimizer_context_));
@@ -146,11 +143,31 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
     rules.emplace_back(new ReorderColumns());
   }
 
+  // This optimization pass eliminates duplicate aggregates and converts AVG to
+  // SUM/COUNT if appropriate. Note that this optimization needs to be done before
+  // ExtractCommonSubexpression.
+  rules.emplace_back(new ReuseAggregateExpressions(optimizer_context_));
+
   rules.emplace_back(new FuseAggregateJoin());
 
-  // NOTE(jianqiao): Adding rules after InjectJoinFilters (or AttachLIPFilters) requires
-  // extra handling of LIPFilterConfiguration for transformed nodes. So currently it is
-  // suggested that all the new rules be placed before this point.
+  // Some of the optimization passes (e.g. PushDownLowCostDisjunctivePredicate
+  // and ReuseAggregateExpressions) might add extra Selection nodes and extra
+  // projection columns for their convenience. So we collapse Selection nodes
+  // and prune unnecessary columns here.
+  rules.emplace_back(new CollapseSelection());
+  rules.emplace_back(new PruneColumns());
+
+  // This optimization pass identifies common subexpressions and wraps them with
+  // CommonSubexpression nodes, where identical CommonSubexpression nodes share
+  // a same unique integer ID. Later in the backend we use memoization tables to
+  // memorize the result column vectors for each ID so that each group has its
+  // common subexpression evaluated only once.
+  rules.emplace_back(new ExtractCommonSubexpression(optimizer_context_));
+
+  // NOTE(jianqiao): Adding rules after InjectJoinFilters (or AttachLIPFilters)
+  // requires extra handling of LIPFilterConfiguration for transformed nodes.
+  // So currently it is suggested that all the new rules be placed before this
+  // point.
   if (FLAGS_use_filter_joins) {
     rules.emplace_back(new InjectJoinFilters());
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/AttributeReference.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/AttributeReference.cpp b/query_optimizer/expressions/AttributeReference.cpp
index f0e49d4..facfb39 100644
--- a/query_optimizer/expressions/AttributeReference.cpp
+++ b/query_optimizer/expressions/AttributeReference.cpp
@@ -19,6 +19,8 @@
 
 #include "query_optimizer/expressions/AttributeReference.hpp"
 
+#include <cstddef>
+#include <functional>
 #include <string>
 #include <unordered_map>
 #include <vector>
@@ -26,6 +28,7 @@
 #include "expressions/scalar/ScalarAttribute.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/Expression.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
 
 #include "glog/logging.h"
 
@@ -57,6 +60,22 @@ std::vector<AttributeReferencePtr> AttributeReference::getReferencedAttributes()
   return new ::quickstep::ScalarAttribute(*found_it->second);
 }
 
+std::size_t AttributeReference::computeHash() const {
+  return std::hash<std::size_t>()(static_cast<std::size_t>(id()));
+}
+
+bool AttributeReference::equals(const ScalarPtr &other) const {
+  AttributeReferencePtr attr;
+  if (SomeAttributeReference::MatchesWithConditionalCast(other, &attr)) {
+    if (id() != attr->id()) {
+      return false;
+    }
+    DCHECK(type_.equals(attr->type_));
+    return true;
+  }
+  return false;
+}
+
 void AttributeReference::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/AttributeReference.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/AttributeReference.hpp b/query_optimizer/expressions/AttributeReference.hpp
index f5207b1..5ace894 100644
--- a/query_optimizer/expressions/AttributeReference.hpp
+++ b/query_optimizer/expressions/AttributeReference.hpp
@@ -20,6 +20,7 @@
 #ifndef QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_ATTRIBUTE_REFERENCE_HPP_
 #define QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_ATTRIBUTE_REFERENCE_HPP_
 
+#include <cstddef>
 #include <memory>
 #include <string>
 #include <unordered_map>
@@ -30,6 +31,7 @@
 #include "query_optimizer/expressions/Expression.hpp"
 #include "query_optimizer/expressions/ExpressionType.hpp"
 #include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/expressions/Scalar.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -88,6 +90,8 @@ class AttributeReference : public NamedExpression {
   ::quickstep::Scalar* concretize(
       const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const override;
 
+  bool equals(const ScalarPtr &other) const override;
+
   /**
    * @brief Creates an immutable AttributReference.
    *
@@ -114,6 +118,8 @@ class AttributeReference : public NamedExpression {
   }
 
  protected:
+  std::size_t computeHash() const override;
+
   void getFieldStringItems(
      std::vector<std::string> *inline_field_names,
      std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/BinaryExpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/BinaryExpression.cpp b/query_optimizer/expressions/BinaryExpression.cpp
index 446dd55..f49c6a2 100644
--- a/query_optimizer/expressions/BinaryExpression.cpp
+++ b/query_optimizer/expressions/BinaryExpression.cpp
@@ -19,6 +19,8 @@
 
 #include "query_optimizer/expressions/BinaryExpression.hpp"
 
+#include <algorithm>
+#include <cstddef>
 #include <string>
 #include <unordered_map>
 #include <vector>
@@ -31,6 +33,7 @@
 #include "query_optimizer/expressions/PatternMatcher.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
 #include "types/operations/binary_operations/BinaryOperationID.hpp"
+#include "utility/HashPair.hpp"
 
 #include "glog/logging.h"
 
@@ -104,6 +107,40 @@ std::vector<AttributeReferencePtr> BinaryExpression::getReferencedAttributes() c
       right_->concretize(substitution_map));
 }
 
+std::size_t BinaryExpression::computeHash() const {
+  std::size_t left_hash = left_->hash();
+  std::size_t right_hash = right_->hash();
+
+  if (operation_.isCommutative() && left_hash > right_hash) {
+    std::swap(left_hash, right_hash);
+  }
+
+  return CombineHashes(
+      CombineHashes(static_cast<std::size_t>(ExpressionType::kBinaryExpression),
+                    static_cast<std::size_t>(operation_.getBinaryOperationID())),
+      CombineHashes(left_hash, right_hash));
+}
+
+bool BinaryExpression::equals(const ScalarPtr &other) const {
+  BinaryExpressionPtr expr;
+  if (SomeBinaryExpression::MatchesWithConditionalCast(other, &expr) &&
+      &operation_ == &expr->operation_) {
+    ScalarPtr left = left_;
+    ScalarPtr right = right_;
+
+    if (operation_.isCommutative()) {
+      const bool self_order = (left_->hash() < right_->hash());
+      const bool other_order = (expr->left_->hash() < expr->right_->hash());
+      if (self_order ^ other_order) {
+        std::swap(left, right);
+      }
+    }
+
+    return left->equals(expr->left_) && right->equals(expr->right_);
+  }
+  return false;
+}
+
 void BinaryExpression::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/BinaryExpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/BinaryExpression.hpp b/query_optimizer/expressions/BinaryExpression.hpp
index 9b11ed1..6a37679 100644
--- a/query_optimizer/expressions/BinaryExpression.hpp
+++ b/query_optimizer/expressions/BinaryExpression.hpp
@@ -90,6 +90,8 @@ class BinaryExpression : public Scalar {
   ::quickstep::Scalar* concretize(
       const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const override;
 
+  bool equals(const ScalarPtr &other) const override;
+
   static BinaryExpressionPtr Create(const BinaryOperation &operation,
                                     const ScalarPtr &left,
                                     const ScalarPtr &right) {
@@ -97,6 +99,8 @@ class BinaryExpression : public Scalar {
   }
 
  protected:
+  std::size_t computeHash() const override;
+
   void getFieldStringItems(
       std::vector<std::string> *inline_field_names,
       std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/CMakeLists.txt b/query_optimizer/expressions/CMakeLists.txt
index 35fac90..3e7f8e4 100644
--- a/query_optimizer/expressions/CMakeLists.txt
+++ b/query_optimizer/expressions/CMakeLists.txt
@@ -21,7 +21,11 @@ add_library(quickstep_queryoptimizer_expressions_Alias Alias.cpp Alias.hpp)
 add_library(quickstep_queryoptimizer_expressions_AttributeReference AttributeReference.cpp AttributeReference.hpp)
 add_library(quickstep_queryoptimizer_expressions_BinaryExpression BinaryExpression.cpp BinaryExpression.hpp)
 add_library(quickstep_queryoptimizer_expressions_Cast Cast.cpp Cast.hpp)
-add_library(quickstep_queryoptimizer_expressions_ComparisonExpression ComparisonExpression.cpp
+add_library(quickstep_queryoptimizer_expressions_CommonSubexpression
+            CommonSubexpression.cpp
+            CommonSubexpression.hpp)
+add_library(quickstep_queryoptimizer_expressions_ComparisonExpression
+            ComparisonExpression.cpp
             ComparisonExpression.hpp)
 add_library(quickstep_queryoptimizer_expressions_Exists Exists.cpp Exists.hpp)
 add_library(quickstep_queryoptimizer_expressions_Expression ../../empty_src.cpp Expression.hpp)
@@ -43,7 +47,9 @@ add_library(quickstep_queryoptimizer_expressions_SearchedCase SearchedCase.cpp S
 add_library(quickstep_queryoptimizer_expressions_SimpleCase SimpleCase.cpp SimpleCase.hpp)
 add_library(quickstep_queryoptimizer_expressions_SubqueryExpression SubqueryExpression.cpp SubqueryExpression.hpp)
 add_library(quickstep_queryoptimizer_expressions_UnaryExpression UnaryExpression.cpp UnaryExpression.hpp)
-add_library(quickstep_queryoptimizer_expressions_WindowAggregateFunction WindowAggregateFunction.cpp WindowAggregateFunction.hpp)
+add_library(quickstep_queryoptimizer_expressions_WindowAggregateFunction
+            WindowAggregateFunction.cpp
+            WindowAggregateFunction.hpp)
 
 # Link dependencies:
 target_link_libraries(quickstep_queryoptimizer_expressions_AggregateFunction
@@ -78,6 +84,8 @@ target_link_libraries(quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_Expression
                       quickstep_queryoptimizer_expressions_ExpressionType
                       quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_expressions_PatternMatcher
+                      quickstep_queryoptimizer_expressions_Scalar
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_BinaryExpression
                       glog
@@ -91,6 +99,7 @@ target_link_libraries(quickstep_queryoptimizer_expressions_BinaryExpression
                       quickstep_queryoptimizer_expressions_Scalar
                       quickstep_types_operations_binaryoperations_BinaryOperation
                       quickstep_types_operations_binaryoperations_BinaryOperationID
+                      quickstep_utility_HashPair
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_Cast
                       glog
@@ -105,6 +114,18 @@ target_link_libraries(quickstep_queryoptimizer_expressions_Cast
                       quickstep_queryoptimizer_expressions_Scalar
                       quickstep_types_Type
                       quickstep_types_operations_unaryoperations_NumericCastOperation
+                      quickstep_utility_HashPair
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_expressions_CommonSubexpression
+                      glog
+                      quickstep_expressions_scalar_ScalarSharedExpression
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExprId
+                      quickstep_queryoptimizer_expressions_Expression
+                      quickstep_queryoptimizer_expressions_ExpressionType
+                      quickstep_queryoptimizer_expressions_PatternMatcher
+                      quickstep_queryoptimizer_expressions_Scalar
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_ComparisonExpression
                       glog
@@ -233,6 +254,7 @@ target_link_libraries(quickstep_queryoptimizer_expressions_Scalar
                       glog
                       quickstep_queryoptimizer_expressions_ExprId
                       quickstep_queryoptimizer_expressions_Expression
+                      quickstep_utility_HashError
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_ScalarLiteral
                       glog
@@ -242,9 +264,11 @@ target_link_libraries(quickstep_queryoptimizer_expressions_ScalarLiteral
                       quickstep_queryoptimizer_expressions_ExprId
                       quickstep_queryoptimizer_expressions_Expression
                       quickstep_queryoptimizer_expressions_ExpressionType
+                      quickstep_queryoptimizer_expressions_PatternMatcher
                       quickstep_queryoptimizer_expressions_Scalar
                       quickstep_types_Type
                       quickstep_types_TypedValue
+                      quickstep_utility_HashPair
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_SearchedCase
                       quickstep_expressions_predicate_Predicate
@@ -272,12 +296,14 @@ target_link_libraries(quickstep_queryoptimizer_expressions_SimpleCase
                       quickstep_queryoptimizer_expressions_ExprId
                       quickstep_queryoptimizer_expressions_Expression
                       quickstep_queryoptimizer_expressions_ExpressionType
+                      quickstep_queryoptimizer_expressions_PatternMatcher
                       quickstep_queryoptimizer_expressions_Predicate
                       quickstep_queryoptimizer_expressions_Scalar
                       quickstep_types_Type
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
                       quickstep_utility_Cast
+                      quickstep_utility_HashPair
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_SubqueryExpression
                       glog
@@ -301,6 +327,7 @@ target_link_libraries(quickstep_queryoptimizer_expressions_UnaryExpression
                       quickstep_queryoptimizer_expressions_Scalar
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_types_operations_unaryoperations_UnaryOperationID
+                      quickstep_utility_HashPair
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_WindowAggregateFunction
                       glog
@@ -324,6 +351,7 @@ target_link_libraries(quickstep_queryoptimizer_expressions
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_BinaryExpression
                       quickstep_queryoptimizer_expressions_Cast
+                      quickstep_queryoptimizer_expressions_CommonSubexpression
                       quickstep_queryoptimizer_expressions_ComparisonExpression
                       quickstep_queryoptimizer_expressions_Exists
                       quickstep_queryoptimizer_expressions_Expression

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/Cast.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Cast.cpp b/query_optimizer/expressions/Cast.cpp
index c0813c5..e6eb1bd 100644
--- a/query_optimizer/expressions/Cast.cpp
+++ b/query_optimizer/expressions/Cast.cpp
@@ -19,6 +19,7 @@
 
 #include "query_optimizer/expressions/Cast.hpp"
 
+#include <cstddef>
 #include <string>
 #include <unordered_map>
 #include <vector>
@@ -33,6 +34,7 @@
 #include "query_optimizer/expressions/Scalar.hpp"
 #include "types/Type.hpp"
 #include "types/operations/unary_operations/NumericCastOperation.hpp"
+#include "utility/HashPair.hpp"
 
 #include "glog/logging.h"
 
@@ -55,6 +57,21 @@ ExpressionPtr Cast::copyWithNewChildren(
                                                 operand_->concretize(substitution_map));
 }
 
+std::size_t Cast::computeHash() const {
+  return CombineHashes(
+      CombineHashes(static_cast<std::size_t>(ExpressionType::kCast),
+                    operand_->hash()),
+      static_cast<std::size_t>(target_type_.getTypeID()));
+}
+
+bool Cast::equals(const ScalarPtr &other) const {
+  CastPtr expr;
+  if (SomeCast::MatchesWithConditionalCast(other, &expr)) {
+    return operand_->equals(expr->operand_) && target_type_.equals(expr->target_type_);
+  }
+  return false;
+}
+
 void Cast::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/Cast.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Cast.hpp b/query_optimizer/expressions/Cast.hpp
index ac5bd02..11be775 100644
--- a/query_optimizer/expressions/Cast.hpp
+++ b/query_optimizer/expressions/Cast.hpp
@@ -78,6 +78,8 @@ class Cast : public Scalar {
   ::quickstep::Scalar* concretize(
       const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const override;
 
+  bool equals(const ScalarPtr &other) const override;
+
   /**
    * @brief Creates a Cast expression that converts \p operand to \p target_type.
    *
@@ -90,6 +92,8 @@ class Cast : public Scalar {
   }
 
  protected:
+  std::size_t computeHash() const override;
+
   void getFieldStringItems(
       std::vector<std::string> *inline_field_names,
       std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/CommonSubexpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/CommonSubexpression.cpp b/query_optimizer/expressions/CommonSubexpression.cpp
new file mode 100644
index 0000000..4b13a0e
--- /dev/null
+++ b/query_optimizer/expressions/CommonSubexpression.cpp
@@ -0,0 +1,72 @@
+/**
+ * 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/expressions/CommonSubexpression.hpp"
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "expressions/scalar/ScalarSharedExpression.hpp"
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/expressions/Expression.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
+#include "query_optimizer/expressions/Scalar.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace expressions {
+
+ExpressionPtr CommonSubexpression::copyWithNewChildren(
+    const std::vector<ExpressionPtr> &new_children) const {
+  DCHECK_EQ(new_children.size(), children().size());
+  DCHECK(SomeScalar::Matches(new_children[0]));
+  return CommonSubexpression::Create(
+      common_subexpression_id_,
+      std::static_pointer_cast<const Scalar>(new_children[0]));
+}
+
+::quickstep::Scalar* CommonSubexpression::concretize(
+    const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const {
+  return new ::quickstep::ScalarSharedExpression(
+      static_cast<int>(common_subexpression_id_),
+      operand_->concretize(substitution_map));
+}
+
+void CommonSubexpression::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 {
+  inline_field_names->push_back("common_subexpression_id");
+  inline_field_values->push_back(std::to_string(common_subexpression_id_));
+
+  non_container_child_field_names->push_back("Operand");
+  non_container_child_fields->push_back(operand_);
+}
+
+}  // namespace expressions
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/CommonSubexpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/CommonSubexpression.hpp b/query_optimizer/expressions/CommonSubexpression.hpp
new file mode 100644
index 0000000..ce7589d
--- /dev/null
+++ b/query_optimizer/expressions/CommonSubexpression.hpp
@@ -0,0 +1,141 @@
+/**
+ * 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_EXPRESSIONS_COMMON_SUBEXPRESSION_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_COMMON_SUBEXPRESSION_HPP_
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExprId.hpp"
+#include "query_optimizer/expressions/Expression.hpp"
+#include "query_optimizer/expressions/ExpressionType.hpp"
+#include "query_optimizer/expressions/Scalar.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class Scalar;
+class Type;
+
+namespace optimizer {
+namespace expressions {
+
+/** \addtogroup OptimizerExpressions
+ *  @{
+ */
+
+class CommonSubexpression;
+typedef std::shared_ptr<const CommonSubexpression> CommonSubexpressionPtr;
+
+/**
+ * @brief A wrapper expression that represents a common subexpression.
+ */
+class CommonSubexpression : public Scalar {
+ public:
+  ExpressionType getExpressionType() const override {
+    return ExpressionType::kCommonSubexpression;
+  }
+
+  std::string getName() const override {
+    return "CommonSubexpression";
+  }
+
+  bool isConstant() const override {
+    return operand_->isConstant();
+  }
+
+  /**
+   * @return The unique ID for the equivalence class that this common subexpression
+   *         belongs to.
+   */
+  inline ExprId common_subexpression_id() const {
+    return common_subexpression_id_;
+  }
+
+  /**
+   * @return The operand that represents the underlying subexpression.
+   */
+  const ScalarPtr& operand() const {
+    return operand_;
+  }
+
+  const Type& getValueType() const override {
+    return operand_->getValueType();
+  }
+
+  ExpressionPtr copyWithNewChildren(
+      const std::vector<ExpressionPtr> &new_children) const override;
+
+  std::vector<AttributeReferencePtr> getReferencedAttributes() const override {
+    return operand_->getReferencedAttributes();
+  }
+
+  ::quickstep::Scalar* concretize(
+      const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const override;
+
+  /**
+   * @brief Creates an immutable CommonSubexpression.
+   *
+   * @param common_subexpression_id A unique ID for the equivalence class that
+   *        this common subexpressions belongs to.
+   * @param operand The operand that represents the underlying subexpression.
+   * @return An immutable CommonSubexpression.
+   */
+  static CommonSubexpressionPtr Create(const ExprId common_subexpression_id,
+                                       const ScalarPtr &operand) {
+    return CommonSubexpressionPtr(
+        new CommonSubexpression(common_subexpression_id, operand));
+  }
+
+ 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:
+  CommonSubexpression(const ExprId common_subexpression_id,
+                      const ScalarPtr &operand)
+      : common_subexpression_id_(common_subexpression_id),
+        operand_(operand) {
+    addChild(operand);
+  }
+
+  ExprId common_subexpression_id_;
+  ScalarPtr operand_;
+
+  DISALLOW_COPY_AND_ASSIGN(CommonSubexpression);
+};
+
+/** @} */
+
+}  // namespace expressions
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_COMMON_SUBEXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/ExpressionType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ExpressionType.hpp b/query_optimizer/expressions/ExpressionType.hpp
index 5008f1d..ba7f639 100644
--- a/query_optimizer/expressions/ExpressionType.hpp
+++ b/query_optimizer/expressions/ExpressionType.hpp
@@ -32,11 +32,12 @@ namespace expressions {
  * @brief Optimizer expression types.
  **/
 enum class ExpressionType {
-  kAggregateFunction,
+  kAggregateFunction = 0,
   kAlias,
   kAttributeReference,
   kBinaryExpression,
   kCast,
+  kCommonSubexpression,
   kComparisonExpression,
   kExists,
   kInTableQuery,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/ExpressionUtil.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ExpressionUtil.hpp b/query_optimizer/expressions/ExpressionUtil.hpp
index 6b8666e..29d90f0 100644
--- a/query_optimizer/expressions/ExpressionUtil.hpp
+++ b/query_optimizer/expressions/ExpressionUtil.hpp
@@ -85,9 +85,9 @@ template <class NamedExpressionType1, class NamedExpressionType2>
 bool ContainsExpression(
     const std::vector<std::shared_ptr<const NamedExpressionType1>> &expressions,
     const std::shared_ptr<const NamedExpressionType2> &expression_to_check) {
-  for (const std::shared_ptr<const NamedExpressionType1> &expression :
-       expressions) {
-    if (expression->equals(expression_to_check)) {
+  for (const auto &expression : expressions) {
+    if (expression->id() == expression_to_check->id()) {
+      DCHECK(expression->getExpressionType() == expression_to_check->getExpressionType());
       return true;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/NamedExpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/NamedExpression.hpp b/query_optimizer/expressions/NamedExpression.hpp
index 9de8005..6725567 100644
--- a/query_optimizer/expressions/NamedExpression.hpp
+++ b/query_optimizer/expressions/NamedExpression.hpp
@@ -69,19 +69,6 @@ class NamedExpression : public Scalar {
    */
   inline const std::string& relation_name() const { return relation_name_; }
 
-  /**
-   * @brief Compares this named expression with \p other. Two named expressions
-   *        are equal if they have the same ExprId and are both Alias or
-   *        AttributeReference.
-   *
-   * @param other Another named expression to compare with.
-   * @return True if the named expression is equal to \p other.
-   */
-  inline bool equals(const NamedExpressionPtr &other) const {
-    return getExpressionType() == other->getExpressionType() &&
-           id_ == other->id();
-  }
-
  protected:
   /**
    * @brief Constructor.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/PatternMatcher.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/PatternMatcher.hpp b/query_optimizer/expressions/PatternMatcher.hpp
index 18d6b1c..e30a4d9 100644
--- a/query_optimizer/expressions/PatternMatcher.hpp
+++ b/query_optimizer/expressions/PatternMatcher.hpp
@@ -35,6 +35,7 @@ class Avg;
 class AttributeReference;
 class BinaryExpression;
 class Cast;
+class CommonSubexpression;
 class ComparisonExpression;
 class Count;
 class Exists;
@@ -50,6 +51,7 @@ class Predicate;
 class PredicateLiteral;
 class Scalar;
 class ScalarLiteral;
+class SimpleCase;
 class Sum;
 class UnaryExpression;
 class WindowAggregateFunction;
@@ -145,16 +147,13 @@ using SomeScalar = SomeExpressionNode<Scalar,
                                       ExpressionType::kAttributeReference,
                                       ExpressionType::kBinaryExpression,
                                       ExpressionType::kCast,
-                                      ExpressionType::kComparisonExpression,
-                                      ExpressionType::kLogicalAnd,
-                                      ExpressionType::kLogicalNot,
-                                      ExpressionType::kLogicalOr,
-                                      ExpressionType::kPredicateLiteral,
+                                      ExpressionType::kCommonSubexpression,
                                       ExpressionType::kScalarLiteral,
                                       ExpressionType::kSearchedCase,
                                       ExpressionType::kSimpleCase,
                                       ExpressionType::kUnaryExpression>;
 using SomeScalarLiteral = SomeExpressionNode<ScalarLiteral, ExpressionType::kScalarLiteral>;
+using SomeSimpleCase = SomeExpressionNode<SimpleCase, ExpressionType::kSimpleCase>;
 using SomeUnaryExpression = SomeExpressionNode<UnaryExpression, ExpressionType::kUnaryExpression>;
 using SomeWindowAggregateFunction = SomeExpressionNode<WindowAggregateFunction,
                                                        ExpressionType::kWindowAggregateFunction>;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/Scalar.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Scalar.hpp b/query_optimizer/expressions/Scalar.hpp
index 4870118..a163b21 100644
--- a/query_optimizer/expressions/Scalar.hpp
+++ b/query_optimizer/expressions/Scalar.hpp
@@ -20,11 +20,13 @@
 #ifndef QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_SCALAR_HPP_
 #define QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_SCALAR_HPP_
 
+#include <cstddef>
 #include <memory>
 #include <unordered_map>
 
 #include "query_optimizer/expressions/Expression.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
+#include "utility/HashError.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -65,10 +67,49 @@ class Scalar : public Expression {
       const std::unordered_map<ExprId, const CatalogAttribute*>& substitution_map)
       const = 0;
 
+  /**
+   * @brief Check whether this scalar is semantically equivalent to \p other.
+   *
+   * @note The fact that two scalars are semantically equal brings more
+   *       optimization opportunities, e.g. common subexpression elimination.
+   *       Meanwhile, it is always safe to assume that two scalars are not equal.
+   *
+   * @return True if this scalar equals \p other; false otherwise.
+   */
+  virtual bool equals(const ScalarPtr &other) const {
+    return false;
+  }
+
+  /**
+   * @brief Get a hash of this scalar.
+   *
+   * @return A hash of this scalar.
+   */
+  std::size_t hash() const {
+    if (hash_cache_ == nullptr) {
+      hash_cache_ = std::make_unique<std::size_t>(computeHash());
+    }
+    return *hash_cache_;
+  }
+
  protected:
   Scalar() {}
 
+  /**
+   * @brief Compute a hash of this scalar.
+   *
+   * @note Override this method to actually compute the hash. Note that the
+   *       public method hash() is a caching wrapper for this method.
+   *
+   * @return A hash of this scalar.
+   */
+  virtual std::size_t computeHash() const {
+    throw HashNotSupported("Unsupported computeHash() in " + getName());
+  }
+
  private:
+  mutable std::unique_ptr<std::size_t> hash_cache_;
+
   DISALLOW_COPY_AND_ASSIGN(Scalar);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/ScalarLiteral.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ScalarLiteral.cpp b/query_optimizer/expressions/ScalarLiteral.cpp
index d70c4cf..d2ab527 100644
--- a/query_optimizer/expressions/ScalarLiteral.cpp
+++ b/query_optimizer/expressions/ScalarLiteral.cpp
@@ -19,6 +19,7 @@
 
 #include "query_optimizer/expressions/ScalarLiteral.hpp"
 
+#include <cstddef>
 #include <string>
 #include <unordered_map>
 #include <vector>
@@ -28,7 +29,9 @@
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/Expression.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
 #include "types/Type.hpp"
+#include "utility/HashPair.hpp"
 
 #include "glog/logging.h"
 
@@ -51,6 +54,26 @@ ExpressionPtr ScalarLiteral::copyWithNewChildren(
   return new ::quickstep::ScalarLiteral(value_, value_type_);
 }
 
+std::size_t ScalarLiteral::computeHash() const {
+  std::size_t hash_code = static_cast<std::size_t>(ExpressionType::kScalarLiteral);
+  if (!value_.isNull()) {
+    hash_code = CombineHashes(hash_code, value_.getHash());
+  }
+  return hash_code;
+}
+
+bool ScalarLiteral::equals(const ScalarPtr &other) const {
+  ScalarLiteralPtr lit;
+  if (SomeScalarLiteral::MatchesWithConditionalCast(other, &lit) &&
+      value_type_.equals(lit->value_type_)) {
+    if (value_.isNull() || lit->value_.isNull()) {
+      return value_.isNull() && lit->value_.isNull();
+    }
+    return value_.fastEqualCheck(lit->value_);
+  }
+  return false;
+}
+
 void ScalarLiteral::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/ScalarLiteral.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/ScalarLiteral.hpp b/query_optimizer/expressions/ScalarLiteral.hpp
index 8ebc41c..bff52bb 100644
--- a/query_optimizer/expressions/ScalarLiteral.hpp
+++ b/query_optimizer/expressions/ScalarLiteral.hpp
@@ -20,6 +20,7 @@
 #ifndef QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_SCALAR_LITERAL_HPP_
 #define QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_SCALAR_LITERAL_HPP_
 
+#include <cstddef>
 #include <memory>
 #include <string>
 #include <unordered_map>
@@ -81,6 +82,8 @@ class ScalarLiteral : public Scalar {
   ::quickstep::Scalar* concretize(
       const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const override;
 
+  bool equals(const ScalarPtr &other) const override;
+
   /**
    * @brief Creates an immutable ScalarLiteral.
    * @param literal_value The literal value.
@@ -92,6 +95,8 @@ class ScalarLiteral : public Scalar {
   }
 
  protected:
+  std::size_t computeHash() const override;
+
   void getFieldStringItems(
       std::vector<std::string> *inline_field_names,
       std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/SimpleCase.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/SimpleCase.cpp b/query_optimizer/expressions/SimpleCase.cpp
index 454d7b9..ccdd8e5 100644
--- a/query_optimizer/expressions/SimpleCase.cpp
+++ b/query_optimizer/expressions/SimpleCase.cpp
@@ -31,12 +31,14 @@
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ComparisonExpression.hpp"
 #include "query_optimizer/expressions/Expression.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
 #include "query_optimizer/expressions/Predicate.hpp"
 #include "query_optimizer/expressions/Scalar.hpp"
 #include "types/Type.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "utility/Cast.hpp"
+#include "utility/HashPair.hpp"
 
 #include "glog/logging.h"
 
@@ -161,6 +163,50 @@ ExpressionPtr SimpleCase::copyWithNewChildren(const std::vector<ExpressionPtr> &
       else_result_expression.release());
 }
 
+std::size_t SimpleCase::computeHash() const {
+  std::size_t hash_code =
+      CombineHashes(static_cast<std::size_t>(ExpressionType::kSimpleCase),
+                    case_operand_->hash());
+  for (std::size_t i = 0; i < condition_operands_.size(); ++i) {
+    hash_code = CombineHashes(hash_code, condition_operands_[i]->hash());
+    hash_code = CombineHashes(hash_code, conditional_result_expressions_[i]->hash());
+  }
+  if (else_result_expression_ != nullptr) {
+    hash_code = CombineHashes(hash_code, else_result_expression_->hash());
+  }
+  return hash_code;
+}
+
+bool SimpleCase::equals(const ScalarPtr &other) const {
+  SimpleCasePtr expr;
+  if (!SomeSimpleCase::MatchesWithConditionalCast(other, &expr)) {
+    return false;
+  }
+  if (!case_operand_->equals(expr->case_operand_)) {
+    return false;
+  }
+  if (condition_operands_.size() != expr->condition_operands_.size()) {
+    return false;
+  }
+  for (std::size_t i = 0; i < condition_operands_.size(); ++i) {
+    if (!condition_operands_[i]->equals(expr->condition_operands_[i])
+        || !conditional_result_expressions_[i]->equals(
+                expr->conditional_result_expressions_[i])) {
+      return false;
+    }
+  }
+  if ((else_result_expression_ == nullptr
+       || expr->else_result_expression_ == nullptr)
+      && else_result_expression_ != expr->else_result_expression_) {
+    return false;
+  }
+  if (!else_result_expression_->equals(expr->else_result_expression_)) {
+    return false;
+  }
+  DCHECK(value_type_.equals(expr->value_type_));
+  return true;
+}
+
 void SimpleCase::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/SimpleCase.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/SimpleCase.hpp b/query_optimizer/expressions/SimpleCase.hpp
index 897d87f..0820fa3 100644
--- a/query_optimizer/expressions/SimpleCase.hpp
+++ b/query_optimizer/expressions/SimpleCase.hpp
@@ -20,6 +20,7 @@
 #ifndef QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_SIMPLE_CASE_HPP_
 #define QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_SIMPLE_CASE_HPP_
 
+#include <cstddef>
 #include <memory>
 #include <string>
 #include <unordered_map>
@@ -110,6 +111,8 @@ class SimpleCase : public Scalar {
   ::quickstep::Scalar* concretize(
       const std::unordered_map<ExprId, const CatalogAttribute*>& substitution_map) const override;
 
+  bool equals(const ScalarPtr &other) const override;
+
   /**
    * @brief Creates an immutable SimpleCase.
    *
@@ -136,6 +139,8 @@ class SimpleCase : public Scalar {
   }
 
  protected:
+  std::size_t computeHash() const override;
+
   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,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/UnaryExpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/UnaryExpression.cpp b/query_optimizer/expressions/UnaryExpression.cpp
index b0fff62..b448553 100644
--- a/query_optimizer/expressions/UnaryExpression.cpp
+++ b/query_optimizer/expressions/UnaryExpression.cpp
@@ -19,6 +19,7 @@
 
 #include "query_optimizer/expressions/UnaryExpression.hpp"
 
+#include <cstddef>
 #include <string>
 #include <unordered_map>
 #include <vector>
@@ -31,6 +32,7 @@
 #include "query_optimizer/expressions/Scalar.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
 #include "types/operations/unary_operations/UnaryOperationID.hpp"
+#include "utility/HashPair.hpp"
 
 #include "glog/logging.h"
 
@@ -56,6 +58,21 @@ ExpressionPtr UnaryExpression::copyWithNewChildren(
       operation_, operand_->concretize(substitution_map));
 }
 
+std::size_t UnaryExpression::computeHash() const {
+  return CombineHashes(
+      CombineHashes(static_cast<std::size_t>(ExpressionType::kUnaryExpression),
+                    static_cast<std::size_t>(operation_.getUnaryOperationID())),
+      operand_->hash());
+}
+
+bool UnaryExpression::equals(const ScalarPtr &other) const {
+  UnaryExpressionPtr expr;
+  if (SomeUnaryExpression::MatchesWithConditionalCast(other, &expr)) {
+    return &operation_ == &expr->operation_ && operand_->equals(expr->operand_);
+  }
+  return false;
+}
+
 void UnaryExpression::getFieldStringItems(
     std::vector<std::string> *inline_field_names,
     std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/expressions/UnaryExpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/UnaryExpression.hpp b/query_optimizer/expressions/UnaryExpression.hpp
index c4542d0..14201ff 100644
--- a/query_optimizer/expressions/UnaryExpression.hpp
+++ b/query_optimizer/expressions/UnaryExpression.hpp
@@ -20,6 +20,7 @@
 #ifndef QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_UNARY_EXPRESSION_HPP_
 #define QUICKSTEP_QUERY_OPTIMIZER_EXPRESSIONS_UNARY_EXPRESSION_HPP_
 
+#include <cstddef>
 #include <memory>
 #include <string>
 #include <unordered_map>
@@ -85,6 +86,8 @@ class UnaryExpression : public Scalar {
   ::quickstep::Scalar* concretize(
       const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const override;
 
+  bool equals(const ScalarPtr &other) const override;
+
   /**
    * @brief Creates an immutable UnaryExpression.
    *
@@ -99,6 +102,8 @@ class UnaryExpression : public Scalar {
   }
 
  protected:
+  std::size_t computeHash() const override;
+
   void getFieldStringItems(
       std::vector<std::string> *inline_field_names,
       std::vector<std::string> *inline_field_values,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 6847951..36e5959 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -21,6 +21,10 @@ add_subdirectory(tests)
 add_library(quickstep_queryoptimizer_rules_AttachLIPFilters AttachLIPFilters.cpp AttachLIPFilters.hpp)
 add_library(quickstep_queryoptimizer_rules_BottomUpRule ../../empty_src.cpp BottomUpRule.hpp)
 add_library(quickstep_queryoptimizer_rules_CollapseProject CollapseProject.cpp CollapseProject.hpp)
+add_library(quickstep_queryoptimizer_rules_CollapseSelection CollapseSelection.cpp CollapseSelection.hpp)
+add_library(quickstep_queryoptimizer_rules_ExtractCommonSubexpression
+            ExtractCommonSubexpression.cpp
+            ExtractCommonSubexpression.hpp)
 add_library(quickstep_queryoptimizer_rules_FuseAggregateJoin FuseAggregateJoin.cpp FuseAggregateJoin.hpp)
 add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp GenerateJoins.hpp)
 add_library(quickstep_queryoptimizer_rules_InjectJoinFilters InjectJoinFilters.cpp InjectJoinFilters.hpp)
@@ -34,6 +38,9 @@ add_library(quickstep_queryoptimizer_rules_ReduceGroupByAttributes
             ReduceGroupByAttributes.cpp
             ReduceGroupByAttributes.hpp)
 add_library(quickstep_queryoptimizer_rules_ReorderColumns ReorderColumns.cpp ReorderColumns.hpp)
+add_library(quickstep_queryoptimizer_rules_ReuseAggregateExpressions
+            ReuseAggregateExpressions.cpp
+            ReuseAggregateExpressions.hpp)
 add_library(quickstep_queryoptimizer_rules_Rule ../../empty_src.cpp Rule.hpp)
 add_library(quickstep_queryoptimizer_rules_RuleHelper RuleHelper.cpp RuleHelper.hpp)
 add_library(quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
@@ -77,6 +84,34 @@ target_link_libraries(quickstep_queryoptimizer_rules_CollapseProject
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_queryoptimizer_rules_RuleHelper
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_CollapseSelection
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_rules_BottomUpRule
+                      quickstep_queryoptimizer_rules_RuleHelper
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_ExtractCommonSubexpression
+                      glog
+                      quickstep_queryoptimizer_OptimizerContext
+                      quickstep_queryoptimizer_expressions_AggregateFunction
+                      quickstep_queryoptimizer_expressions_Alias
+                      quickstep_queryoptimizer_expressions_CommonSubexpression
+                      quickstep_queryoptimizer_expressions_Expression
+                      quickstep_queryoptimizer_expressions_ExpressionType
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_expressions_PatternMatcher
+                      quickstep_queryoptimizer_expressions_Scalar
+                      quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_NestedLoopsJoin
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_HashError
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_FuseAggregateJoin
                       quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
                       quickstep_queryoptimizer_expressions_AggregateFunction
@@ -200,6 +235,34 @@ target_link_libraries(quickstep_queryoptimizer_rules_ReorderColumns
                       quickstep_queryoptimizer_physical_TableReference
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_ReuseAggregateExpressions
+                      ${GFLAGS_LIB_NAME}
+                      glog
+                      quickstep_expressions_aggregation_AggregateFunction
+                      quickstep_expressions_aggregation_AggregateFunctionFactory
+                      quickstep_expressions_aggregation_AggregationID
+                      quickstep_queryoptimizer_OptimizerContext
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
+                      quickstep_queryoptimizer_expressions_AggregateFunction
+                      quickstep_queryoptimizer_expressions_Alias
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_BinaryExpression
+                      quickstep_queryoptimizer_expressions_ExpressionType
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_NamedExpression
+                      quickstep_queryoptimizer_expressions_Scalar
+                      quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_BottomUpRule
+                      quickstep_types_operations_binaryoperations_BinaryOperation
+                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
+                      quickstep_types_operations_binaryoperations_BinaryOperationID
+                      quickstep_utility_HashError
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_Rule
                       glog
                       quickstep_utility_Macros)
@@ -311,6 +374,8 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_AttachLIPFilters
                       quickstep_queryoptimizer_rules_BottomUpRule
                       quickstep_queryoptimizer_rules_CollapseProject
+                      quickstep_queryoptimizer_rules_CollapseSelection
+                      quickstep_queryoptimizer_rules_ExtractCommonSubexpression
                       quickstep_queryoptimizer_rules_FuseAggregateJoin
                       quickstep_queryoptimizer_rules_GenerateJoins
                       quickstep_queryoptimizer_rules_InjectJoinFilters
@@ -320,6 +385,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
                       quickstep_queryoptimizer_rules_ReduceGroupByAttributes
                       quickstep_queryoptimizer_rules_ReorderColumns
+                      quickstep_queryoptimizer_rules_ReuseAggregateExpressions
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_queryoptimizer_rules_RuleHelper
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/query_optimizer/rules/CollapseSelection.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CollapseSelection.cpp b/query_optimizer/rules/CollapseSelection.cpp
new file mode 100644
index 0000000..e5427b4
--- /dev/null
+++ b/query_optimizer/rules/CollapseSelection.cpp
@@ -0,0 +1,59 @@
+/**
+ * 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/CollapseSelection.hpp"
+
+#include <vector>
+
+#include "query_optimizer/expressions/NamedExpression.hpp"
+#include "query_optimizer/physical/PatternMatcher.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/Selection.hpp"
+#include "query_optimizer/rules/RuleHelper.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr CollapseSelection::applyToNode(const P::PhysicalPtr &input) {
+  P::SelectionPtr selection;
+  P::SelectionPtr child_selection;
+
+  // TODO(jianqiao): Handle the case where filter predicates are present.
+  if (P::SomeSelection::MatchesWithConditionalCast(input, &selection) &&
+      P::SomeSelection::MatchesWithConditionalCast(selection->input(), &child_selection) &&
+      selection->filter_predicate() == nullptr &&
+      child_selection->filter_predicate() == nullptr) {
+    std::vector<E::NamedExpressionPtr> project_expressions =
+        selection->project_expressions();
+    PullUpProjectExpressions(child_selection->project_expressions(),
+                             {} /* non_project_expression_lists */,
+                             {&project_expressions} /* project_expression_lists */);
+    return P::Selection::Create(child_selection->input(),
+                                project_expressions,
+                                selection->filter_predicate());
+  }
+
+  return input;
+}
+
+}  // namespace optimizer
+}  // namespace quickstep



[21/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/MultiplyBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/MultiplyBinaryOperation.cpp b/types/operations/binary_operations/MultiplyBinaryOperation.cpp
deleted file mode 100644
index a206364..0000000
--- a/types/operations/binary_operations/MultiplyBinaryOperation.cpp
+++ /dev/null
@@ -1,410 +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 "types/operations/binary_operations/MultiplyBinaryOperation.hpp"
-
-#include <string>
-#include <utility>
-
-#include "types/DateOperatorOverloads.hpp"
-#include "types/DatetimeIntervalType.hpp"
-#include "types/DoubleType.hpp"
-#include "types/FloatType.hpp"
-#include "types/IntType.hpp"
-#include "types/IntervalLit.hpp"
-#include "types/LongType.hpp"
-#include "types/Type.hpp"
-#include "types/TypeErrors.hpp"
-#include "types/TypeFactory.hpp"
-#include "types/TypeID.hpp"
-#include "types/YearMonthIntervalType.hpp"
-#include "types/operations/binary_operations/ArithmeticBinaryOperators.hpp"
-#include "utility/EqualsAnyConstant.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-bool MultiplyBinaryOperation::canApplyToTypes(const Type &left, const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      return (right.getSuperTypeID() == Type::kNumeric ||
-              right.getTypeID() == kDatetimeInterval   ||
-              right.getTypeID() == kYearMonthInterval);
-    }
-    case kDatetimeInterval:
-    case kYearMonthInterval: {
-      return (right.getSuperTypeID() == Type::kNumeric);
-    }
-    default:
-      return false;
-  }
-}
-
-const Type* MultiplyBinaryOperation::resultTypeForArgumentTypes(const Type &left, const Type &right) const {
-  if (left.getSuperTypeID() == Type::kNumeric && right.getSuperTypeID() == Type::kNumeric) {
-    return TypeFactory::GetUnifyingType(left, right);
-  } else if ((left.getSuperTypeID() == Type::kNumeric && right.getTypeID() == kDatetimeInterval) ||
-             (left.getTypeID() == kDatetimeInterval && right.getSuperTypeID() == Type::kNumeric)) {
-    return &(DatetimeIntervalType::Instance(left.isNullable() || right.isNullable()));
-  } else if ((left.getSuperTypeID() == Type::kNumeric && right.getTypeID() == kYearMonthInterval) ||
-             (left.getTypeID() == kYearMonthInterval && right.getSuperTypeID() == Type::kNumeric)) {
-    return &(YearMonthIntervalType::Instance(left.isNullable() || right.isNullable()));
-  } else {
-    return nullptr;
-  }
-}
-
-const Type* MultiplyBinaryOperation::resultTypeForPartialArgumentTypes(
-    const Type *left,
-    const Type *right) const {
-  if ((left == nullptr) && (right == nullptr)) {
-    return nullptr;
-  }
-
-  if ((left != nullptr) && (right != nullptr)) {
-    return resultTypeForArgumentTypes(*left, *right);
-  }
-
-  // Multiplication is commutative, so we just determine based on the known
-  // type, left or right.
-  const Type *known_type = (left != nullptr) ? left : right;
-  switch (known_type->getTypeID()) {
-    case kDatetimeInterval:
-      // DatetimeInterval can be multiplied against any numeric type, yielding
-      // DatetimeInterval.
-      return &TypeFactory::GetType(kDatetimeInterval, true);
-    case kYearMonthInterval:
-      // Same deal for YearMonthInterval.
-      return &TypeFactory::GetType(kYearMonthInterval, true);
-    default:
-      // Ambiguous or inapplicable. Note that we can't apply numeric precedence
-      // order for a Double argument, because the other argument could be a
-      // numeric type OR an interval type.
-      return nullptr;
-  }
-}
-
-bool MultiplyBinaryOperation::partialTypeSignatureIsPlausible(
-    const Type *result_type,
-    const Type *left_argument_type,
-    const Type *right_argument_type) const {
-  if ((left_argument_type == nullptr) && (right_argument_type == nullptr)) {
-    if (result_type == nullptr) {
-      return true;
-    } else if (!result_type->isNullable()) {
-      // Unknown arguments are assumed to be nullable, since they arise from
-      // untyped NULL literals in the parser. Therefore, a non-nullable result
-      // Type is not plausible with unknown arguments.
-      return false;
-    } else {
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          result_type->getTypeID(),
-          kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-    }
-  }
-
-  if ((left_argument_type != nullptr) && (right_argument_type != nullptr)) {
-    const Type *actual_result_type = resultTypeForArgumentTypes(*left_argument_type,
-                                                                *right_argument_type);
-    if (actual_result_type == nullptr) {
-      // Both argument Types are known, but this operation is NOT applicable to
-      // them. No matter what the result_type is, the signature is not
-      // plausible.
-      return false;
-    } else if (result_type == nullptr) {
-      return true;
-    } else {
-      return result_type->equals(*actual_result_type);
-    }
-  }
-
-  // Multiplication is commutative, so we just determine based on the known
-  // type, left or right.
-  const Type *known_argument_type = (left_argument_type != nullptr)
-                                    ? left_argument_type
-                                    : right_argument_type;
-  if (result_type == nullptr) {
-    return QUICKSTEP_EQUALS_ANY_CONSTANT(
-        known_argument_type->getTypeID(),
-        kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-  }
-
-  if (!result_type->isNullable()) {
-    // One of the arguments is unknown, but it is nevertheless assumed
-    // nullable, since unknown argument Types arise from untyped NULL literals
-    // in the parser. Therefore, a non-nullable result Type is not plausible
-    // with an unknown argument.
-    return false;
-  }
-
-  switch (result_type->getTypeID()) {
-    case kInt:
-      return (known_argument_type->getTypeID() == kInt);
-    case kLong:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kInt, kLong);
-    case kFloat:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kInt, kFloat);
-    case kDouble:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kInt, kLong, kFloat, kDouble);
-    case kDatetimeInterval:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kInt, kLong, kFloat, kDouble, kDatetimeInterval);
-    case kYearMonthInterval:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kInt, kLong, kFloat, kDouble, kYearMonthInterval);
-    default:
-      return false;
-  }
-}
-
-std::pair<const Type*, const Type*> MultiplyBinaryOperation::pushDownTypeHint(
-    const Type *result_type_hint) const {
-  if (result_type_hint == nullptr) {
-    return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-
-  switch (result_type_hint->getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble:
-      return std::pair<const Type*, const Type*>(result_type_hint, result_type_hint);
-    case kDatetimeInterval:
-    case kYearMonthInterval:
-      // Ambiguous hint. One of the arguments should be the same as the
-      // '*type_hint', the other can be any numeric type, but either order is
-      // OK.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-    default:
-      // Inapplicable.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-}
-
-TypedValue MultiplyBinaryOperation::applyToChecked(const TypedValue &left,
-                                                   const Type &left_type,
-                                                   const TypedValue &right,
-                                                   const Type &right_type) const {
-  switch (left_type.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      if (right_type.getSuperTypeID() == Type::kNumeric) {
-        return applyToCheckedNumericHelper<MultiplyFunctor>(left, left_type,
-                                                            right, right_type);
-      } else if (right_type.getTypeID() == kDatetimeInterval) {
-        return applyToCheckedIntervalMultiplyNumericHelper<DatetimeIntervalType>(right, right_type,
-                                                                                 left, left_type);
-      } else if (right_type.getTypeID() == kYearMonthInterval) {
-        return applyToCheckedIntervalMultiplyNumericHelper<YearMonthIntervalType>(right, right_type,
-                                                                                  left, left_type);
-      }
-      break;
-    }
-    case kDatetimeInterval: {
-      if (right_type.getSuperTypeID() == Type::kNumeric) {
-        return applyToCheckedIntervalMultiplyNumericHelper<DatetimeIntervalType>(left, left_type,
-                                                                                 right, right_type);
-      }
-      break;
-    }
-    case kYearMonthInterval: {
-      if (right_type.getSuperTypeID() == Type::kNumeric) {
-        return applyToCheckedIntervalMultiplyNumericHelper<YearMonthIntervalType>(left, left_type,
-                                                                                  right, right_type);
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-             << left_type.getName() << " and " << right_type.getName();
-}
-
-UncheckedBinaryOperator* MultiplyBinaryOperation::makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                                                      const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt: {
-      if (right.getSuperTypeID() == Type::kNumeric) {
-        return makeNumericBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator>(left, right);
-      } else if (right.getTypeID() == kDatetimeInterval) {
-        return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                 DatetimeIntervalType,
-                                                 IntType::cpptype, DatetimeIntervalLit>(left, right);
-      } else if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                 YearMonthIntervalType,
-                                                 IntType::cpptype, YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kLong: {
-      if (right.getSuperTypeID() == Type::kNumeric) {
-        return makeNumericBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator>(left, right);
-      } else if (right.getTypeID() == kDatetimeInterval) {
-        return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                 DatetimeIntervalType,
-                                                 LongType::cpptype, DatetimeIntervalLit>(left, right);
-      } else if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                 YearMonthIntervalType,
-                                                 LongType::cpptype, YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kFloat: {
-      if (right.getSuperTypeID() == Type::kNumeric) {
-        return makeNumericBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator>(left, right);
-      } else if (right.getTypeID() == kDatetimeInterval) {
-        return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                 DatetimeIntervalType,
-                                                 FloatType::cpptype, DatetimeIntervalLit>(left, right);
-      } else if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                 YearMonthIntervalType,
-                                                 FloatType::cpptype, YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kDouble: {
-      if (right.getSuperTypeID() == Type::kNumeric) {
-        return makeNumericBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator>(left, right);
-      } else if (right.getTypeID() == kDatetimeInterval) {
-        return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                 DatetimeIntervalType,
-                                                 DoubleType::cpptype, DatetimeIntervalLit>(left, right);
-      } else if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                 YearMonthIntervalType,
-                                                 DoubleType::cpptype, YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kDatetimeInterval: {
-      switch (right.getTypeID()) {
-        case kInt: {
-          return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                   DatetimeIntervalType,
-                                                   DatetimeIntervalLit, IntType::cpptype>(left, right);
-        }
-        case kLong: {
-          return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                   DatetimeIntervalType,
-                                                   DatetimeIntervalLit, LongType::cpptype>(left, right);
-        }
-        case kFloat: {
-          return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                   DatetimeIntervalType,
-                                                   DatetimeIntervalLit, FloatType::cpptype>(left, right);
-        }
-        case kDouble: {
-          return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                   DatetimeIntervalType,
-                                                   DatetimeIntervalLit, DoubleType::cpptype>(left, right);
-        }
-        default:
-          break;
-      }
-      break;
-    }
-    case kYearMonthInterval: {
-      switch (right.getTypeID()) {
-        case kInt: {
-          return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                   YearMonthIntervalType,
-                                                   YearMonthIntervalLit, IntType::cpptype>(left, right);
-        }
-        case kLong: {
-          return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                   YearMonthIntervalType,
-                                                   YearMonthIntervalLit, LongType::cpptype>(left, right);
-        }
-        case kFloat: {
-          return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                   YearMonthIntervalType,
-                                                   YearMonthIntervalLit, FloatType::cpptype>(left, right);
-        }
-        case kDouble: {
-          return makeDateBinaryOperatorOuterHelper<MultiplyArithmeticUncheckedBinaryOperator,
-                                                   YearMonthIntervalType,
-                                                   YearMonthIntervalLit, DoubleType::cpptype>(left, right);
-        }
-        default:
-          break;
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-}
-
-template <typename IntervalType>
-TypedValue MultiplyBinaryOperation::applyToCheckedIntervalMultiplyNumericHelper(
-    const TypedValue &left,
-    const Type &left_type,
-    const TypedValue &right,
-    const Type &right_type) const {
-  DCHECK(IntervalType::kStaticTypeID == kDatetimeInterval ||
-         IntervalType::kStaticTypeID == kYearMonthInterval);
-  DCHECK(IntervalType::kStaticTypeID == left_type.getTypeID());
-  DCHECK_EQ(Type::kNumeric, right_type.getSuperTypeID());
-
-  if (left.isNull() || right.isNull()) {
-    return TypedValue(IntervalType::kStaticTypeID);
-  }
-
-  switch (right_type.getTypeID()) {
-    case kInt: {
-      return TypedValue(left.getLiteral<typename IntervalType::cpptype>() * right.getLiteral<IntType::cpptype>());
-    }
-    case kLong: {
-      return TypedValue(left.getLiteral<typename IntervalType::cpptype>() * right.getLiteral<LongType::cpptype>());
-    }
-    case kFloat: {
-      return TypedValue(left.getLiteral<typename IntervalType::cpptype>() * right.getLiteral<FloatType::cpptype>());
-    }
-    case kDouble: {
-      return TypedValue(left.getLiteral<typename IntervalType::cpptype>() * right.getLiteral<DoubleType::cpptype>());
-    }
-    default: {
-      LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-                 << left_type.getName() << " and " << right_type.getName();
-    }
-  }
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/SubtractBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/SubtractBinaryOperation.cpp b/types/operations/binary_operations/SubtractBinaryOperation.cpp
deleted file mode 100644
index 53e4266..0000000
--- a/types/operations/binary_operations/SubtractBinaryOperation.cpp
+++ /dev/null
@@ -1,459 +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 "types/operations/binary_operations/SubtractBinaryOperation.hpp"
-
-#include <string>
-#include <utility>
-
-#include "types/DateOperatorOverloads.hpp"
-#include "types/DateType.hpp"
-#include "types/DatetimeIntervalType.hpp"
-#include "types/DatetimeLit.hpp"
-#include "types/DatetimeType.hpp"
-#include "types/IntervalLit.hpp"
-#include "types/Type.hpp"
-#include "types/TypeErrors.hpp"
-#include "types/TypeFactory.hpp"
-#include "types/TypeID.hpp"
-#include "types/YearMonthIntervalType.hpp"
-#include "types/operations/binary_operations/ArithmeticBinaryOperators.hpp"
-#include "utility/EqualsAnyConstant.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-bool SubtractBinaryOperation::canApplyToTypes(const Type &left, const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      return (right.getSuperTypeID() == Type::kNumeric);
-    }
-    case kDate: {
-      return (right.getTypeID() == kYearMonthInterval);
-    }
-    case kDatetime: {
-      return (right.getTypeID() == kDatetime         ||
-              right.getTypeID() == kDatetimeInterval ||
-              right.getTypeID() == kYearMonthInterval);
-    }
-    case kDatetimeInterval: {
-      return (right.getTypeID() == kDatetimeInterval);
-    }
-    case kYearMonthInterval: {
-      return (right.getTypeID() == kYearMonthInterval ||
-              right.getTypeID() == kDate);
-    }
-    default:
-      return false;
-  }
-}
-
-const Type* SubtractBinaryOperation::resultTypeForArgumentTypes(const Type &left, const Type &right) const {
-  if (left.getSuperTypeID() == Type::kNumeric && right.getSuperTypeID() == Type::kNumeric) {
-    return TypeFactory::GetUnifyingType(left, right);
-  } else if ((left.getTypeID() == kDate && right.getTypeID() == kYearMonthInterval)) {
-    // For DATE type, only one possibility: DATE - YEAR-MONTH-INTERVAL.
-    return &(DateType::Instance(left.isNullable() || right.isNullable()));
-  } else if ((left.getTypeID() == kDatetime && right.getTypeID() == kDatetime) ||
-             (left.getTypeID() == kDatetimeInterval && right.getTypeID() == kDatetimeInterval)) {
-    // NOTE(zuyu): we set the result type of the Subtract
-    // between two Datetimes as DatetimeInterval, instead of YearMonthInterval.
-    return &(DatetimeIntervalType::Instance(left.isNullable() || right.isNullable()));
-  } else if (left.getTypeID() == kDatetime && right.getTypeID() == kDatetimeInterval) {
-    return &(DatetimeType::Instance(left.isNullable() || right.isNullable()));
-  } else if (left.getTypeID() == kDatetime && right.getTypeID() == kYearMonthInterval) {
-    return &(DatetimeType::Instance(left.isNullable() || right.isNullable()));
-  } else if (left.getTypeID() == kYearMonthInterval && right.getTypeID() == kYearMonthInterval) {
-    return &(YearMonthIntervalType::Instance(left.isNullable() || right.isNullable()));
-  } else {
-    return nullptr;
-  }
-}
-
-const Type* SubtractBinaryOperation::resultTypeForPartialArgumentTypes(
-    const Type *left,
-    const Type *right) const {
-  if (left == nullptr) {
-    if (right == nullptr) {
-      return nullptr;
-    } else {
-      switch (right->getTypeID()) {
-        case kDouble:
-          // Double has highest precedence of numeric types.
-          return &TypeFactory::GetType(kDouble, true);
-        case kDatetime:
-          // If the subtrahend is Datetime, then the only allowed minuend is
-          // another Datetime, and the result is an interval.
-          return &TypeFactory::GetType(kDatetimeInterval, true);
-        default:
-          // Ambiguous or inapplicable.
-          return nullptr;
-      }
-    }
-  } else {
-    if (right == nullptr) {
-      switch (left->getTypeID()) {
-        case kDouble:
-          // Double has highest precedence of numeric types.
-          return &TypeFactory::GetType(kDouble, true);
-        case kDate:
-          // If left is a Date, right must be a YearMonthInterval and the result
-          // must be a Date.
-          return &TypeFactory::GetType(kDate, true);
-        case kDatetimeInterval:
-          // If minuend is a DatetimeInterval, the subtrahend and result must
-          // also be DatetimeInterval.
-          return &TypeFactory::GetType(kDatetimeInterval, true);
-        case kYearMonthInterval:
-          // Similarly, if minuend is a YearMonthInterval, the subtrahend and
-          // result must also be YearMonthInterval.
-          return &TypeFactory::GetType(kYearMonthInterval, true);
-        default:
-          // Ambiguous or inapplicable.
-          return nullptr;
-      }
-    } else {
-      return resultTypeForArgumentTypes(*left, *right);
-    }
-  }
-}
-
-bool SubtractBinaryOperation::partialTypeSignatureIsPlausible(
-    const Type *result_type,
-    const Type *left_argument_type,
-    const Type *right_argument_type) const {
-  // Early check: if either argument type is nullable or unknown, result type
-  // must also be nullable.
-  if ((left_argument_type == nullptr)
-      || left_argument_type->isNullable()
-      || (right_argument_type == nullptr)
-      || right_argument_type->isNullable()) {
-    if ((result_type != nullptr) && (!result_type->isNullable())) {
-      return false;
-    }
-  }
-
-  if (left_argument_type == nullptr) {
-    if (right_argument_type == nullptr) {
-      if (result_type == nullptr) {
-        // All types unknown.
-        return true;
-      } else {
-        // Only result type is known, just check that it is one of the types
-        // that can possibly be returned.
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(result_type->getTypeID(),
-                                             kInt,
-                                             kLong,
-                                             kFloat,
-                                             kDouble,
-                                             kDate,
-                                             kDatetime,
-                                             kDatetimeInterval,
-                                             kYearMonthInterval);
-      }
-    }
-
-    if (result_type == nullptr) {
-      // Right (minuend) argument type is known, left (subtrahend) argument and
-      // result types are unknown. Just check that right (minuend) type can be
-      // subtracted.
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(right_argument_type->getTypeID(),
-                                           kInt,
-                                           kLong,
-                                           kFloat,
-                                           kDouble,
-                                           kDatetime,
-                                           kDatetimeInterval,
-                                           kYearMonthInterval);
-    }
-
-    // Return type and right (minuend) argument type are known, left
-    // (subtrahend) argument type is unknown. Check that result and subtrahend
-    // are compatible.
-    switch (right_argument_type->getTypeID()) {
-      case kInt:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble);
-      case kLong:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kLong, kDouble);
-      case kFloat:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kFloat, kDouble);
-      case kDouble:
-        return (result_type->getTypeID() == kDouble);
-      case kDate:
-        return (result_type->getTypeID() == kDate);
-      case kDatetime:
-        return (result_type->getTypeID() == kDatetimeInterval);
-      case kDatetimeInterval:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kDatetime, kDatetimeInterval);
-      case kYearMonthInterval:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kDate, kDatetime, kYearMonthInterval);
-      default:
-        return false;
-    }
-  } else {  // left_argument_type != nullptr
-    if (right_argument_type == nullptr) {
-      if (result_type == nullptr) {
-        // Left (subtrahend) argument type is known, right (minuend) argument
-        // type and result type are unknown. Just check that the left
-        // (subtrahend) type can be subtracted from.
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(left_argument_type->getTypeID(),
-                                             kInt,
-                                             kLong,
-                                             kFloat,
-                                             kDouble,
-                                             kDate,
-                                             kDatetime,
-                                             kDatetimeInterval,
-                                             kYearMonthInterval);
-      }
-
-      // Result type and left (subtrahend) argument type are known, but right
-      // (minuend) argument type is unknown. Check that result and minuend are
-      // compatible.
-      switch (left_argument_type->getTypeID()) {
-        case kInt:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kInt, kLong, kFloat, kDouble);
-        case kLong:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kLong, kDouble);
-        case kFloat:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kFloat, kDouble);
-        case kDouble:
-          return (result_type->getTypeID() == kDouble);
-        case kDate:
-          return (result_type->getTypeID() == kDate);
-        case kDatetime:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kDatetime, kDatetimeInterval);
-        case kDatetimeInterval:
-          return (result_type->getTypeID() == kDatetimeInterval);
-        case kYearMonthInterval:
-          return (result_type->getTypeID() == kYearMonthInterval);
-        default:
-          return false;
-      }
-    }
-
-    // Left and right (subtrahend and minuend) argument types are both known.
-    const Type *actual_result_type = resultTypeForArgumentTypes(*left_argument_type,
-                                                                *right_argument_type);
-    if (actual_result_type == nullptr) {
-      // Both argument Types are known, but this operation is NOT applicable to
-      // them. No matter what the result_type is, the signature is not
-      // plausible.
-      return false;
-    } else if (result_type == nullptr) {
-      return true;
-    } else {
-      // Check if result type matches.
-      return result_type->equals(*actual_result_type);
-    }
-  }
-}
-
-std::pair<const Type*, const Type*> SubtractBinaryOperation::pushDownTypeHint(
-    const Type *result_type_hint) const {
-  if (result_type_hint == nullptr) {
-    return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-
-  switch (result_type_hint->getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble:
-    case kYearMonthInterval:
-      return std::pair<const Type*, const Type*>(result_type_hint, result_type_hint);
-    case kDate:
-      // Left should be a Date, right should be YearMonthInterval.
-      return std::pair<const Type *, const Type *>(
-          result_type_hint, &TypeFactory::GetType(kYearMonthInterval, true));
-    case kDatetime:
-      // Left should be a Datetime, right may be either interval type.
-      return std::pair<const Type*, const Type*>(result_type_hint, nullptr);
-    case kDatetimeInterval:
-      // Ambiguous: could be subtracting two Datetimes or two DatetimeIntervals.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-    default:
-      // Inapplicable.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-}
-
-TypedValue SubtractBinaryOperation::applyToChecked(const TypedValue &left,
-                                                   const Type &left_type,
-                                                   const TypedValue &right,
-                                                   const Type &right_type) const {
-  switch (left_type.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      if (right_type.getSuperTypeID() == Type::kNumeric) {
-        return applyToCheckedNumericHelper<SubtractFunctor>(left, left_type,
-                                                            right, right_type);
-      }
-      break;
-    }
-    case kDate: {
-      if (right_type.getTypeID() == kYearMonthInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDate);
-        }
-
-        return TypedValue(left.getLiteral<DateLit>() - right.getLiteral<YearMonthIntervalLit>());
-      }
-      break;
-    }
-    case kDatetime: {
-      if (right_type.getTypeID() == kDatetime) {
-        // NOTE(zuyu): The result type of the Subtract between two Datetimes is DatetimeInterval,
-        // instead of YearMonthInterval.
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetimeInterval);
-        }
-
-        return TypedValue(left.getLiteral<DatetimeLit>() - right.getLiteral<DatetimeLit>());
-      } else if (right_type.getTypeID() == kDatetimeInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetime);
-        }
-
-        return TypedValue(left.getLiteral<DatetimeLit>() - right.getLiteral<DatetimeIntervalLit>());
-      } else if (right_type.getTypeID() == kYearMonthInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetime);
-        }
-
-        return TypedValue(left.getLiteral<DatetimeLit>() - right.getLiteral<YearMonthIntervalLit>());
-      }
-      break;
-    }
-    case kDatetimeInterval: {
-      if (right_type.getTypeID() == kDatetimeInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetimeInterval);
-        }
-
-        return TypedValue(left.getLiteral<DatetimeIntervalLit>() - right.getLiteral<DatetimeIntervalLit>());
-      }
-      break;
-    }
-    case kYearMonthInterval: {
-      if (right_type.getTypeID() == kYearMonthInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kYearMonthInterval);
-        }
-
-        return TypedValue(left.getLiteral<YearMonthIntervalLit>() - right.getLiteral<YearMonthIntervalLit>());
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-             << left_type.getName() << " and " << right_type.getName();
-}
-
-UncheckedBinaryOperator* SubtractBinaryOperation::makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                                                      const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      if (right.getSuperTypeID() == Type::kNumeric) {
-        return makeNumericBinaryOperatorOuterHelper<SubtractArithmeticUncheckedBinaryOperator>(left, right);
-      }
-      break;
-    }
-    case kDate: {
-      if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<
-            SubtractArithmeticUncheckedBinaryOperator,
-            DateType,
-            DateLit,
-            YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kDatetime: {
-      if (right.getTypeID() == kDatetime) {
-        // NOTE(zuyu): The result type of the Subtract between two Datetimes is DatetimeInterval,
-        // instead of YearMonthInterval.
-        return makeDateBinaryOperatorOuterHelper<SubtractArithmeticUncheckedBinaryOperator,
-                                                 DatetimeIntervalType,
-                                                 DatetimeLit, DatetimeLit>(left, right);
-      } else if (right.getTypeID() == kDatetimeInterval) {
-        return makeDateBinaryOperatorOuterHelper<SubtractArithmeticUncheckedBinaryOperator,
-                                                 DatetimeType,
-                                                 DatetimeLit, DatetimeIntervalLit>(left, right);
-      } else if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<SubtractArithmeticUncheckedBinaryOperator,
-                                                 DatetimeType,
-                                                 DatetimeLit, YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kDatetimeInterval: {
-      if (right.getTypeID() == kDatetimeInterval) {
-        return makeDateBinaryOperatorOuterHelper<SubtractArithmeticUncheckedBinaryOperator,
-                                                 DatetimeIntervalType,
-                                                 DatetimeIntervalLit, DatetimeIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kYearMonthInterval: {
-      if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<SubtractArithmeticUncheckedBinaryOperator,
-                                                 YearMonthIntervalType,
-                                                 YearMonthIntervalLit, YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/SubtractBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/SubtractBinaryOperation.hpp b/types/operations/binary_operations/SubtractBinaryOperation.hpp
deleted file mode 100644
index 8e54362..0000000
--- a/types/operations/binary_operations/SubtractBinaryOperation.hpp
+++ /dev/null
@@ -1,93 +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_TYPES_OPERATIONS_BINARY_OPERATIONS_SUBTRACT_BINARY_OPERATION_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_SUBTRACT_BINARY_OPERATION_HPP_
-
-#include <utility>
-
-#include "types/TypedValue.hpp"
-#include "types/operations/binary_operations/ArithmeticBinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-class Type;
-class UncheckedBinaryOperator;
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief The BinaryOperation for subtraction.
- *
- * @note SubtractBinaryOperation is not commutative: the left argument is the
- *       minuend and the right argument is the subtrahend.
- **/
-class SubtractBinaryOperation : public ArithmeticBinaryOperation {
- public:
-  /**
-   * @brief Get a reference to the singleton instance of this Operation.
-   *
-   * @return A reference to the singleton instance of this Operation.
-   **/
-  static const SubtractBinaryOperation& Instance() {
-    static SubtractBinaryOperation instance;
-    return instance;
-  }
-
-  bool canApplyToTypes(const Type &left,
-                       const Type &right) const override;
-
-  const Type* resultTypeForArgumentTypes(const Type &left,
-                                         const Type &right) const override;
-
-  const Type* resultTypeForPartialArgumentTypes(const Type *left,
-                                                const Type *right) const override;
-
-  bool partialTypeSignatureIsPlausible(const Type *result_type,
-                                       const Type *left_argument_type,
-                                       const Type *right_argument_type) const override;
-
-  std::pair<const Type*, const Type*> pushDownTypeHint(
-      const Type *result_type_hint) const override;
-
-  TypedValue applyToChecked(const TypedValue &left,
-                            const Type &left_type,
-                            const TypedValue &right,
-                            const Type &right_type) const override;
-
-  UncheckedBinaryOperator *makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                               const Type &right) const override;
-
- private:
-  SubtractBinaryOperation()
-      : ArithmeticBinaryOperation(BinaryOperationID::kSubtract) {
-  }
-
-  DISALLOW_COPY_AND_ASSIGN(SubtractBinaryOperation);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_SUBTRACT_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/comparisons/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/CMakeLists.txt b/types/operations/comparisons/CMakeLists.txt
index 321c0f6..933f4fa 100644
--- a/types/operations/comparisons/CMakeLists.txt
+++ b/types/operations/comparisons/CMakeLists.txt
@@ -105,11 +105,13 @@ target_link_libraries(quickstep_types_operations_comparisons_ComparisonID
 target_link_libraries(quickstep_types_operations_comparisons_ComparisonUtil
                       glog
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_types_CharType
                       quickstep_types_DatetimeLit
                       quickstep_types_IntervalLit
                       quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
+                      quickstep_types_VarCharType
                       quickstep_types_containers_Tuple
                       quickstep_types_operations_comparisons_AsciiStringComparators
                       quickstep_types_operations_comparisons_Comparison
@@ -179,8 +181,8 @@ target_link_libraries(quickstep_types_operations_comparisons_PatternMatchingComp
                       quickstep_types_operations_comparisons_PatternMatchingComparators
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonID
-                      quickstep_utility_TemplateUtil
-                      quickstep_utility_Macros)
+                      quickstep_utility_Macros
+                      quickstep_utility_meta_Dispatchers)
 
 # Module all-in-one library:
 add_library(quickstep_types_operations_comparisons ../../../empty_src.cpp)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/comparisons/Comparison.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/Comparison.hpp b/types/operations/comparisons/Comparison.hpp
index c300e74..33c853d 100644
--- a/types/operations/comparisons/Comparison.hpp
+++ b/types/operations/comparisons/Comparison.hpp
@@ -606,11 +606,7 @@ class Comparison : public Operation {
 
  protected:
   explicit Comparison(const ComparisonID comparison_id)
-      : Operation(Operation::kComparison,
-                  kComparisonNames[
-                      static_cast<typename std::underlying_type<ComparisonID>::type>(comparison_id)],
-                  kComparisonShortNames[
-                      static_cast<typename std::underlying_type<ComparisonID>::type>(comparison_id)]),
+      : Operation(Operation::kComparison),
         comparison_id_(comparison_id) {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/comparisons/ComparisonUtil.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/ComparisonUtil.hpp b/types/operations/comparisons/ComparisonUtil.hpp
index 5d868fc..425566d 100644
--- a/types/operations/comparisons/ComparisonUtil.hpp
+++ b/types/operations/comparisons/ComparisonUtil.hpp
@@ -28,11 +28,13 @@
 #include <type_traits>
 
 #include "catalog/CatalogTypedefs.hpp"
+#include "types/CharType.hpp"
 #include "types/DatetimeLit.hpp"
 #include "types/IntervalLit.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
+#include "types/VarCharType.hpp"
 #include "types/containers/Tuple.hpp"
 #include "types/operations/comparisons/AsciiStringComparators.hpp"
 #include "types/operations/comparisons/AsciiStringComparators-inl.hpp"
@@ -153,7 +155,7 @@ auto InvokeOnLessComparatorForTypeIgnoreNullability(const Type &type,
     }
     case kChar: {
       const std::size_t string_length
-          = static_cast<const AsciiStringSuperType&>(type).getStringLength();
+          = static_cast<const CharType&>(type).getStringLength();
       LessAsciiStringUncheckedComparator<false, false, false,
                                          false, false, false>
           comp(string_length, string_length);
@@ -340,11 +342,11 @@ auto InvokeOnLessComparatorForDifferentTypesIgnoreNullability(
     }
     case kChar: {
       const std::size_t left_string_length
-          = static_cast<const AsciiStringSuperType&>(left_type).getStringLength();
+          = static_cast<const CharType&>(left_type).getStringLength();
       switch (right_type.getTypeID()) {
         case kChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const CharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, false, false,
                                                false, false, true>
@@ -364,7 +366,7 @@ auto InvokeOnLessComparatorForDifferentTypesIgnoreNullability(
         }
         case kVarChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const VarCharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, false, false,
                                                false, true, true>
@@ -389,11 +391,11 @@ auto InvokeOnLessComparatorForDifferentTypesIgnoreNullability(
     }
     case kVarChar: {
       const std::size_t left_string_length
-          = static_cast<const AsciiStringSuperType&>(left_type).getStringLength();
+          = static_cast<const VarCharType&>(left_type).getStringLength();
       switch (right_type.getTypeID()) {
         case kChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const CharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, true, false,
                                                false, false, true>
@@ -413,7 +415,7 @@ auto InvokeOnLessComparatorForDifferentTypesIgnoreNullability(
         }
         case kVarChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const VarCharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, true, false,
                                                false, true, true>
@@ -653,11 +655,11 @@ auto InvokeOnBothLessComparatorsForDifferentTypesIgnoreNullability(
     }
     case kChar: {
       const std::size_t left_string_length
-          = static_cast<const AsciiStringSuperType&>(left_type).getStringLength();
+          = static_cast<const CharType&>(left_type).getStringLength();
       switch (right_type.getTypeID()) {
         case kChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const CharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, false, false,
                                                false, false, true>
@@ -686,7 +688,7 @@ auto InvokeOnBothLessComparatorsForDifferentTypesIgnoreNullability(
         }
         case kVarChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const VarCharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, false, false,
                                                false, true, true>
@@ -720,11 +722,11 @@ auto InvokeOnBothLessComparatorsForDifferentTypesIgnoreNullability(
     }
     case kVarChar: {
       const std::size_t left_string_length
-          = static_cast<const AsciiStringSuperType&>(left_type).getStringLength();
+          = static_cast<const VarCharType&>(left_type).getStringLength();
       switch (right_type.getTypeID()) {
         case kChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const CharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, true, false,
                                                false, false, true>
@@ -753,7 +755,7 @@ auto InvokeOnBothLessComparatorsForDifferentTypesIgnoreNullability(
         }
         case kVarChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const VarCharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, true, false,
                                                false, true, true>
@@ -991,7 +993,7 @@ inline bool CheckUntypedValuesEqual(const Type &type, const void *left, const vo
     case kDouble:
       return STLLiteralEqual<double>()(left, right);
     case kChar:
-      return STLCharEqual(static_cast<const AsciiStringSuperType&>(type).getStringLength())(left, right);
+      return STLCharEqual(static_cast<const CharType&>(type).getStringLength())(left, right);
     case kVarChar:
       return STLVarCharEqual()(left, right);
     case kDate:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/comparisons/PatternMatchingComparison.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/PatternMatchingComparison.cpp b/types/operations/comparisons/PatternMatchingComparison.cpp
index 4207f0f..c94ba52 100644
--- a/types/operations/comparisons/PatternMatchingComparison.cpp
+++ b/types/operations/comparisons/PatternMatchingComparison.cpp
@@ -29,7 +29,7 @@
 #include "types/operations/comparisons/ComparisonID.hpp"
 #include "types/operations/comparisons/PatternMatchingComparators.hpp"
 #include "types/operations/comparisons/PatternMatchingComparators-inl.hpp"
-#include "utility/TemplateUtil.hpp"
+#include "utility/meta/Dispatchers.hpp"
 
 #include "glog/logging.h"
 
@@ -121,11 +121,19 @@ UncheckedComparator* PatternMatchingComparison::makeUncheckedComparatorForTypes(
                  << " in PatternMatchinComparison::makeUncheckedComparatorForTypes()";
   }
 
-  return CreateBoolInstantiatedInstance<PatternMatchingUncheckedComparator, UncheckedComparator>(
-      std::forward_as_tuple(left_max_length, right_max_length),
+  return meta::InvokeOnBools(
       is_like_pattern, is_negation,
-      left.isNullable(), right.isNullable());
+      left.isNullable(), right.isNullable(),
+      [&](auto is_like_pattern,  // NOLINT(build/c++11)
+          auto is_negation,
+          auto is_left_nullable,
+          auto is_right_nullable) -> UncheckedComparator* {
+    return new PatternMatchingUncheckedComparator<
+        decltype(is_like_pattern)::value,
+        decltype(is_negation)::value,
+        decltype(is_left_nullable)::value,
+        decltype(is_right_nullable)::value>(left_max_length, right_max_length);
+  });
 }
 
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/ArithmeticUnaryOperations.cpp b/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
deleted file mode 100644
index c10d5cf..0000000
--- a/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
+++ /dev/null
@@ -1,145 +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 "types/operations/unary_operations/ArithmeticUnaryOperations.hpp"
-
-#include <string>
-
-#include "types/DatetimeIntervalType.hpp"
-#include "types/DoubleType.hpp"
-#include "types/FloatType.hpp"
-#include "types/IntType.hpp"
-#include "types/LongType.hpp"
-#include "types/Type.hpp"
-#include "types/TypeErrors.hpp"
-#include "types/TypeID.hpp"
-#include "types/TypedValue.hpp"
-#include "types/YearMonthIntervalType.hpp"
-#include "types/operations/unary_operations/ArithmeticUnaryOperators.hpp"
-#include "utility/EqualsAnyConstant.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-bool ArithmeticUnaryOperation::canApplyToType(const Type &type) const {
-  return QUICKSTEP_EQUALS_ANY_CONSTANT(
-      type.getTypeID(),
-      kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-}
-
-const Type* ArithmeticUnaryOperation::resultTypeForArgumentType(const Type &type) const {
-  if (canApplyToType(type)) {
-    return &type;
-  } else {
-    return nullptr;
-  }
-}
-
-const Type* ArithmeticUnaryOperation::pushDownTypeHint(const Type *type_hint) const {
-  if (type_hint == nullptr) {
-    return nullptr;
-  }
-
-  if (canApplyToType(*type_hint)) {
-    return type_hint;
-  } else {
-    return nullptr;
-  }
-}
-
-bool NegateUnaryOperation::resultTypeIsPlausible(const Type &result_type) const {
-  return QUICKSTEP_EQUALS_ANY_CONSTANT(
-      result_type.getTypeID(),
-      kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-}
-
-TypedValue NegateUnaryOperation::applyToChecked(const TypedValue &argument,
-                                                const Type &argument_type) const {
-  DCHECK_EQ(argument.getTypeID(), argument_type.getTypeID());
-
-  if (argument.isNull()) {
-    return argument;
-  }
-
-  switch (argument.getTypeID()) {
-    case kInt:
-      return TypedValue(-argument.getLiteral<typename IntType::cpptype>());
-    case kLong:
-      return TypedValue(-argument.getLiteral<typename LongType::cpptype>());
-    case kFloat:
-      return TypedValue(-argument.getLiteral<typename FloatType::cpptype>());
-    case kDouble:
-      return TypedValue(-argument.getLiteral<typename DoubleType::cpptype>());
-    case kDatetimeInterval:
-      return TypedValue(-argument.getLiteral<typename DatetimeIntervalType::cpptype>());
-    case kYearMonthInterval:
-      return TypedValue(-argument.getLiteral<typename YearMonthIntervalType::cpptype>());
-    default: {
-      LOG(FATAL) << "Can not apply UnaryOperation " << getName()
-                 << " to argument of type " << argument_type.getName();
-    }
-  }
-}
-
-UncheckedUnaryOperator* NegateUnaryOperation::makeUncheckedUnaryOperatorForType(const Type &type) const {
-  switch (type.getTypeID()) {
-    case kInt:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<IntType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<IntType, false>();
-      }
-    case kLong:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<LongType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<LongType, false>();
-      }
-    case kFloat:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<FloatType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<FloatType, false>();
-      }
-    case kDouble:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<DoubleType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<DoubleType, false>();
-      }
-    case kDatetimeInterval:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<DatetimeIntervalType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<DatetimeIntervalType, false>();
-      }
-    case kYearMonthInterval:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<YearMonthIntervalType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<YearMonthIntervalType, false>();
-      }
-    default:
-      throw OperationInapplicableToType(getName(), 1, type.getName().c_str());
-  }
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/ArithmeticUnaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/ArithmeticUnaryOperations.hpp b/types/operations/unary_operations/ArithmeticUnaryOperations.hpp
index 5eed073..4c212c0 100644
--- a/types/operations/unary_operations/ArithmeticUnaryOperations.hpp
+++ b/types/operations/unary_operations/ArithmeticUnaryOperations.hpp
@@ -20,73 +20,60 @@
 #ifndef QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATIONS_HPP_
 #define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATIONS_HPP_
 
-#include "types/TypedValue.hpp"
-#include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
-#include "utility/Macros.hpp"
+#include <string>
 
-namespace quickstep {
+#include "types/DatetimeIntervalType.hpp"
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/YearMonthIntervalType.hpp"
+#include "types/operations/OperationUtil.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
 
-class Type;
+namespace quickstep {
 
 /** \addtogroup Types
  *  @{
  */
 
-/**
- * @brief A UnaryOperation which applies to and yields numeric values.
- **/
-class ArithmeticUnaryOperation : public UnaryOperation {
- public:
-  bool canApplyToType(const Type &type) const override;
-
-  const Type* resultTypeForArgumentType(const Type &type) const override;
-
-  const Type* pushDownTypeHint(const Type *type_hint) const override;
-
- protected:
-  explicit ArithmeticUnaryOperation(const UnaryOperationID operation_id)
-      : UnaryOperation(operation_id) {
+template <typename ArgumentT, typename ResultT>
+struct NegateFunctor : public UnaryFunctor<ArgumentT, ResultT> {
+  inline typename ResultT::cpptype apply(
+      const typename ArgumentT::cpptype &argument) const {
+    return -argument;
   }
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(ArithmeticUnaryOperation);
-};
-
-/**
- * @brief The UnaryOperation for negation.
- **/
-class NegateUnaryOperation : public ArithmeticUnaryOperation {
- public:
-  /**
-   * @brief Get a reference to the singleton instance of this Operation.
-   *
-   * @return A reference to the singleton instance of this Operation.
-   **/
-  static const NegateUnaryOperation& Instance() {
-    static NegateUnaryOperation instance;
-    return instance;
+  inline static std::string GetName() {
+    return "-";
   }
+};
 
-  const Type* fixedNullableResultType() const override {
-    return nullptr;
+template <typename ArgumentT>
+struct SgnFunctor : public UnaryFunctor<ArgumentT, IntType> {
+  inline int apply(const typename ArgumentT::cpptype &argument) const {
+    return (argument > 0) - (argument < 0);
   }
-
-  bool resultTypeIsPlausible(const Type &result_type) const override;
-
-  TypedValue applyToChecked(const TypedValue &argument,
-                            const Type &argument_type) const override;
-
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorForType(const Type &type) const override;
-
- private:
-  NegateUnaryOperation()
-      : ArithmeticUnaryOperation(UnaryOperationID::kNegate) {
+  inline static std::string GetName() {
+    return "Sgn";
   }
-
-  DISALLOW_COPY_AND_ASSIGN(NegateUnaryOperation);
 };
 
+using ArithmeticUnaryFunctorPack = FunctorPack<
+// negate
+  NegateFunctor<IntType, IntType>,
+  NegateFunctor<LongType, LongType>,
+  NegateFunctor<FloatType, FloatType>,
+  NegateFunctor<DoubleType, DoubleType>,
+  NegateFunctor<DatetimeIntervalType, DatetimeIntervalType>,
+  NegateFunctor<YearMonthIntervalType, YearMonthIntervalType>,
+
+// sgn (Sign of a numeric value)
+  SgnFunctor<IntType>,
+  SgnFunctor<LongType>,
+  SgnFunctor<FloatType>,
+  SgnFunctor<DoubleType>
+>;
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/ArithmeticUnaryOperators.hpp b/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
deleted file mode 100644
index bf3f7b6..0000000
--- a/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
+++ /dev/null
@@ -1,169 +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_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATORS_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATORS_HPP_
-
-#include <cstddef>
-#include <utility>
-#include <vector>
-
-#include "catalog/CatalogTypedefs.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
-#include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief The UncheckedUnaryOperator for negation.
- **/
-template <class ResultType, bool argument_nullable>
-class NegateUncheckedUnaryOperator : public UncheckedUnaryOperator {
- public:
-  NegateUncheckedUnaryOperator() : UncheckedUnaryOperator() {
-  }
-
-  inline TypedValue applyToTypedValue(const TypedValue &argument) const override {
-    return applyToTypedValueInl(argument);
-  }
-
-  inline TypedValue applyToTypedValueInl(const TypedValue &argument) const {
-    if (argument_nullable && argument.isNull()) {
-      return argument;
-    }
-    return TypedValue(-argument.getLiteral<typename ResultType::cpptype>());
-  }
-
-  inline TypedValue applyToDataPtr(const void *argument) const override {
-    return applyToDataPtrInl(argument);
-  }
-
-  inline TypedValue applyToDataPtrInl(const void *argument) const {
-    if (argument_nullable && (argument == nullptr)) {
-      return TypedValue(ResultType::kStaticTypeID);
-    }
-    return TypedValue(-*static_cast<const typename ResultType::cpptype*>(argument));
-  }
-
-  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override {
-    DCHECK(NativeColumnVector::UsableForType(ResultType::Instance(argument_nullable)));
-    // All arithmetic types (numbers, datetime, and intervals) are usable with
-    // NativeColumnVector, so 'argument' should always be native.
-    DCHECK(argument.isNative());
-    const NativeColumnVector &native_argument = static_cast<const NativeColumnVector&>(argument);
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(argument_nullable),
-        native_argument.size());
-    for (std::size_t pos = 0;
-         pos < native_argument.size();
-         ++pos) {
-      const typename ResultType::cpptype *scalar_arg
-          = static_cast<const typename ResultType::cpptype*>(
-              native_argument.getUntypedValue<argument_nullable>(pos));
-      if (argument_nullable && (scalar_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-            = -(*scalar_arg);
-      }
-    }
-    return result;
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
-                                     const attribute_id argument_attr_id) const override {
-    DCHECK(NativeColumnVector::UsableForType(ResultType::Instance(argument_nullable)));
-    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        accessor,
-        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-      NativeColumnVector *result = new NativeColumnVector(
-          ResultType::Instance(argument_nullable),
-          accessor->getNumTuples());
-      accessor->beginIteration();
-      while (accessor->next()) {
-        const typename ResultType::cpptype *scalar_arg
-            = static_cast<const typename ResultType::cpptype*>(
-                accessor->template getUntypedValue<argument_nullable>(argument_attr_id));
-        if (argument_nullable && (scalar_arg == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-              = -(*scalar_arg);
-        }
-      }
-      return result;
-    });
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  ColumnVector* applyToValueAccessorForJoin(
-      ValueAccessor *accessor,
-      const bool use_left_relation,
-      const attribute_id argument_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    DCHECK(NativeColumnVector::UsableForType(ResultType::Instance(argument_nullable)));
-    NativeColumnVector *result = new NativeColumnVector(ResultType::Instance(argument_nullable),
-                                                        joined_tuple_ids.size());
-    InvokeOnValueAccessorNotAdapter(
-        accessor,
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-      for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-        const typename ResultType::cpptype *scalar_arg
-            = static_cast<const typename ResultType::cpptype*>(
-                accessor->template getUntypedValueAtAbsolutePosition<argument_nullable>(
-                    argument_attr_id,
-                    use_left_relation ? joined_pair.first : joined_pair.second));
-        if (argument_nullable && (scalar_arg == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-              = -(*scalar_arg);
-        }
-      }
-    });
-    return result;
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(NegateUncheckedUnaryOperator);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/AsciiStringUnaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/AsciiStringUnaryOperations.hpp b/types/operations/unary_operations/AsciiStringUnaryOperations.hpp
new file mode 100644
index 0000000..1ee1867
--- /dev/null
+++ b/types/operations/unary_operations/AsciiStringUnaryOperations.hpp
@@ -0,0 +1,122 @@
+/**
+ * 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_TYPES_OPERATIONS_UNARY_OPERATIONS_ASCII_STRING_UNARY_OPERATIONS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ASCII_STRING_UNARY_OPERATIONS_HPP_
+
+#include <cctype>
+#include <cstring>
+#include <string>
+
+#include "types/CharType.hpp"
+#include "types/IntType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/VarCharType.hpp"
+#include "types/operations/OperationUtil.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
+#include "types/port/strnlen.hpp"
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename ArgumentT>
+struct AsciiStringLengthFunctor : public UnaryFunctor<ArgumentT, IntType> {
+  explicit AsciiStringLengthFunctor(const ArgumentT &argument_type)
+      : max_string_length_(argument_type.getStringLength()) {}
+  inline int apply(const void *argument) const {
+    return strnlen(static_cast<const char*>(argument), max_string_length_);
+  }
+  inline int apply(const TypedValue &argument) const {
+    DCHECK(argument.getTypeID() == kVarChar);
+    return std::strlen(static_cast<const char*>(argument.getOutOfLineData()));
+  }
+  inline static std::string GetName() {
+    return "length";
+  }
+  const std::size_t max_string_length_;
+};
+
+template <typename ArgumentT, int transform(int), typename FunctorNameT>
+struct AsciiStringTranformFunctor : public UnaryFunctor<ArgumentT, ArgumentT> {
+  explicit AsciiStringTranformFunctor(const ArgumentT &argument_type)
+      : max_string_length_(argument_type.getStringLength()) {}
+  inline void apply(const void *argument, void *result) const {
+    DCHECK(ArgumentT::kStaticTypeID == kChar);
+    const char *argument_str = static_cast<const char*>(argument);
+    char *result_str = static_cast<char*>(result);
+    for (std::size_t i = 0; i < max_string_length_; ++i) {
+      if ((result_str[i] = transform(argument_str[i])) == 0) {
+        break;
+      }
+    }
+  }
+  inline TypedValue apply(const TypedValue &argument) const {
+    DCHECK(argument.getTypeID() == kVarChar);
+    const char *argument_str = static_cast<const char*>(argument.getOutOfLineData());
+    const std::size_t length = argument.getDataSize();
+    char *buf = static_cast<char*>(std::malloc(length));
+
+    for (std::size_t i = 0; i < length; ++i) {
+      buf[i] = transform(argument_str[i]);
+    }
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, length);
+  }
+  inline static std::string GetName() {
+    return FunctorNameT::ToString();
+  }
+  inline static const Type* GetResultType(const Type &argument_type) {
+    DCHECK(argument_type.getTypeID() == ArgumentT::kStaticTypeID);
+    return &argument_type;
+  }
+  const std::size_t max_string_length_;
+};
+
+template <typename ArgumentT>
+using AsciiStringToLowerCaseFunctor =
+    AsciiStringTranformFunctor<ArgumentT, std::tolower,
+                               meta::StringLiteral<'t', 'o', 'l', 'o', 'w', 'e', 'r'>>;
+
+template <typename ArgumentT>
+using AsciiStringToUpperCaseFunctor =
+    AsciiStringTranformFunctor<ArgumentT, std::toupper,
+                               meta::StringLiteral<'t', 'o', 'u', 'p', 'p', 'e', 'r'>>;
+
+using AsciiStringUnaryFunctorPack = FunctorPack<
+// length
+    AsciiStringLengthFunctor<CharType>,
+    AsciiStringLengthFunctor<VarCharType>,
+// tolower
+    AsciiStringToLowerCaseFunctor<CharType>,
+    AsciiStringToLowerCaseFunctor<VarCharType>,
+// toupper
+    AsciiStringToUpperCaseFunctor<CharType>,
+    AsciiStringToUpperCaseFunctor<VarCharType>
+>;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ASCII_STRING_UNARY_OPERATIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CMakeLists.txt b/types/operations/unary_operations/CMakeLists.txt
index 6e1923a..bcd756e 100644
--- a/types/operations/unary_operations/CMakeLists.txt
+++ b/types/operations/unary_operations/CMakeLists.txt
@@ -16,14 +16,26 @@
 # under the License.
 
 # Declare micro-libs:
-add_library(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations ArithmeticUnaryOperations.cpp ArithmeticUnaryOperations.hpp)
-add_library(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperators ../../../empty_src.cpp ArithmeticUnaryOperators.hpp)
-add_library(quickstep_types_operations_unaryoperations_DateExtractOperation DateExtractOperation.cpp DateExtractOperation.hpp)
-add_library(quickstep_types_operations_unaryoperations_NumericCastOperation ../../../empty_src.cpp NumericCastOperation.hpp)
-add_library(quickstep_types_operations_unaryoperations_SubstringOperation SubstringOperation.cpp SubstringOperation.hpp)
+add_library(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
+            ../../../empty_src.cpp
+            ArithmeticUnaryOperations.hpp)
+add_library(quickstep_types_operations_unaryoperations_AsciiStringUnaryOperations
+            ../../../empty_src.cpp
+            AsciiStringUnaryOperations.hpp)
+add_library(quickstep_types_operations_unaryoperations_CMathUnaryOperations
+            ../../../empty_src.cpp
+            CMathUnaryOperations.hpp)
+add_library(quickstep_types_operations_unaryoperations_CastOperation CastOperation.cpp CastOperation.hpp)
+add_library(quickstep_types_operations_unaryoperations_DateExtractOperation
+            DateExtractOperation.cpp
+            DateExtractOperation.hpp)
+add_library(quickstep_types_operations_unaryoperations_SubstringOperation
+            SubstringOperation.cpp
+            SubstringOperation.hpp)
 add_library(quickstep_types_operations_unaryoperations_UnaryOperation UnaryOperation.cpp UnaryOperation.hpp)
-add_library(quickstep_types_operations_unaryoperations_UnaryOperationFactory UnaryOperationFactory.cpp UnaryOperationFactory.hpp)
-add_library(quickstep_types_operations_unaryoperations_UnaryOperationID UnaryOperationID.cpp UnaryOperationID.hpp)
+add_library(quickstep_types_operations_unaryoperations_UnaryOperationWrapper
+            ../../../empty_src.cpp
+            UnaryOperationWrapper.hpp)
 
 # Link dependencies:
 target_link_libraries(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
@@ -33,109 +45,110 @@ target_link_libraries(quickstep_types_operations_unaryoperations_ArithmeticUnary
                       quickstep_types_FloatType
                       quickstep_types_IntType
                       quickstep_types_LongType
+                      quickstep_types_YearMonthIntervalType
+                      quickstep_types_operations_OperationUtil
+                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper)
+target_link_libraries(quickstep_types_operations_unaryoperations_AsciiStringUnaryOperations
+                      quickstep_types_CharType
+                      quickstep_types_IntType
                       quickstep_types_Type
-                      quickstep_types_TypeErrors
+                      quickstep_types_TypeFactory
                       quickstep_types_TypeID
-                      quickstep_types_TypedValue
-                      quickstep_types_YearMonthIntervalType
-                      quickstep_types_operations_unaryoperations_ArithmeticUnaryOperators
-                      quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
-                      quickstep_utility_EqualsAnyConstant
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperators
-                      glog
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_ValueAccessor
-                      quickstep_storage_ValueAccessorUtil
-                      quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
-                      quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_unaryoperations_DateExtractOperation
+                      quickstep_types_VarCharType
+                      quickstep_types_operations_OperationUtil
+                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper
+                      quickstep_types_port_strnlen
+                      quickstep_utility_meta_Common)
+target_link_libraries(quickstep_types_operations_unaryoperations_CMathUnaryOperations
+                      quickstep_types_DoubleType
+                      quickstep_types_FloatType
+                      quickstep_types_IntType
+                      quickstep_types_LongType
+                      quickstep_types_operations_OperationUtil
+                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper
+                      quickstep_utility_meta_Common)
+target_link_libraries(quickstep_types_operations_unaryoperations_CastOperation
                       glog
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_ValueAccessor
-                      quickstep_storage_ValueAccessorUtil
-                      quickstep_types_DatetimeLit
+                      quickstep_types_CharType
+                      quickstep_types_DoubleType
+                      quickstep_types_FloatType
                       quickstep_types_IntType
                       quickstep_types_LongType
                       quickstep_types_Type
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
+                      quickstep_types_TypeUtil
                       quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
-                      quickstep_types_operations_Operation_proto
+                      quickstep_types_VarCharType
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_unaryoperations_NumericCastOperation
+                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper
+                      quickstep_types_port_strnlen
+                      quickstep_utility_EqualsAnyConstant
+                      quickstep_utility_Macros
+                      quickstep_utility_StringUtil)
+target_link_libraries(quickstep_types_operations_unaryoperations_DateExtractOperation
                       glog
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_ValueAccessor
-                      quickstep_storage_ValueAccessorUtil
-                      quickstep_types_DoubleType
-                      quickstep_types_FloatType
+                      quickstep_types_DateType
+                      quickstep_types_DatetimeLit
+                      quickstep_types_DatetimeType
                       quickstep_types_IntType
                       quickstep_types_LongType
                       quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
-                      quickstep_types_operations_Operation_proto
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
+                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper
                       quickstep_utility_Macros
-                      quickstep_utility_PtrMap)
+                      quickstep_utility_StringUtil)
 target_link_libraries(quickstep_types_operations_unaryoperations_SubstringOperation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_ValueAccessor
                       quickstep_storage_ValueAccessorUtil
+                      quickstep_types_CharType
                       quickstep_types_Type
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
+                      quickstep_types_VarCharType
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorUtil
-                      quickstep_types_operations_Operation_proto
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_types_port_strnlen
-                      quickstep_utility_HashPair
                       quickstep_utility_Macros
-                      quickstep_utility_TemplateUtil)
+                      quickstep_utility_meta_Dispatchers)
 target_link_libraries(quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_StorageBlockInfo
                       quickstep_types_TypedValue
                       quickstep_types_operations_Operation
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_Operation_proto
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_unaryoperations_UnaryOperationFactory
+target_link_libraries(quickstep_types_operations_unaryoperations_UnaryOperationWrapper
                       glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_types_Type
                       quickstep_types_TypeFactory
-                      quickstep_types_operations_Operation_proto
-                      quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
-                      quickstep_types_operations_unaryoperations_DateExtractOperation
-                      quickstep_types_operations_unaryoperations_NumericCastOperation
-                      quickstep_types_operations_unaryoperations_SubstringOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
+                      quickstep_types_TypeID
+                      quickstep_types_TypedValue
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_operations_OperationSignature
+                      quickstep_types_operations_OperationUtil
+                      quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_utility_Macros)
 
 # Module all-in-one library:
 add_library(quickstep_types_operations_unaryoperations ../../../empty_src.cpp)
 target_link_libraries(quickstep_types_operations_unaryoperations
                       quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
-                      quickstep_types_operations_unaryoperations_ArithmeticUnaryOperators
+                      quickstep_types_operations_unaryoperations_AsciiStringUnaryOperations
+                      quickstep_types_operations_unaryoperations_CMathUnaryOperations
+                      quickstep_types_operations_unaryoperations_CastOperation
                       quickstep_types_operations_unaryoperations_DateExtractOperation
-                      quickstep_types_operations_unaryoperations_NumericCastOperation
                       quickstep_types_operations_unaryoperations_SubstringOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationFactory
-                      quickstep_types_operations_unaryoperations_UnaryOperationID)
+                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper)
 
 # Tests:
 
@@ -160,11 +173,9 @@ target_link_libraries(UnaryOperation_tests
                       quickstep_types_containers_ColumnVector
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
+                      quickstep_types_operations_unaryoperations_CastOperation
                       quickstep_types_operations_unaryoperations_DateExtractOperation
-                      quickstep_types_operations_unaryoperations_NumericCastOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationFactory
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros)
 add_test(UnaryOperation_tests UnaryOperation_tests)



[18/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/utility/meta/TypeListMetaFunctions.hpp
----------------------------------------------------------------------
diff --git a/utility/meta/TypeListMetaFunctions.hpp b/utility/meta/TypeListMetaFunctions.hpp
new file mode 100644
index 0000000..d908493
--- /dev/null
+++ b/utility/meta/TypeListMetaFunctions.hpp
@@ -0,0 +1,245 @@
+/**
+ * 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_META_TYPE_LIST_META_FUNCTIONS_HPP_
+#define QUICKSTEP_UTILITY_META_TYPE_LIST_META_FUNCTIONS_HPP_
+
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+namespace meta {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+template <typename ...Ts>
+class TypeList;
+
+namespace internal {
+
+template <typename TL, typename PosTL, typename Enable = void>
+struct ElementAtImpl;
+
+template <typename TL, typename PosTL>
+struct ElementAtImpl<TL, PosTL,
+                     std::enable_if_t<PosTL::length == 0>> {
+  using type = TL;
+};
+
+template <typename TL, typename PosTL>
+struct ElementAtImpl<TL, PosTL,
+                     std::enable_if_t<PosTL::length != 0>>
+    : ElementAtImpl<typename std::tuple_element<
+                        PosTL::head::value,
+                        typename TL::template bind_to<std::tuple>>::type,
+                    typename PosTL::tail> {};
+
+
+template <typename Out, typename Rest, typename Enable = void>
+struct UniqueImpl;
+
+template <typename Out, typename Rest>
+struct UniqueImpl<Out, Rest,
+                  std::enable_if_t<Rest::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename Rest>
+struct UniqueImpl<Out, Rest,
+                  std::enable_if_t<Out::template contains<typename Rest::head>::value>>
+    : UniqueImpl<Out, typename Rest::tail> {};
+
+template <typename Out, typename Rest>
+struct UniqueImpl<Out, Rest,
+                  std::enable_if_t<!Out::template contains<typename Rest::head>::value>>
+    : UniqueImpl<typename Out::template push_back<typename Rest::head>,
+                 typename Rest::tail> {};
+
+
+template <typename Out, typename Rest, typename Subtrahend, typename Enable = void>
+struct SubtractImpl;
+
+template <typename Out, typename Rest, typename Subtrahend>
+struct SubtractImpl<Out, Rest, Subtrahend,
+                    std::enable_if_t<Rest::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename Rest, typename Subtrahend>
+struct SubtractImpl<Out, Rest, Subtrahend,
+                    std::enable_if_t<Subtrahend::template contains<
+                        typename Rest::head>::value>>
+    : SubtractImpl<Out, typename Rest::tail, Subtrahend> {};
+
+template <typename Out, typename Rest, typename Subtrahend>
+struct SubtractImpl<Out, Rest, Subtrahend,
+                    std::enable_if_t<!Subtrahend::template contains<
+                        typename Rest::head>::value>>
+    : SubtractImpl<typename Out::template push_back<typename Rest::head>,
+                   typename Rest::tail, Subtrahend> {};
+
+
+template <typename LeftTL, typename RightTL>
+struct CartesianProductImpl {
+  template <typename LeftT>
+  struct LeftHelper {
+    template <typename RightT>
+    struct RightHelper {
+      using type = TypeList<LeftT, RightT>;
+    };
+    using type = typename RightTL::template map<RightHelper>;
+  };
+  using type = typename LeftTL::template flatmap<LeftHelper>;
+};
+
+
+template <typename Out, typename Rest, template <typename ...> class Op,
+          typename Enable = void>
+struct FlatmapImpl;
+
+template <typename Out, typename Rest, template <typename ...> class Op>
+struct FlatmapImpl<Out, Rest, Op,
+                   std::enable_if_t<Rest::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename Rest, template <typename ...> class Op>
+struct FlatmapImpl<Out, Rest, Op,
+                   std::enable_if_t<Rest::length != 0>>
+    : FlatmapImpl<typename Out::template append<typename Op<typename Rest::head>::type>,
+                  typename Rest::tail, Op> {};
+
+
+template <typename Out, typename Rest, template <typename ...> class Op,
+          typename Enable = void>
+struct FilterImpl;
+
+template <typename Out, typename Rest, template <typename ...> class Op>
+struct FilterImpl<Out, Rest, Op,
+                  std::enable_if_t<Rest::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename Rest, template <typename ...> class Op>
+struct FilterImpl<Out, Rest, Op,
+                  std::enable_if_t<Op<typename Rest::head>::value>>
+    : FilterImpl<typename Out::template push_back<typename Rest::head>,
+                 typename Rest::tail, Op> {};
+
+template <typename Out, typename Rest, template <typename ...> class Op>
+struct FilterImpl<Out, Rest, Op,
+                  std::enable_if_t<!Op<typename Rest::head>::value>>
+    : FilterImpl<Out, typename Rest::tail, Op> {};
+
+
+template <typename Out, typename Rest, template <typename ...> class Op,
+          typename Enable = void>
+struct FiltermapImpl;
+
+template <typename Out, typename Rest, template <typename ...> class Op>
+struct FiltermapImpl<Out, Rest, Op,
+                     std::enable_if_t<Rest::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename Rest, template <typename ...> class Op>
+struct FiltermapImpl<Out, Rest, Op,
+                     std::enable_if_t<Rest::length != 0 &&
+                                      IsTrait<Op<typename Rest::head>>::value>>
+    : FiltermapImpl<typename Out::template push_back<typename Op<typename Rest::head>::type>,
+                    typename Rest::tail, Op> {};
+
+template <typename Out, typename Rest, template <typename ...> class Op>
+struct FiltermapImpl<Out, Rest, Op,
+                     std::enable_if_t<Rest::length != 0 &&
+                                      !IsTrait<Op<typename Rest::head>>::value>>
+    : FiltermapImpl<Out, typename Rest::tail, Op> {};
+
+
+template <typename Out, typename Rest, typename Enable = void>
+struct FlattenOnceImpl;
+
+template <typename Out, typename Rest>
+struct FlattenOnceImpl<Out, Rest,
+                       std::enable_if_t<Rest::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename Rest>
+struct FlattenOnceImpl<Out, Rest,
+                       std::enable_if_t<Rest::length != 0>>
+    : FlattenOnceImpl<typename Out::template append<typename Rest::head>,
+                      typename Rest::tail> {};
+
+
+template <typename Out, typename RestL, typename RestR, typename Enable = void>
+struct ZipImpl;
+
+template <typename Out, typename RestL, typename RestR>
+struct ZipImpl<Out, RestL, RestR,
+               std::enable_if_t<RestL::length == 0 || RestR::length == 0>> {
+  static_assert(RestL::length == 0 && RestR::length == 0,
+                "Zip failed: TypeLists have unequal lengths");
+  using type = Out;
+};
+
+template <typename Out, typename RestL, typename RestR>
+struct ZipImpl<Out, RestL, RestR,
+               std::enable_if_t<RestL::length != 0 && RestR::length != 0>>
+    : ZipImpl<typename Out::template push_back<
+                  TypeList<typename RestL::head, typename RestR::head>>,
+              typename RestL::tail, typename RestR::tail> {};
+
+
+template <typename Out, typename RestL, typename RestR,
+          template <typename ...> class Op, typename Enable = void>
+struct ZipWithImpl;
+
+template <typename Out, typename RestL, typename RestR,
+          template <typename ...> class Op>
+struct ZipWithImpl<Out, RestL, RestR, Op,
+                   std::enable_if_t<RestL::length == 0 || RestR::length == 0>> {
+  static_assert(RestL::length == 0 && RestR::length == 0,
+                "ZipWith failed: TypeLists have unequal lengths");
+  using type = Out;
+};
+
+template <typename Out, typename RestL, typename RestR,
+          template <typename ...> class Op>
+struct ZipWithImpl<Out, RestL, RestR, Op,
+                   std::enable_if_t<RestL::length != 0 && RestR::length != 0>>
+    : ZipWithImpl<typename Out::template push_back<
+                      typename Op<typename RestL::head, typename RestR::head>::type>,
+                  typename RestL::tail, typename RestR::tail, Op> {};
+
+
+template <typename T, typename ...Ts>
+struct AsSequenceImpl {
+  using type = Sequence<T, static_cast<T>(Ts::value)...>;
+};
+
+}  // namespace internal
+
+/** @} */
+
+}  // namespace meta
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_META_TYPE_LIST_META_FUNCTIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/utility/tests/TemplateUtil_unittest.cpp
----------------------------------------------------------------------
diff --git a/utility/tests/TemplateUtil_unittest.cpp b/utility/tests/TemplateUtil_unittest.cpp
deleted file mode 100644
index ce5d662..0000000
--- a/utility/tests/TemplateUtil_unittest.cpp
+++ /dev/null
@@ -1,115 +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 "utility/TemplateUtil.hpp"
-
-#include <memory>
-#include <string>
-#include <sstream>
-#include <tuple>
-#include <utility>
-
-#include "utility/Macros.hpp"
-
-#include "gtest/gtest.h"
-
-namespace quickstep {
-
-class SomeArgType {
- public:
-  explicit SomeArgType(const std::string &value)
-      : value_(value) {
-  }
-
-  SomeArgType(SomeArgType &&arg)
-      : value_(std::move(arg.value_)) {
-  }
-
-  std::string toString() const {
-    return value_;
-  }
-
- private:
-  const std::string value_;
-
-  DISALLOW_COPY_AND_ASSIGN(SomeArgType);
-};
-
-class BaseClass {
- public:
-  virtual std::string toString() const = 0;
-};
-
-template <bool c1, bool c2, bool c3, bool c4, bool c5, bool c6>
-class SomeClass : public BaseClass {
- public:
-  SomeClass(const int a1, SomeArgType &&a2)
-      : a1_(a1), a2_(std::forward<SomeArgType>(a2)) {
-  }
-
-  std::string toString() const override {
-    std::ostringstream oss;
-    oss << "{ ";
-    if (c1) {
-      oss << "c1 ";
-    }
-    if (c2) {
-      oss << "c2 ";
-    }
-    if (c3) {
-      oss << "c3 ";
-    }
-    if (c4) {
-      oss << "c4 ";
-    }
-    if (c5) {
-      oss << "c5 ";
-    }
-    if (c6) {
-      oss << "c6 ";
-    }
-    oss << "} " << a1_ << " " << a2_.toString();
-    return oss.str();
-  }
-
- private:
-  const int a1_;
-  const SomeArgType a2_;
-};
-
-void RunTest(const bool c1, const bool c2, const bool c3,
-             const bool c4, const bool c5, const bool c6,
-             const std::string &expected) {
-  // arg should be perfectly forwarded.
-  SomeArgType arg("xyz");
-
-  std::unique_ptr<BaseClass> base(
-      CreateBoolInstantiatedInstance<SomeClass, BaseClass>(std::forward_as_tuple(10, std::move(arg)),
-                                                           c1, c2, c3, c4, c5, c6));
-  EXPECT_STREQ(expected.c_str(), base->toString().c_str());
-}
-
-TEST(TemplateUtilTest, TemplateUtilTest) {
-  RunTest(true, false, true, false, true, false, "{ c1 c3 c5 } 10 xyz");
-  RunTest(true, true, true, true, true, true, "{ c1 c2 c3 c4 c5 c6 } 10 xyz");
-  RunTest(false, false, true, true, false, false, "{ c3 c4 } 10 xyz");
-  RunTest(false, false, false, false, false, false, "{ } 10 xyz");
-}
-
-}  // namespace quickstep


[24/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/OperationUtil.hpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationUtil.hpp b/types/operations/OperationUtil.hpp
new file mode 100644
index 0000000..076dc0c
--- /dev/null
+++ b/types/operations/OperationUtil.hpp
@@ -0,0 +1,334 @@
+/**
+ * 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_TYPES_OPERATIONS_OPERATION_UTIL_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_OPERATION_UTIL_HPP_
+
+#include <cstddef>
+#include <list>
+#include <string>
+#include <type_traits>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "types/Type.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename FunctorT, typename ...SpecArgs>
+struct FunctorSpecializer {
+  template <bool specialize = (sizeof...(SpecArgs) != 0),
+            typename EnableT = void>
+  struct Implementation;
+
+  typedef Implementation<> type;
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <bool specialize>
+struct FunctorSpecializer<FunctorT, SpecArgs...>
+    ::Implementation<specialize, std::enable_if_t<specialize>> {
+  template <typename ...FuncArgs>
+  inline static auto Invoke(const FunctorT &functor, FuncArgs &&...args) {
+    return functor.template apply<SpecArgs...>(std::forward<FuncArgs>(args)...);
+  }
+  typedef FunctorT FunctorType;
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <bool specialize>
+struct FunctorSpecializer<FunctorT, SpecArgs...>
+    ::Implementation<specialize, std::enable_if_t<!specialize>> {
+  template <typename ...FuncArgs>
+  inline static auto Invoke(const FunctorT &functor, FuncArgs &&...args) {
+    return functor.apply(std::forward<FuncArgs>(args)...);
+  }
+  typedef FunctorT FunctorType;
+};
+
+template <typename ColumnVectorT>
+struct ColumnVectorValueAccessor {
+  explicit ColumnVectorValueAccessor(const ColumnVectorT &column_vector_in)
+      : column_vector(column_vector_in),
+        length(column_vector.size()) {}
+
+  inline void beginIteration() {
+    pos = static_cast<std::size_t>(-1);
+  }
+
+  inline bool next() {
+    return (++pos) < length;
+  }
+
+  inline std::size_t getNumTuples() const {
+    return length;
+  }
+
+  template <bool nullable>
+  inline const void* getUntypedValue(const attribute_id) const {
+    return column_vector.template getUntypedValue<nullable>(pos);
+  }
+
+  inline TypedValue getTypedValue(const attribute_id) const {
+    return column_vector.getTypedValue(pos);
+  }
+
+  const ColumnVectorT &column_vector;
+  const std::size_t length;
+  std::size_t pos;
+};
+
+template <typename FuncSpec, typename T, typename EnableT = void>
+struct Codegen;
+
+template <typename FuncSpec, typename T>
+struct Codegen<FuncSpec, T, std::enable_if_t<T::kLayout == kNativeEmbedded>> {
+  using ColumnVectorType = NativeColumnVector;
+  using FunctorSpecializer = FuncSpec;
+
+  using NativeType = typename T::cpptype;
+  using NativeTypeConst = const typename T::cpptype;
+  using NativeTypeConstRef = const NativeType&;
+  using NativeTypeConstPtr = const NativeType*;
+
+  template <typename ArgumentGen>
+  inline static TypedValue ApplyUnaryTypedValue(
+      typename ArgumentGen::NativeTypeConstRef argument,
+      const Type &result_type,
+      const typename FuncSpec::FunctorType &functor) {
+    return TypedValue(FuncSpec::Invoke(functor, argument));
+  }
+
+  template <typename ArgumentGen>
+  inline static void ApplyUnaryColumnVector(
+      const typename ArgumentGen::NativeTypeConstRef argument,
+      const typename FuncSpec::FunctorType &functor,
+      ColumnVectorType *cv) {
+    *static_cast<NativeType *>(cv->getPtrForDirectWrite()) =
+        FuncSpec::Invoke(functor, argument);
+  }
+
+  template <typename LeftGen, typename RightGen>
+  inline static TypedValue ApplyBinaryTypedValue(
+      typename LeftGen::NativeTypeConstRef left,
+      typename RightGen::NativeTypeConstRef right,
+      const Type &result_type,
+      const typename FuncSpec::FunctorType &functor) {
+    return TypedValue(FuncSpec::Invoke(functor, left, right));
+  }
+
+  template <typename LeftGen, typename RightGen>
+  inline static void ApplyBinaryColumnVector(
+      const typename LeftGen::NativeTypeConstRef left,
+      const typename RightGen::NativeTypeConstRef right,
+      const typename FuncSpec::FunctorType &functor,
+      ColumnVectorType *cv) {
+    *static_cast<NativeType *>(cv->getPtrForDirectWrite()) =
+        FuncSpec::Invoke(functor, left, right);
+  }
+
+  template <bool nullable, typename AccessorT>
+  inline static NativeTypeConstPtr GetValuePtr(
+      const AccessorT *accessor,
+      const attribute_id attr_id) {
+    return static_cast<NativeTypeConstPtr>(
+        accessor->template getUntypedValue<nullable>(attr_id));
+  }
+
+  inline static bool IsNull(const NativeType *value) {
+    return value == nullptr;
+  }
+
+  // Dereference: NativeTypeConstPtr& -> const NativeType&
+  inline static const NativeType& Dereference(const NativeType *value) {
+    return *value;
+  }
+
+  inline static const NativeType ToNativeValueConst(const TypedValue &value) {
+    return value.getLiteral<NativeType>();
+  }
+};
+
+template <typename FuncSpec, typename T>
+struct Codegen<FuncSpec, T, std::enable_if_t<T::kLayout == kNonNativeInline>> {
+  using ColumnVectorType = NativeColumnVector;
+  using FunctorSpecializer = FuncSpec;
+
+  using NativeType = void*;
+  using NativeTypeConst = const void*;
+  using NativeTypeConstRef = const void*;
+  using NativeTypeConstPtr = const void*;
+
+  template <typename ArgumentGen>
+  inline static TypedValue ApplyUnaryTypedValue(
+      typename ArgumentGen::NativeTypeConstRef argument,
+      const Type &result_type,
+      const typename FuncSpec::FunctorType &functor) {
+    void *result = std::malloc(result_type.maximumByteLength());
+    FuncSpec::Invoke(functor, argument, result);
+    return TypedValue::CreateWithOwnedData(T::kStaticTypeID,
+                                           result,
+                                           result_type.maximumByteLength());
+  }
+
+  template <typename ArgumentGen>
+  inline static void ApplyUnaryColumnVector(
+      const typename ArgumentGen::NativeTypeConstRef argument,
+      const typename FuncSpec::FunctorType &functor,
+      ColumnVectorType *cv) {
+    FuncSpec::Invoke(functor, argument, cv->getPtrForDirectWrite());
+  }
+
+  template <typename LeftGen, typename RightGen>
+  inline static TypedValue ApplyBinaryTypedValue(
+      typename LeftGen::NativeTypeConstRef left,
+      typename RightGen::NativeTypeConstRef right,
+      const Type &result_type,
+      const typename FuncSpec::FunctorType &functor) {
+    void *result = std::malloc(result_type.maximumByteLength());
+    FuncSpec::Invoke(functor, left, right, result);
+    return TypedValue::CreateWithOwnedData(T::kStaticTypeID,
+                                           result,
+                                           result_type.maximumByteLength());
+  }
+
+  template <typename LeftGen, typename RightGen>
+  inline static void ApplyBinaryColumnVector(
+      const typename LeftGen::NativeTypeConstRef left,
+      const typename RightGen::NativeTypeConstRef right,
+      const typename FuncSpec::FunctorType &functor,
+      ColumnVectorType *cv) {
+    FuncSpec::Invoke(functor, left, right, cv->getPtrForDirectWrite());
+  }
+
+  template <bool nullable, typename AccessorT>
+  inline static NativeTypeConstPtr GetValuePtr(
+      const AccessorT *accessor,
+      const attribute_id attr_id) {
+    return accessor->template getUntypedValue<nullable>(attr_id);
+  }
+
+  inline static bool IsNull(const void *value) {
+    return value == nullptr;
+  }
+
+  // Dereference: NativeTypeConstPtr& -> const NativeType&
+  inline static const void* Dereference(const void *value) {
+    return value;
+  }
+
+  inline static const void* ToNativeValueConst(const TypedValue &value) {
+    return value.getDataPtr();
+  }
+};
+
+template <typename FuncSpec, typename T>
+struct Codegen<FuncSpec, T, std::enable_if_t<T::kLayout == kOutOfLine>> {
+  using ColumnVectorType = IndirectColumnVector;
+  using FunctorSpecializer = FuncSpec;
+
+  using NativeType = TypedValue;
+  using NativeTypeConst = const TypedValue;
+  using NativeTypeConstRef = const TypedValue&;
+  using NativeTypeConstPtr = const TypedValue;
+
+  template <typename ArgumentGen>
+  inline static TypedValue ApplyUnaryTypedValue(
+      typename ArgumentGen::NativeTypeConstRef argument,
+      const Type &result_type,
+      const typename FuncSpec::FunctorType &functor) {
+    return FuncSpec::Invoke(functor, argument);
+  }
+
+  template <typename ArgumentGen>
+  inline static void ApplyUnaryColumnVector(
+      const typename ArgumentGen::NativeTypeConstRef argument,
+      const typename FuncSpec::FunctorType &functor,
+      ColumnVectorType *cv) {
+    cv->appendTypedValue(FuncSpec::Invoke(functor, argument));
+  }
+
+  template <typename LeftGen, typename RightGen>
+  inline static TypedValue ApplyBinaryTypedValue(
+      typename LeftGen::NativeTypeConstRef left,
+      typename RightGen::NativeTypeConstRef right,
+      const Type &result_type,
+      const typename FuncSpec::FunctorType &functor) {
+    return FuncSpec::Invoke(functor, left, right);
+  }
+
+  template <typename LeftGen, typename RightGen>
+  inline static void ApplyBinaryColumnVector(
+      const typename LeftGen::NativeTypeConstRef left,
+      const typename RightGen::NativeTypeConstRef right,
+      const typename FuncSpec::FunctorType &functor,
+      ColumnVectorType *cv) {
+    cv->appendTypedValue(FuncSpec::Invoke(functor, left, right));
+  }
+
+  template <bool nullable, typename AccessorT>
+  inline static NativeTypeConstPtr GetValuePtr(
+      const AccessorT *accessor,
+      const attribute_id attr_id) {
+    return accessor->getTypedValue(attr_id);
+  }
+
+  inline static bool IsNull(NativeTypeConstPtr &value) {
+    return value.isNull();
+  }
+
+  // Dereference: NativeTypeConstPtr& -> const NativeType&
+  inline static const NativeType& Dereference(NativeTypeConstPtr &value) {
+    return value;
+  }
+
+  inline static const NativeType& ToNativeValueConst(const TypedValue &value) {
+    return value;
+  }
+};
+
+template <typename ...FunctorTypes>
+struct FunctorPack {
+  template <typename Dispatcher>
+  inline static std::list<OperationPtr> GenerateOperations() {
+    std::vector<std::list<OperationPtr>> op_list_groups =
+        { Dispatcher::template Generate<FunctorTypes>()... };
+
+    std::list<OperationPtr> operations;
+    for (std::list<OperationPtr> &op_list : op_list_groups) {
+      operations.splice(operations.end(), std::move(op_list));
+    }
+    return operations;
+  }
+};
+
+struct OperationPack {
+  virtual std::vector<OperationPtr> generateOperations() = 0;
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_OPERATION_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/AddBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/AddBinaryOperation.cpp b/types/operations/binary_operations/AddBinaryOperation.cpp
deleted file mode 100644
index 8f56a61..0000000
--- a/types/operations/binary_operations/AddBinaryOperation.cpp
+++ /dev/null
@@ -1,418 +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 "types/operations/binary_operations/AddBinaryOperation.hpp"
-
-#include <string>
-#include <utility>
-
-#include "types/DateOperatorOverloads.hpp"
-#include "types/DateType.hpp"
-#include "types/DatetimeIntervalType.hpp"
-#include "types/DatetimeLit.hpp"
-#include "types/DatetimeType.hpp"
-#include "types/IntervalLit.hpp"
-#include "types/Type.hpp"
-#include "types/TypeErrors.hpp"
-#include "types/TypeFactory.hpp"
-#include "types/TypeID.hpp"
-#include "types/YearMonthIntervalType.hpp"
-#include "types/operations/binary_operations/ArithmeticBinaryOperators.hpp"
-#include "utility/EqualsAnyConstant.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-bool AddBinaryOperation::canApplyToTypes(const Type &left, const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:  // Fall through.
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      return (right.getSuperTypeID() == Type::kNumeric);
-    }
-    case kDate: {
-      return (right.getTypeID() == kYearMonthInterval);
-    }
-    case kDatetime: {
-      return (right.getTypeID() == kDatetimeInterval ||
-              right.getTypeID() == kYearMonthInterval);
-    }
-    case kDatetimeInterval: {
-      return (right.getTypeID() == kDatetime ||
-              right.getTypeID() == kDatetimeInterval);
-    }
-    case kYearMonthInterval: {
-      return (right.getTypeID() == kDate ||
-              right.getTypeID() == kDatetime ||
-              right.getTypeID() == kYearMonthInterval);
-    }
-    default:
-      return false;
-  }
-}
-
-const Type* AddBinaryOperation::resultTypeForArgumentTypes(const Type &left, const Type &right) const {
-  if (left.getSuperTypeID() == Type::kNumeric && right.getSuperTypeID() == Type::kNumeric) {
-    return TypeFactory::GetUnifyingType(left, right);
-  } else if ((left.getTypeID() == kDatetime && right.getTypeID() == kDatetimeInterval)  ||
-             (left.getTypeID() == kDatetimeInterval && right.getTypeID() == kDatetime)  ||
-             (left.getTypeID() == kDatetime && right.getTypeID() == kYearMonthInterval) ||
-             (left.getTypeID() == kYearMonthInterval && right.getTypeID() == kDatetime)) {
-    return &(DatetimeType::Instance(left.isNullable() || right.isNullable()));
-  } else if ((left.getTypeID() == kDate && right.getTypeID() == kYearMonthInterval) ||
-             (left.getTypeID() == kYearMonthInterval && right.getTypeID() == kDate)) {
-    return &(DateType::Instance(left.isNullable() || right.isNullable()));
-  } else if (left.getTypeID() == kDatetimeInterval && right.getTypeID() == kDatetimeInterval) {
-    return &(DatetimeIntervalType::Instance(left.isNullable() || right.isNullable()));
-  } else if (left.getTypeID() == kYearMonthInterval && right.getTypeID() == kYearMonthInterval) {
-    return &(YearMonthIntervalType::Instance(left.isNullable() || right.isNullable()));
-  } else {
-    return nullptr;
-  }
-}
-
-const Type* AddBinaryOperation::resultTypeForPartialArgumentTypes(const Type *left,
-                                                                  const Type *right) const {
-  if ((left == nullptr) && (right == nullptr)) {
-    return nullptr;
-  }
-
-  if ((left != nullptr) && (right != nullptr)) {
-    return resultTypeForArgumentTypes(*left, *right);
-  }
-
-  // Addition is commutative, so we just determine based on the known type,
-  // left or right.
-  const Type *known_type = (left != nullptr) ? left : right;
-  switch (known_type->getTypeID()) {
-    case kDouble:
-      // Double has highest precedence of the numeric types.
-      return &TypeFactory::GetType(kDouble, true);
-    case kDatetime:
-      // Datetime can be added with either interval type, and always yields
-      // Datetime.
-      return &TypeFactory::GetType(kDatetime, true);
-    case kDate:
-      // Date can be added with YearMonthInterval type only, and always yields
-      // Date.
-      return &TypeFactory::GetType(kDate, true);
-    default:
-      // Ambiguous or inapplicable.
-      return nullptr;
-  }
-}
-
-bool AddBinaryOperation::partialTypeSignatureIsPlausible(
-    const Type *result_type,
-    const Type *left_argument_type,
-    const Type *right_argument_type) const {
-  if ((left_argument_type == nullptr) && (right_argument_type == nullptr)) {
-    if (result_type == nullptr) {
-      return true;
-    } else if (!result_type->isNullable()) {
-      // Unknown arguments are assumed to be nullable, since they arise from
-      // untyped NULL literals in the parser. Therefore, a non-nullable result
-      // Type is not plausible with unknown arguments.
-      return false;
-    } else {
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(result_type->getTypeID(),
-                                           kInt,
-                                           kLong,
-                                           kFloat,
-                                           kDouble,
-                                           kDate,
-                                           kDatetime,
-                                           kDatetimeInterval,
-                                           kYearMonthInterval);
-    }
-  }
-
-  if ((left_argument_type != nullptr) && (right_argument_type != nullptr)) {
-    const Type *actual_result_type = resultTypeForArgumentTypes(*left_argument_type,
-                                                                *right_argument_type);
-    if (actual_result_type == nullptr) {
-      // Both argument Types are known, but this operation is NOT applicable to
-      // them. No matter what the result_type is, the signature is not
-      // plausible.
-      return false;
-    } else if (result_type == nullptr) {
-      return true;
-    } else {
-      return result_type->equals(*actual_result_type);
-    }
-  }
-
-  // Addition is commutative, so we just determine based on the known type,
-  // left or right.
-  const Type *known_argument_type = (left_argument_type != nullptr)
-                                    ? left_argument_type
-                                    : right_argument_type;
-  if (result_type == nullptr) {
-    return QUICKSTEP_EQUALS_ANY_CONSTANT(known_argument_type->getTypeID(),
-                                         kInt,
-                                         kLong,
-                                         kFloat,
-                                         kDouble,
-                                         kDate,
-                                         kDatetime,
-                                         kDatetimeInterval,
-                                         kYearMonthInterval);
-  }
-
-  if (!result_type->isNullable()) {
-    // One of the arguments is unknown, but it is nevertheless assumed
-    // nullable, since unknown argument Types arise from untyped NULL literals
-    // in the parser. Therefore, a non-nullable result Type is not plausible
-    // with an unknown argument.
-    return false;
-  }
-
-  switch (result_type->getTypeID()) {
-    case kInt:
-      return (known_argument_type->getTypeID() == kInt);
-    case kLong:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kInt, kLong);
-    case kFloat:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kInt, kFloat);
-    case kDouble:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kInt, kLong, kFloat, kDouble);
-    case kDate:
-      return (known_argument_type->getTypeID() == kDate);
-    case kDatetime:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kDatetime, kDatetimeInterval);
-    case kDatetimeInterval:
-      return (known_argument_type->getTypeID() == kDatetimeInterval);
-    case kYearMonthInterval:
-      return (known_argument_type->getTypeID() == kYearMonthInterval);
-    default:
-      return false;
-  }
-}
-
-std::pair<const Type*, const Type*> AddBinaryOperation::pushDownTypeHint(
-    const Type *result_type_hint) const {
-  if (result_type_hint == nullptr) {
-    return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-
-  switch (result_type_hint->getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble:
-    case kDatetimeInterval:
-    case kYearMonthInterval:
-      // Hint the same as the result type. Note that, for numeric types, one of
-      // the argument Types can be a less precise Type and still yield the
-      // specified result Type (e.g. DoubleType + IntType = DoubleType). We
-      // choose the highest-precision suitable Type (i.e. the same as the
-      // result type) in such cases.
-      return std::pair<const Type*, const Type*>(result_type_hint, result_type_hint);
-    case kDate:
-      // Hint is ambiguous: one argument should be a Date, other has to be
-      // kYearMonthInterval, but order is not important.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-    case kDatetime:
-      // Hint is ambiguous: one argument should be a Datetime, the other should
-      // be one of the interval types, but either order is acceptable.
-      // Fortunately, the 3 types in question have syntactically distinct
-      // representations in the SQL parser, so their literals don't need
-      // disambiguation anyway.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-    default:
-      // Inapplicable.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-}
-
-TypedValue AddBinaryOperation::applyToChecked(const TypedValue &left,
-                                              const Type &left_type,
-                                              const TypedValue &right,
-                                              const Type &right_type) const {
-  switch (left_type.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      switch (right_type.getTypeID()) {
-        case kInt:
-        case kLong:
-        case kFloat:
-        case kDouble:
-          return applyToCheckedNumericHelper<AddFunctor>(left, left_type,
-                                                         right, right_type);
-        default:
-          break;
-      }
-      break;
-    }
-    case kDate: {
-      if (right_type.getTypeID() == kYearMonthInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDate);
-        }
-
-        return TypedValue(left.getLiteral<DateLit>() + right.getLiteral<YearMonthIntervalLit>());
-      }
-      break;
-    }
-    case kDatetime: {
-      if (right_type.getTypeID() == kDatetimeInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetime);
-        }
-
-        return TypedValue(left.getLiteral<DatetimeLit>() + right.getLiteral<DatetimeIntervalLit>());
-      } else if (right_type.getTypeID() == kYearMonthInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetime);
-        }
-
-        return TypedValue(left.getLiteral<DatetimeLit>() + right.getLiteral<YearMonthIntervalLit>());
-      }
-      break;
-    }
-    case kDatetimeInterval: {
-      if (right_type.getTypeID() == kDatetime) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetime);
-        }
-
-        return TypedValue(left.getLiteral<DatetimeIntervalLit>() + right.getLiteral<DatetimeLit>());
-      } else if (right_type.getTypeID() == kDatetimeInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetimeInterval);
-        }
-
-        return TypedValue(left.getLiteral<DatetimeIntervalLit>() + right.getLiteral<DatetimeIntervalLit>());
-      }
-      break;
-    }
-    case kYearMonthInterval: {
-      if (right_type.getTypeID() == kDate) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetime);
-        }
-
-        return TypedValue(left.getLiteral<YearMonthIntervalLit>() + right.getLiteral<DateLit>());
-      } else if (right_type.getTypeID() == kDatetime) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetime);
-        }
-
-        return TypedValue(left.getLiteral<YearMonthIntervalLit>() + right.getLiteral<DatetimeLit>());
-      } else if (right_type.getTypeID() == kYearMonthInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kYearMonthInterval);
-        }
-
-        return TypedValue(left.getLiteral<YearMonthIntervalLit>() + right.getLiteral<YearMonthIntervalLit>());
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-             << left_type.getName() << " and " << right_type.getName();
-}
-
-UncheckedBinaryOperator* AddBinaryOperation::makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                                                 const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      if (right.getSuperTypeID() == Type::kNumeric) {
-        return makeNumericBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator>(left, right);
-      }
-      break;
-    }
-    case kDate: {
-      if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<
-            AddArithmeticUncheckedBinaryOperator,
-            DateType,
-            DateLit,
-            YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kDatetime: {
-      if (right.getTypeID() == kDatetimeInterval) {
-        return makeDateBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator,
-                                                 DatetimeType,
-                                                 DatetimeLit, DatetimeIntervalLit>(left, right);
-      } else if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator,
-                                                 DatetimeType,
-                                                 DatetimeLit, YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kDatetimeInterval: {
-      if (right.getTypeID() == kDatetime) {
-        return makeDateBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator,
-                                                 DatetimeType,
-                                                 DatetimeIntervalLit, DatetimeLit>(left, right);
-      } else if (right.getTypeID() == kDatetimeInterval) {
-        return makeDateBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator,
-                                                 DatetimeIntervalType,
-                                                 DatetimeIntervalLit, DatetimeIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kYearMonthInterval: {
-      if (right.getTypeID() == kDate) {
-        return makeDateBinaryOperatorOuterHelper<
-            AddArithmeticUncheckedBinaryOperator,
-            DateType,
-            YearMonthIntervalLit,
-            DateLit>(left, right);
-      } else if (right.getTypeID() == kDatetime) {
-        return makeDateBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator,
-                                                 DatetimeType,
-                                                 YearMonthIntervalLit, DatetimeLit>(left, right);
-      } else if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator,
-                                                 YearMonthIntervalType,
-                                                 YearMonthIntervalLit, YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/ArithmeticBinaryFunctorOverloads.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryFunctorOverloads.hpp b/types/operations/binary_operations/ArithmeticBinaryFunctorOverloads.hpp
new file mode 100644
index 0000000..4c6f76c
--- /dev/null
+++ b/types/operations/binary_operations/ArithmeticBinaryFunctorOverloads.hpp
@@ -0,0 +1,176 @@
+/**
+ * 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_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_FUNCTOR_OVERLOADS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_FUNCTOR_OVERLOADS_HPP_
+
+#include <cmath>
+#include <cstddef>
+#include <cstdint>
+#include <type_traits>
+#include <utility>
+
+#include "types/DateOperatorOverloads.hpp"
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+// We use these functors instead of the standard-library ones, because the
+// standard-library functors in <functional> have to be instantiated for the
+// most specific argument type, which would unnecessisarily introduce
+// multiple copies of distinct template instantiations of operators.
+template <typename LeftCppType, typename RightCppType, typename EnableT = void>
+struct AddFunctorOverloads {
+  inline auto operator() (const LeftCppType &left,
+                          const RightCppType &right) const -> decltype(left + right) {
+    return left + right;
+  }
+};
+
+// NOTE(zuyu): The C++ compiler in general converts all integers to floats
+//             when doing the following operations,
+//             but we could like to return double instead.
+template <>
+struct AddFunctorOverloads<std::int64_t, float> {
+  inline double operator() (const std::int64_t &left, const float &right) const {
+    return static_cast<double>(left) + static_cast<double>(right);
+  }
+};
+
+template <>
+struct AddFunctorOverloads<float, std::int64_t> {
+  inline double operator() (const float &left, const std::int64_t &right) const {
+    return static_cast<double>(left) + static_cast<double>(right);
+  }
+};
+
+template <typename LeftCppType, typename RightCppType, typename EnableT = void>
+struct SubtractFunctorOverloads {
+  inline auto operator() (const LeftCppType &left,
+                          const RightCppType &right) const -> decltype(left - right) {
+    return left - right;
+  }
+};
+
+// NOTE(zuyu): The C++ compiler in general converts all integers to floats
+//             when doing the following operations,
+//             but we could like to return double instead.
+template <>
+struct SubtractFunctorOverloads<std::int64_t, float> {
+  inline double operator() (const std::int64_t &left, const float &right) const {
+    return static_cast<double>(left) - static_cast<double>(right);
+  }
+};
+
+template <>
+struct SubtractFunctorOverloads<float, std::int64_t> {
+  inline double operator() (const float &left, const std::int64_t &right) const {
+    return static_cast<double>(left) - static_cast<double>(right);
+  }
+};
+
+template <typename LeftCppType, typename RightCppType, typename EnableT = void>
+struct MultiplyFunctorOverloads {
+  inline auto operator() (const LeftCppType &left,
+                          const RightCppType &right) const -> decltype(left * right) {
+    return left * right;
+  }
+};
+
+// NOTE(zuyu): The C++ compiler in general converts all integers to floats
+//             when doing the following operations,
+//             but we could like to return double instead.
+template <>
+struct MultiplyFunctorOverloads<std::int64_t, float> {
+  inline double operator() (const std::int64_t &left, const float &right) const {
+    return static_cast<double>(left) * static_cast<double>(right);
+  }
+};
+
+template <>
+struct MultiplyFunctorOverloads<float, std::int64_t> {
+  inline double operator() (const float &left, const std::int64_t &right) const {
+    return static_cast<double>(left) * static_cast<double>(right);
+  }
+};
+
+template <typename LeftCppType, typename RightCppType, typename EnableT = void>
+struct DivideFunctorOverloads {
+  inline auto operator() (const LeftCppType &left,
+                          const RightCppType &right) const -> decltype(left / right) {
+    return left / right;
+  }
+};
+
+// NOTE(zuyu): The C++ compiler in general converts all integers to floats
+//             when doing the following operations,
+//             but we could like to return double instead.
+template <>
+struct DivideFunctorOverloads<std::int64_t, float> {
+  inline double operator() (const std::int64_t &left, const float &right) const {
+    return static_cast<double>(left) / static_cast<double>(right);
+  }
+};
+
+template <>
+struct DivideFunctorOverloads<float, std::int64_t> {
+  inline double operator() (const float &left, const std::int64_t &right) const {
+    return static_cast<double>(left) / static_cast<double>(right);
+  }
+};
+
+template <typename LeftCppType, typename RightCppType, typename EnableT = void>
+struct ModuloFunctorOverloads;
+
+template <typename LeftCppType, typename RightCppType>
+struct ModuloFunctorOverloads<
+    LeftCppType, RightCppType,
+    std::enable_if_t<meta::EqualsAny<LeftCppType, int, std::int64_t>::value &&
+                     meta::EqualsAny<RightCppType, int, std::int64_t>::value>> {
+  inline auto operator() (const LeftCppType &left,
+                          const RightCppType &right) const -> decltype(left % right) {
+    return left % right;
+  }
+};
+
+// NOTE(jianqiao): The C++11 standard specifies the following type signatures for fmod:
+// (1) (double, double) -> double
+// (2) (float, float) -> float
+// (3) (long double, long double) -> long double
+// (3) (Arithmetic, Arithmetic) -> double
+template <typename LeftCppType, typename RightCppType>
+struct ModuloFunctorOverloads<
+    LeftCppType, RightCppType,
+    std::enable_if_t<meta::EqualsAny<LeftCppType, float, double>::value ||
+                     meta::EqualsAny<RightCppType, float, double>::value>> {
+  inline auto operator() (const LeftCppType &left,
+                          const RightCppType &right) const -> decltype(std::fmod(left, right)) {
+    return std::fmod(left, right);
+  }
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_FUNCTOR_OVERLOADS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/ArithmeticBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryOperation.hpp b/types/operations/binary_operations/ArithmeticBinaryOperation.hpp
deleted file mode 100644
index f9a27a8..0000000
--- a/types/operations/binary_operations/ArithmeticBinaryOperation.hpp
+++ /dev/null
@@ -1,404 +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_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATION_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATION_HPP_
-
-#include <string>
-
-#include "types/DoubleType.hpp"
-#include "types/FloatType.hpp"
-#include "types/IntType.hpp"
-#include "types/LongType.hpp"
-#include "types/NumericTypeUnifier.hpp"
-#include "types/Type.hpp"
-#include "types/TypeErrors.hpp"
-#include "types/TypeFactory.hpp"
-#include "types/TypeID.hpp"
-#include "types/TypedValue.hpp"
-#include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief A BinaryOperation which applies to and yields values
- *        including numeric, datetime, and intervals.
- **/
-class ArithmeticBinaryOperation : public BinaryOperation {
- protected:
-  explicit ArithmeticBinaryOperation(const BinaryOperationID operation_id)
-      : BinaryOperation(operation_id) {
-  }
-
-  template <template <typename LeftCppType, typename RightCppType> class OperationFunctor>
-  TypedValue applyToCheckedIntegerHelper(const TypedValue &left,
-                                         const Type &left_type,
-                                         const TypedValue &right,
-                                         const Type &right_type) const;
-
-  template <template <typename LeftCppType, typename RightCppType> class OperationFunctor>
-  TypedValue applyToCheckedNumericHelper(const TypedValue &left,
-                                         const Type &left_type,
-                                         const TypedValue &right,
-                                         const Type &right_type) const;
-
-  template <template <class ResultType,
-                      typename LeftCppType, bool left_nullable,
-                      typename RightCppType, bool right_nullable> class OperatorType>
-  UncheckedBinaryOperator* makeIntegerBinaryOperatorOuterHelper(const Type &left,
-                                                                const Type &right) const;
-
-  template <template <class ResultType,
-                      typename LeftCppType, bool left_nullable,
-                      typename RightCppType, bool right_nullable> class OperatorType,
-            typename LeftType, bool left_nullable>
-  UncheckedBinaryOperator* makeIntegerBinaryOperatorInnerHelper(const Type &left,
-                                                                const Type &right) const;
-
-  template <template <class ResultType,
-                      typename LeftCppType, bool left_nullable,
-                      typename RightCppType, bool right_nullable> class OperatorType>
-  UncheckedBinaryOperator* makeNumericBinaryOperatorOuterHelper(const Type &left,
-                                                                const Type &right) const;
-
-  template <template <class ResultType,
-                      typename LeftCppType, bool left_nullable,
-                      typename RightCppType, bool right_nullable> class OperatorType,
-            typename LeftType, bool left_nullable>
-  UncheckedBinaryOperator* makeNumericBinaryOperatorInnerHelper(const Type &left,
-                                                                const Type &right) const;
-
-  template <template <typename ResultType,
-                      typename LeftCppType, bool left_nullable,
-                      typename RightCppType, bool right_nullable> class OperatorType,
-            typename ResultType,
-            typename LeftCppType,
-            typename RightCppType>
-  UncheckedBinaryOperator* makeDateBinaryOperatorOuterHelper(const Type &left,
-                                                             const Type &right) const;
-
-  template <template <typename ResultType,
-                      typename LeftCppType, bool left_nullable,
-                      typename RightCppType, bool right_nullable> class OperatorType,
-            typename ResultType,
-            typename LeftCppType,
-            bool left_nullable,
-            typename RightCppType>
-  UncheckedBinaryOperator* makeDateBinaryOperatorInnerHelper(const Type &right) const;
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(ArithmeticBinaryOperation);
-};
-
-/** @} */
-
-// ----------------------------------------------------------------------------
-// Templated method implementations follow:
-
-template <template <typename LeftCppType, typename RightCppType> class OperationFunctor>
-TypedValue ArithmeticBinaryOperation::applyToCheckedIntegerHelper(
-    const TypedValue &left,
-    const Type &left_type,
-    const TypedValue &right,
-    const Type &right_type) const {
-  DCHECK(left_type.getTypeID() == TypeID::kInt
-         || left_type.getTypeID() == TypeID::kLong);
-  DCHECK(right_type.getTypeID() == TypeID::kInt
-         || right_type.getTypeID() == TypeID::kLong);
-
-  const Type *unifier = TypeFactory::GetUnifyingType(left_type, right_type);
-  DCHECK(unifier != nullptr);
-
-  if (left.isNull() || right.isNull()) {
-    return unifier->makeNullValue();
-  }
-
-  const TypedValue left_coerced = unifier->coerceValue(left, left_type);
-  const TypedValue right_coerced = unifier->coerceValue(right, right_type);
-
-  switch (unifier->getTypeID()) {
-    case kInt: {
-      OperationFunctor<IntType::cpptype, IntType::cpptype> operation_functor;
-      return TypedValue(operation_functor(left_coerced.getLiteral<IntType::cpptype>(),
-                                          right_coerced.getLiteral<IntType::cpptype>()));
-    }
-    case kLong: {
-      OperationFunctor<LongType::cpptype, LongType::cpptype> operation_functor;
-      return TypedValue(operation_functor(left_coerced.getLiteral<LongType::cpptype>(),
-                                          right_coerced.getLiteral<LongType::cpptype>()));
-    }
-    default: {
-      LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-                 << left_type.getName() << " and " << right_type.getName();
-    }
-  }
-
-  QUICKSTEP_UNREACHABLE();
-}
-
-template <template <typename LeftCppType, typename RightCppType> class OperationFunctor>
-TypedValue ArithmeticBinaryOperation::applyToCheckedNumericHelper(
-    const TypedValue &left,
-    const Type &left_type,
-    const TypedValue &right,
-    const Type &right_type) const {
-  DCHECK_EQ(Type::kNumeric, left_type.getSuperTypeID());
-  DCHECK_EQ(Type::kNumeric, right_type.getSuperTypeID());
-
-  const Type *unifier = TypeFactory::GetUnifyingType(left_type, right_type);
-  DCHECK(unifier != nullptr);
-
-  if (left.isNull() || right.isNull()) {
-    return unifier->makeNullValue();
-  }
-
-  const TypedValue left_coerced = unifier->coerceValue(left, left_type);
-  const TypedValue right_coerced = unifier->coerceValue(right, right_type);
-
-  switch (unifier->getTypeID()) {
-    case kInt: {
-      OperationFunctor<IntType::cpptype, IntType::cpptype> operation_functor;
-      return TypedValue(operation_functor(left_coerced.getLiteral<IntType::cpptype>(),
-                                          right_coerced.getLiteral<IntType::cpptype>()));
-    }
-    case kLong: {
-      OperationFunctor<LongType::cpptype, LongType::cpptype> operation_functor;
-      return TypedValue(operation_functor(left_coerced.getLiteral<LongType::cpptype>(),
-                                          right_coerced.getLiteral<LongType::cpptype>()));
-    }
-    case kFloat: {
-      OperationFunctor<FloatType::cpptype, FloatType::cpptype> operation_functor;
-      return TypedValue(operation_functor(left_coerced.getLiteral<FloatType::cpptype>(),
-                                          right_coerced.getLiteral<FloatType::cpptype>()));
-    }
-    case kDouble: {
-      OperationFunctor<DoubleType::cpptype, DoubleType::cpptype> operation_functor;
-      return TypedValue(operation_functor(left_coerced.getLiteral<DoubleType::cpptype>(),
-                                          right_coerced.getLiteral<DoubleType::cpptype>()));
-    }
-    default: {
-      LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-                 << left_type.getName() << " and " << right_type.getName();
-    }
-  }
-
-  QUICKSTEP_UNREACHABLE();
-}
-
-template <template <class ResultType,
-                    typename LeftCppType, bool left_nullable,
-                    typename RightCppType, bool right_nullable> class OperatorType>
-UncheckedBinaryOperator* ArithmeticBinaryOperation::makeIntegerBinaryOperatorOuterHelper(
-    const Type &left,
-    const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-      if (left.isNullable()) {
-        return makeIntegerBinaryOperatorInnerHelper<OperatorType, IntType, true>(
-            left, right);
-      } else {
-        return makeIntegerBinaryOperatorInnerHelper<OperatorType, IntType, false>(
-            left, right);
-      }
-    case kLong:
-      if (left.isNullable()) {
-        return makeIntegerBinaryOperatorInnerHelper<OperatorType, LongType, true>(
-            left, right);
-      } else {
-        return makeIntegerBinaryOperatorInnerHelper<OperatorType, LongType, false>(
-            left, right);
-      }
-    default:
-      throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-  }
-}
-
-template <template <class ResultType,
-                    typename LeftCppType, bool left_nullable,
-                    typename RightCppType, bool right_nullable> class OperatorType,
-          typename LeftType, bool left_nullable>
-UncheckedBinaryOperator* ArithmeticBinaryOperation::makeIntegerBinaryOperatorInnerHelper(
-    const Type &left,
-    const Type &right) const {
-  switch (right.getTypeID()) {
-    case kInt:
-      if (right.isNullable()) {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, IntType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename IntType::cpptype, true>();
-      } else {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, IntType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename IntType::cpptype, false>();
-      }
-    case kLong:
-      if (right.isNullable()) {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, LongType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename LongType::cpptype, true>();
-      } else {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, LongType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename LongType::cpptype, false>();
-      }
-    default:
-      throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-  }
-}
-
-template <template <class ResultType,
-                    typename LeftCppType, bool left_nullable,
-                    typename RightCppType, bool right_nullable> class OperatorType>
-UncheckedBinaryOperator* ArithmeticBinaryOperation::makeNumericBinaryOperatorOuterHelper(
-    const Type &left,
-    const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-      if (left.isNullable()) {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, IntType, true>(
-            left, right);
-      } else {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, IntType, false>(
-            left, right);
-      }
-    case kLong:
-      if (left.isNullable()) {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, LongType, true>(
-            left, right);
-      } else {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, LongType, false>(
-            left, right);
-      }
-    case kFloat:
-      if (left.isNullable()) {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, FloatType, true>(
-            left, right);
-      } else {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, FloatType, false>(
-            left, right);
-      }
-    case kDouble:
-      if (left.isNullable()) {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, DoubleType, true>(
-            left, right);
-      } else {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, DoubleType, false>(
-            left, right);
-      }
-    default:
-      throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-  }
-}
-
-template <template <class ResultType,
-                    typename LeftCppType, bool left_nullable,
-                    typename RightCppType, bool right_nullable> class OperatorType,
-          typename LeftType, bool left_nullable>
-UncheckedBinaryOperator* ArithmeticBinaryOperation::makeNumericBinaryOperatorInnerHelper(
-    const Type &left,
-    const Type &right) const {
-  switch (right.getTypeID()) {
-    case kInt:
-      if (right.isNullable()) {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, IntType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename IntType::cpptype, true>();
-      } else {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, IntType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename IntType::cpptype, false>();
-      }
-    case kLong:
-      if (right.isNullable()) {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, LongType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename LongType::cpptype, true>();
-      } else {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, LongType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename LongType::cpptype, false>();
-      }
-    case kFloat:
-      if (right.isNullable()) {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, FloatType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename FloatType::cpptype, true>();
-      } else {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, FloatType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename FloatType::cpptype, false>();
-      }
-    case kDouble:
-      if (right.isNullable()) {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, DoubleType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename DoubleType::cpptype, true>();
-      } else {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, DoubleType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename DoubleType::cpptype, false>();
-      }
-    default:
-      throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-  }
-}
-
-template <template <typename ResultType,
-                    typename LeftCppType, bool left_nullable,
-                    typename RightCppType, bool right_nullable> class OperatorType,
-          typename ResultType,
-          typename LeftCppType,
-          typename RightCppType>
-UncheckedBinaryOperator* ArithmeticBinaryOperation::makeDateBinaryOperatorOuterHelper(
-    const Type &left,
-    const Type &right) const {
-  if (left.isNullable()) {
-    return makeDateBinaryOperatorInnerHelper<OperatorType, ResultType, LeftCppType, true, RightCppType>(right);
-  } else {
-    return makeDateBinaryOperatorInnerHelper<OperatorType, ResultType, LeftCppType, false, RightCppType>(right);
-  }
-}
-
-template <template <typename ResultType,
-                    typename LeftCppType, bool left_nullable,
-                    typename RightCppType, bool right_nullable> class OperatorType,
-          typename ResultType,
-          typename LeftCppType,
-          bool left_nullable,
-          typename RightCppType>
-UncheckedBinaryOperator* ArithmeticBinaryOperation::makeDateBinaryOperatorInnerHelper(
-    const Type &right) const {
-  if (right.isNullable()) {
-    return new OperatorType<ResultType, LeftCppType, left_nullable, RightCppType, true>();
-  } else {
-    return new OperatorType<ResultType, LeftCppType, left_nullable, RightCppType, false>();
-  }
-}
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/ArithmeticBinaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryOperations.hpp b/types/operations/binary_operations/ArithmeticBinaryOperations.hpp
new file mode 100644
index 0000000..fa4d926
--- /dev/null
+++ b/types/operations/binary_operations/ArithmeticBinaryOperations.hpp
@@ -0,0 +1,182 @@
+/**
+ * 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_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATIONS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATIONS_HPP_
+
+#include <string>
+#include <tuple>
+
+#include "types/DateType.hpp"
+#include "types/DatetimeIntervalType.hpp"
+#include "types/DatetimeLit.hpp"
+#include "types/DatetimeType.hpp"
+#include "types/IntervalLit.hpp"
+#include "types/NumericTypeUnifier.hpp"
+#include "types/Type.hpp"
+#include "types/TypeErrors.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/YearMonthIntervalType.hpp"
+#include "types/operations/binary_operations/ArithmeticBinaryFunctorOverloads.hpp"
+#include "types/operations/binary_operations/BinaryOperationWrapper.hpp"
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename LeftT, typename RightT, typename ResultT,
+          template <typename LeftCppType,
+                    typename RightCppType,
+                    typename EnableT = void> class FunctorOverloadsT,
+          typename FunctorNameT>
+struct ArithmeticBinaryFunctor : public BinaryFunctor<LeftT, RightT, ResultT> {
+  ArithmeticBinaryFunctor() : spec() {}
+  inline typename ResultT::cpptype apply(const typename LeftT::cpptype &left,
+                                         const typename RightT::cpptype &right) const {
+    return spec(left, right);
+  }
+  inline static std::string GetName() {
+    return FunctorNameT::ToString();
+  }
+  const FunctorOverloadsT<typename LeftT::cpptype,
+                          typename RightT::cpptype> spec;
+};
+
+template <typename LeftT, typename RightT, typename ResultT>
+using AddFunctor = ArithmeticBinaryFunctor<LeftT, RightT, ResultT,
+                                           AddFunctorOverloads,
+                                           meta::StringLiteral<'+'>>;
+
+template <typename LeftT, typename RightT, typename ResultT>
+using SubtractFunctor = ArithmeticBinaryFunctor<LeftT, RightT, ResultT,
+                                                SubtractFunctorOverloads,
+                                                meta::StringLiteral<'-'>>;
+
+template <typename LeftT, typename RightT, typename ResultT>
+using MultiplyFunctor = ArithmeticBinaryFunctor<LeftT, RightT, ResultT,
+                                                MultiplyFunctorOverloads,
+                                                meta::StringLiteral<'*'>>;
+
+template <typename LeftT, typename RightT, typename ResultT>
+using DivideFunctor = ArithmeticBinaryFunctor<LeftT, RightT, ResultT,
+                                              DivideFunctorOverloads,
+                                              meta::StringLiteral<'/'>>;
+
+template <typename LeftT, typename RightT, typename ResultT>
+using ModuloFunctor = ArithmeticBinaryFunctor<LeftT, RightT, ResultT,
+                                              ModuloFunctorOverloads,
+                                              meta::StringLiteral<'%'>>;
+
+// ----------------------------------------------------------------------------
+// Packs of functors:
+
+using AddBinaryFunctorPack = FunctorPack<
+// Numeric
+    BinaryFunctorCrossProductPack<
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        AddFunctor, NumericTypeUnifier>,
+// Date
+    AddFunctor<DateType, YearMonthIntervalType, DateType>,
+    AddFunctor<YearMonthIntervalType, DateType, DateType>,
+// Datetime
+    AddFunctor<DatetimeType, DatetimeIntervalType, DatetimeType>,
+    AddFunctor<DatetimeType, YearMonthIntervalType, DatetimeType>,
+    AddFunctor<DatetimeIntervalType, DatetimeType, DatetimeType>,
+    AddFunctor<YearMonthIntervalType, DatetimeType, DatetimeType>,
+// DatetimeInterval
+    AddFunctor<DatetimeIntervalType, DatetimeIntervalType, DatetimeIntervalType>,
+// YearMonthInterval
+    AddFunctor<YearMonthIntervalType, YearMonthIntervalType, YearMonthIntervalType>
+>;
+
+using SubtractBinaryFunctorPack = FunctorPack<
+// Numeric
+    BinaryFunctorCrossProductPack<
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        SubtractFunctor, NumericTypeUnifier>,
+// Date
+    SubtractFunctor<DateType, YearMonthIntervalType, DateType>,
+    // TODO(quickstep-team):
+    // Implement SubtractFunctor<DateType, DateType, YearMonthIntervalType>,
+// Datetime
+    SubtractFunctor<DatetimeType, DatetimeIntervalType, DatetimeType>,
+    SubtractFunctor<DatetimeType, YearMonthIntervalType, DatetimeType>,
+    SubtractFunctor<DatetimeType, DatetimeType, DatetimeIntervalType>,
+// DatetimeInterval
+    SubtractFunctor<DatetimeIntervalType, DatetimeIntervalType, DatetimeIntervalType>,
+// YearMonthInterval
+    SubtractFunctor<YearMonthIntervalType, YearMonthIntervalType, YearMonthIntervalType>
+>;
+
+using MultiplyBinaryFunctorPack = FunctorPack<
+// Numeric
+    BinaryFunctorCrossProductPack<
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        MultiplyFunctor, NumericTypeUnifier>,
+// DatetimeInterval and YearMonthInterval
+    BinaryFunctorCrossProductPack<
+        std::tuple<DatetimeIntervalType, YearMonthIntervalType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        MultiplyFunctor, meta::PairSelectorLeft>,
+    BinaryFunctorCrossProductPack<
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        std::tuple<DatetimeIntervalType, YearMonthIntervalType>,
+        MultiplyFunctor, meta::PairSelectorRight>
+>;
+
+using DivideBinaryFunctorPack = FunctorPack<
+// Numeric
+    BinaryFunctorCrossProductPack<
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        DivideFunctor, NumericTypeUnifier>,
+// DatetimeInterval and YearMonthInterval
+    BinaryFunctorCrossProductPack<
+        std::tuple<DatetimeIntervalType, YearMonthIntervalType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        DivideFunctor, meta::PairSelectorLeft>
+>;
+
+using ModuloBinaryFunctorPack = FunctorPack<
+// Numeric
+    BinaryFunctorCrossProductPack<
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        ModuloFunctor, NumericTypeUnifier>
+>;
+
+using ArithmeticBinaryFunctorPack = FunctorPack<
+    AddBinaryFunctorPack,
+    SubtractBinaryFunctorPack,
+    MultiplyBinaryFunctorPack,
+    DivideBinaryFunctorPack,
+    ModuloBinaryFunctorPack
+>;
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATIONS_HPP_


[30/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/parser/preprocessed/SqlLexer_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.cpp b/parser/preprocessed/SqlLexer_gen.cpp
index bd77ee1..5cc996d 100644
--- a/parser/preprocessed/SqlLexer_gen.cpp
+++ b/parser/preprocessed/SqlLexer_gen.cpp
@@ -1,6 +1,6 @@
-#line 1 "SqlLexer_gen.cpp"
+#line 2 "SqlLexer_gen.cpp"
 
-#line 3 "SqlLexer_gen.cpp"
+#line 4 "SqlLexer_gen.cpp"
 
 #define  YY_INT_ALIGNED short int
 
@@ -9,89 +9,11 @@
 #define FLEX_SCANNER
 #define YY_FLEX_MAJOR_VERSION 2
 #define YY_FLEX_MINOR_VERSION 6
-#define YY_FLEX_SUBMINOR_VERSION 3
+#define YY_FLEX_SUBMINOR_VERSION 0
 #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. */
@@ -166,22 +88,36 @@ typedef unsigned int flex_uint32_t;
 
 #endif /* ! FLEXINT_H */
 
-/* TODO: this is always defined, so inline it */
-#define yyconst const
+#ifdef __cplusplus
 
-#if defined(__GNUC__) && __GNUC__ >= 3
-#define yynoreturn __attribute__((__noreturn__))
+/* 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
+#define yyconst const
 #else
-#define yynoreturn
+#define yyconst
 #endif
 
 /* Returned upon end-of-file. */
 #define YY_NULL 0
 
-/* Promotes a possibly negative, possibly signed char to an
- *   integer in range [0..255] for use as an array index.
+/* 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.
  */
-#define YY_SC_TO_UI(c) ((YY_CHAR) (c))
+#define YY_SC_TO_UI(c) ((unsigned int) (unsigned char) c)
 
 /* An opaque pointer. */
 #ifndef YY_TYPEDEF_YY_SCANNER_T
@@ -205,16 +141,20 @@ 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. */
@@ -247,10 +187,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.
@@ -283,6 +223,7 @@ 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
@@ -297,12 +238,12 @@ struct yy_buffer_state
 	/* Size of input buffer in bytes, not including room for EOB
 	 * characters.
 	 */
-	int yy_buf_size;
+	yy_size_t yy_buf_size;
 
 	/* Number of characters read into yy_ch_buf, not including EOB
 	 * characters.
 	 */
-	int yy_n_chars;
+	yy_size_t 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
@@ -325,7 +266,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.
 	 */
@@ -359,33 +300,36 @@ 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 ( const char *yy_str , yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_scan_bytes ( const char *bytes, int 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 (yyconst char *yy_str ,yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_bytes (yyconst char *bytes,yy_size_t 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 ){ \
@@ -395,6 +339,7 @@ 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 ){\
@@ -404,34 +349,40 @@ 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 flex_uint8_t YY_CHAR;
+
+typedef unsigned char 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 );
-static void yynoreturn yy_fatal_error ( const 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 );
+#if defined(__GNUC__) && __GNUC__ >= 3
+__attribute__((__noreturn__))
+#endif
+static void yy_fatal_error (yyconst 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 = (int) (yy_cp - yy_bp); \
+	yyleng = (size_t) (yy_cp - yy_bp); \
 	yyg->yy_hold_char = *yy_cp; \
 	*yy_cp = '\0'; \
 	yyg->yy_c_buf_p = yy_cp;
-#define YY_NUM_RULES 163
-#define YY_END_OF_BUFFER 164
+
+#define YY_NUM_RULES 164
+#define YY_END_OF_BUFFER 165
 /* This struct is not used in this scanner,
    but its presence is necessary. */
 struct yy_trans_info
@@ -439,77 +390,77 @@ struct yy_trans_info
 	flex_int32_t yy_verify;
 	flex_int32_t yy_nxt;
 	};
-static const flex_int16_t yy_accept[599] =
+static yyconst flex_int16_t yy_accept[600] =
     {   0,
         0,    0,    0,    0,    0,    0,    0,    0,    0,    0,
-        0,    0,  164,    2,    2,  162,  162,  161,  160,  162,
-      139,  135,  138,  135,  135,  158,  131,  128,  132,  157,
-      157,  157,  157,  157,  157,  157,  157,  157,  157,  157,
-      157,  157,  157,  157,  157,  157,  157,  157,  157,  157,
-      157,  157,  157,  157,  136,    4,    5,    5,    3,  154,
-      154,  151,  155,  155,  149,  156,  156,  153,    1,  161,
-      129,  159,  158,  158,  158,    0,  133,  130,  134,  157,
-      157,  157,  157,   10,  157,  157,  157,   22,  157,  157,
-      157,  157,  157,  157,  157,  157,  157,  157,  157,  137,
-
-      157,  157,  157,  157,  157,  157,  157,  157,  157,  157,
-      157,  157,  157,   60,   69,  157,  157,  157,  157,  157,
-      157,  157,  157,  157,  157,  157,   83,   84,  157,  157,
-      157,  157,  157,  157,  157,  157,  157,  157,  157,  157,
-      157,  157,  157,  157,  157,  157,  157,  157,  157,  157,
-      157,  157,  157,    4,    5,    3,  154,  150,  155,  148,
-      148,  140,  142,  143,  144,  145,  146,  147,  148,  156,
-      152,  159,  158,    0,  158,    6,    7,  157,    9,   11,
-      157,  157,   15,  157,  157,  157,  157,  157,  157,  157,
-      157,  157,  157,  157,   33,  157,  157,  157,  157,  157,
-
-      157,  157,  157,   44,  157,  157,  157,  157,  157,  157,
-      157,   52,  157,  157,  157,  157,  157,  157,  157,  157,
-      157,   64,  157,   71,  157,  157,  157,  157,  157,  157,
-      157,   79,  157,   82,  157,  157,  157,  157,  157,  157,
-      157,  157,  157,  157,  157,  157,  157,  100,  157,  157,
-      105,  106,  157,  157,  157,  157,  157,  157,  157,  157,
-      157,  157,  157,  157,  157,  157,  157,  157,  140,  142,
-      141,  157,  157,  157,  157,  157,  157,  157,   20,   23,
-      157,  157,  157,   28,  157,  157,  157,   31,  157,  157,
-      157,  157,   38,  157,  157,   42,   43,  157,  157,  157,
-
-      157,  157,  157,  157,  157,   54,   55,  157,   57,  157,
-       59,  157,  157,  157,  157,   68,   70,   72,   73,   74,
-      157,   76,  157,  157,   80,  157,  157,   87,  157,  157,
-      157,  157,  157,   94,  157,   96,  157,  157,  157,  102,
-      157,  157,  157,  157,  157,  157,  110,  111,  113,  157,
-      157,  157,  157,  157,  157,  157,  157,  122,  157,  157,
-      125,  126,  140,  141,    8,  157,  157,  157,  157,  157,
-      157,  157,   25,  157,  157,  157,  157,  157,  157,  157,
-      157,  157,  157,  157,  157,  157,  157,  157,  157,   48,
-       49,   50,  157,  157,   56,  157,   61,   62,  157,  157,
-
-      157,   75,  157,   78,   81,   85,   86,  157,  157,  157,
-      157,  157,   95,  157,  157,   99,  157,  157,  157,  157,
-      157,  109,  157,  157,  157,  116,  157,  157,  119,  157,
-      157,  123,  157,  157,  157,  157,   14,  157,  157,  157,
-      157,  157,   26,  157,   29,  157,  157,  157,  157,  157,
-       36,  157,  157,  157,   41,  157,   46,  157,  157,  157,
-       58,   63,  157,  157,  157,   77,  157,  157,  157,  157,
-      157,  157,   98,  157,  103,  104,  157,  157,  157,  157,
-      157,  117,  118,  120,  157,  124,  157,  157,   13,  157,
-      157,  157,  157,  157,  157,   21,   30,  157,   34,   35,
-
-      157,  157,  157,  157,   47,  157,   53,   65,  157,  157,
-      157,   90,  157,   92,  157,  157,  157,  157,  157,  157,
-      157,  157,  121,  157,  157,  157,  157,  157,  157,  157,
-      157,   32,  157,  157,   40,  157,  157,  157,   67,  157,
-      157,   93,  157,  157,  107,  157,  157,  157,  157,  157,
-       12,  157,  157,  157,  157,   24,  157,   37,  157,  157,
-       51,   66,   88,   91,  157,  157,  108,  112,  157,  115,
-      127,   16,  157,  157,  157,   27,   39,  157,   89,   97,
-      157,  157,  157,   18,   19,  157,  157,  114,  157,  157,
-      157,  157,  157,  101,  157,   45,   17,    0
+        0,    0,  165,    2,    2,  163,  163,  162,  161,  163,
+      140,  136,  139,  136,  136,  159,  132,  129,  133,  158,
+      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
+      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
+      158,  158,  158,  158,  137,    4,    5,    5,    3,  155,
+      155,  152,  156,  156,  150,  157,  157,  154,    1,  162,
+      130,  160,  159,  159,  159,    0,  134,  131,  135,  158,
+      158,  158,  158,   10,  158,  158,  158,   23,  158,  158,
+      158,  158,  158,  158,  158,  158,  158,  158,  158,  138,
+
+      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
+      158,  158,  158,   61,   70,  158,  158,  158,  158,  158,
+      158,  158,  158,  158,  158,  158,   84,   85,  158,  158,
+      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
+      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
+      158,  158,  158,    4,    5,    3,  155,  151,  156,  149,
+      149,  141,  143,  144,  145,  146,  147,  148,  149,  157,
+      153,  160,  159,    0,  159,    6,    7,  158,    9,   11,
+      158,  158,   15,  158,  158,  158,  158,  158,  158,  158,
+      158,  158,  158,  158,   34,  158,  158,  158,  158,  158,
+
+      158,  158,  158,   45,  158,  158,  158,  158,  158,  158,
+      158,   53,  158,  158,  158,  158,  158,  158,  158,  158,
+      158,   65,  158,   72,  158,  158,  158,  158,  158,  158,
+      158,   80,  158,   83,  158,  158,  158,  158,  158,  158,
+      158,  158,  158,  158,  158,  158,  158,  101,  158,  158,
+      106,  107,  158,  158,  158,  158,  158,  158,  158,  158,
+      158,  158,  158,  158,  158,  158,  158,  158,  141,  143,
+      142,  158,  158,  158,  158,  158,  158,  158,   20,   21,
+       24,  158,  158,  158,   29,  158,  158,  158,   32,  158,
+      158,  158,  158,   39,  158,  158,   43,   44,  158,  158,
+
+      158,  158,  158,  158,  158,  158,   55,   56,  158,   58,
+      158,   60,  158,  158,  158,  158,   69,   71,   73,   74,
+       75,  158,   77,  158,  158,   81,  158,  158,   88,  158,
+      158,  158,  158,  158,   95,  158,   97,  158,  158,  158,
+      103,  158,  158,  158,  158,  158,  158,  111,  112,  114,
+      158,  158,  158,  158,  158,  158,  158,  158,  123,  158,
+      158,  126,  127,  141,  142,    8,  158,  158,  158,  158,
+      158,  158,  158,   26,  158,  158,  158,  158,  158,  158,
+      158,  158,  158,  158,  158,  158,  158,  158,  158,  158,
+       49,   50,   51,  158,  158,   57,  158,   62,   63,  158,
+
+      158,  158,   76,  158,   79,   82,   86,   87,  158,  158,
+      158,  158,  158,   96,  158,  158,  100,  158,  158,  158,
+      158,  158,  110,  158,  158,  158,  117,  158,  158,  120,
+      158,  158,  124,  158,  158,  158,  158,   14,  158,  158,
+      158,  158,  158,   27,  158,   30,  158,  158,  158,  158,
+      158,   37,  158,  158,  158,   42,  158,   47,  158,  158,
+      158,   59,   64,  158,  158,  158,   78,  158,  158,  158,
+      158,  158,  158,   99,  158,  104,  105,  158,  158,  158,
+      158,  158,  118,  119,  121,  158,  125,  158,  158,   13,
+      158,  158,  158,  158,  158,  158,   22,   31,  158,   35,
+
+       36,  158,  158,  158,  158,   48,  158,   54,   66,  158,
+      158,  158,   91,  158,   93,  158,  158,  158,  158,  158,
+      158,  158,  158,  122,  158,  158,  158,  158,  158,  158,
+      158,  158,   33,  158,  158,   41,  158,  158,  158,   68,
+      158,  158,   94,  158,  158,  108,  158,  158,  158,  158,
+      158,   12,  158,  158,  158,  158,   25,  158,   38,  158,
+      158,   52,   67,   89,   92,  158,  158,  109,  113,  158,
+      116,  128,   16,  158,  158,  158,   28,   40,  158,   90,
+       98,  158,  158,  158,   18,   19,  158,  158,  115,  158,
+      158,  158,  158,  158,  102,  158,   46,   17,    0
 
     } ;
 
-static const YY_CHAR yy_ec[256] =
+static yyconst 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,
@@ -541,7 +492,7 @@ static const YY_CHAR yy_ec[256] =
         1,    1,    1,    1,    1
     } ;
 
-static const YY_CHAR yy_meta[72] =
+static yyconst 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,
@@ -553,157 +504,157 @@ static const YY_CHAR yy_meta[72] =
         8
     } ;
 
-static const flex_int16_t yy_base[614] =
+static yyconst flex_uint16_t yy_base[615] =
     {   0,
         0,    1,   46,    0,  117,  162,    2,    3,  127,  128,
-        6,   10,  159, 1324, 1324,    0, 1324,   13, 1324,  138,
-     1324, 1324, 1324,  142,    6,  129,    4, 1324,  134,  124,
+        6,   10,  159, 1325, 1325,    0, 1325,   13, 1325,  138,
+     1325, 1325, 1325,  142,    6,  129,    4, 1325,  134,  124,
       159,  213,  165,  167,  263,   92,  158,  163,   96,  107,
       218,  160,  186,  223,  221,  155,  281,  264,  279,  275,
-      186,  209,    0,  191, 1324,  147,    4,   19,    0,    0,
+      186,  209,    0,  191, 1325,  147,    4,   19,    0,    0,
         0,  140,    0,    0,  343,    0,    0,  141,    0,   22,
-     1324,    0,  307,  326,  338,   18, 1324, 1324, 1324,    0,
+     1325,    0,  307,  326,  338,   18, 1325, 1325, 1325,    0,
       220,  315,  234,  239,  234,  288,  261,    0,  280,  335,
-      330,  342,  346,  332,  333,  382,  332,  331,  339, 1324,
+      330,  342,  346,  332,  333,  382,  332,  331,  339, 1325,
 
       336,  358,  363,  380,  356,  352,  378,  386,  387,  391,
       390,  387,  386,  406,    0,  399,  386,  393,  407,  408,
       401,  402,  410,  408,  431,  440,    0,  443,  431,  447,
       438,  440,  454,  451,  447,  461,  457,  442,  466,  465,
       468,  469,  468,  461,  456,  463,  502,  479,  475,  498,
-      496,  500,  504,  144,   29,    0,    0, 1324,    0, 1324,
-     1324,   22,   24, 1324, 1324, 1324, 1324, 1324,    0,    0,
-     1324,    0,  522,   26,   28,    0,    0,  504,    0,  514,
-      498,  513,  500,  521,  520,  508,  525,  508,  512,  507,
-      534,  517,  523,  537,    0,  535,  545,  554,  544,  542,
-
-      565,  552,  564,    0,  571,  555,  559,  559,  560,  579,
-      569,  577,  570,  572,  564,  578,  578,  571,  585,  588,
-      589,  590,  585,    0,  580,  582,  599,  608,  613,  600,
-      605,    0,  614,    0,  622,  625,  614,  615,  633,  634,
-      625,  619,  634,  630,  638,  639,  637,  628,  633,  646,
-        0,  640,  634,  646,  646,  657,  658,  663,  663,  668,
-      678,  670,  665,  685,  676,  689,  687,  678,   30,  125,
-        0,  679,  684,  694,  686,  696,  691,  690,    0,  704,
-      695,  696,  690,    0,  691,  698,  713,  700,  708,  712,
-      716,  724,  736,  733,  731,    0,    0,  728,  726,  747,
-
-      746,  732,  733,  739,  746,    0,    0,  740,    0,  743,
-        0,  734,  741,  742,  756,    0,    0,    0,    0,    0,
-      741,    0,  744,  759,  749,  751,  758,    0,  769,  774,
-      786,  793,  777,    0,  794,    0,  782,  777,  782,    0,
-      800,  792,  806,  798,  791,  807,    0,  794,    0,  809,
-      794,  802,  796,  798,  812,  816,  814,    0,  820,  811,
-        0,  814,  131, 1324,    0,  828,  829,  816,  836,  833,
-      847,  854,    0,  844,  841,  855,  857,  850,  858,  867,
-      857,  865,  862,  858,  859,  869,  870,  857,  874,    0,
-        0,    0,  856,  873,    0,  876,    0,    0,  864,  880,
-
-      873,    0,  885,    0,    0,    0,    0,  874,  893,  906,
-      893,  906,    0,  911,  901,    0,  913,  916,  902,  916,
-      908,    0,  907,  909,  915,    0,  925,  926,    0,  913,
-      932,    0,  911,  920,  928,  924,    0,  919,  924,  942,
-      939,  930,    0,  951,    0,  948,  945,  955,  957,  953,
-        0,  954,  971,  973,    0,   93,    0,  958,  971,  969,
-        0,    0,  966,  980,  985,    0,  978,  968,  980,  965,
-      971,  978,    0,  981,    0,    0,  981,  987,  998,  999,
-      997,    0,    0,    0,  988,    0,  994,  996,    0, 1002,
-     1008, 1013, 1015, 1026, 1023,    0,    0, 1028,    0,    0,
-
-     1029, 1027, 1018, 1022,    0, 1028,    0,    0, 1040, 1032,
-     1030,    0, 1032,    0, 1022, 1045, 1040, 1030, 1037, 1040,
-     1041, 1052,    0, 1038, 1052, 1050, 1050, 1052, 1049, 1063,
-     1070,    0, 1067, 1075,    0, 1070, 1084, 1072,    0, 1080,
-     1089,    0, 1094, 1087,    0, 1094, 1086, 1087, 1100, 1097,
-        0, 1099, 1102, 1096, 1104,    0, 1091,    0, 1105, 1097,
-        0,    0, 1097,    0, 1098, 1113,    0,    0, 1112,    0,
-        0,    0, 1108, 1122, 1121,    0,    0, 1133,    0,    0,
-     1123, 1142, 1128,    0,    0, 1135, 1145,    0, 1143, 1147,
-     1139, 1153, 1140,    0, 1141,    0,    0, 1324, 1206, 1216,
-
-     1226, 1236, 1246, 1250, 1253, 1259, 1269, 1279, 1289, 1299,
-     1309, 1314, 1316
+      496,  500,  504,  144,   29,    0,    0, 1325,    0, 1325,
+     1325,   22,   24, 1325, 1325, 1325, 1325, 1325,    0,    0,
+     1325,    0,  522,   26,   28,    0,    0,  504,    0,  514,
+      498,  513,  500,  521,  521,  507,  525,  508,  512,  507,
+      534,  517,  524,  539,    0,  537,  546,  554,  559,  547,
+
+      566,  553,  567,    0,  573,  558,  560,  560,  561,  580,
+      570,  578,  571,  573,  565,  579,  580,  572,  588,  589,
+      590,  612,  585,    0,  580,  582,  599,  596,  613,  600,
+      605,    0,  614,    0,  625,  627,  617,  616,  634,  635,
+      626,  620,  635,  631,  639,  640,  639,  629,  636,  647,
+        0,  641,  638,  648,  648,  658,  670,  666,  664,  668,
+      683,  673,  667,  687,  679,  690,  687,  678,   30,  125,
+        0,  680,  685,  695,  687,  697,  692,  692,    0,    0,
+      705,  698,  697,  691,    0,  695,  700,  715,  701,  720,
+      715,  717,  728,  737,  734,  733,    0,    0,  730,  728,
+
+      750,  747,  733,  734,  740,  747,    0,    0,  741,    0,
+      744,    0,  735,  742,  743,  760,    0,    0,    0,    0,
+        0,  743,    0,  744,  760,  750,  755,  758,    0,  770,
+      786,  789,  794,  781,    0,  795,    0,  783,  779,  784,
+        0,  802,  795,  807,  799,  792,  808,    0,  795,    0,
+      810,  795,  803,  797,  799,  814,  817,  817,    0,  821,
+      812,    0,  818,  131, 1325,    0,  830,  831,  817,  848,
+      842,  846,  850,    0,  843,  841,  857,  859,  853,  859,
+      868,  858,  866,  863,  859,  860,  870,  871,  858,  876,
+        0,    0,    0,  857,  876,    0,  877,    0,    0,  865,
+
+      884,  873,    0,  889,    0,    0,    0,    0,  887,  896,
+      907,  897,  907,    0,  912,  903,    0,  915,  918,  905,
+      917,  909,    0,  908,  910,  916,    0,  926,  927,    0,
+      914,  933,    0,  912,  922,  929,  927,    0,  920,  925,
+      946,  941,  932,    0,  952,    0,  960,  948,  956,  961,
+      954,    0,  955,  973,  975,    0,   93,    0,  960,  974,
+      970,    0,    0,  967,  981,  986,    0,  979,  969,  981,
+      966,  972,  979,    0,  983,    0,    0,  982,  990,  999,
+     1000, 1001,    0,    0,    0,  990,    0,  996,  997,    0,
+     1014, 1011, 1014, 1019, 1027, 1024,    0,    0, 1030,    0,
+
+        0, 1031, 1029, 1021, 1023,    0, 1029,    0,    0, 1041,
+     1033, 1031,    0, 1033,    0, 1023, 1046, 1041, 1031, 1039,
+     1041, 1044, 1053,    0, 1039, 1056, 1052, 1052, 1053, 1061,
+     1066, 1071,    0, 1071, 1076,    0, 1071, 1086, 1074,    0,
+     1082, 1092,    0, 1095, 1088,    0, 1095, 1087, 1088, 1101,
+     1098,    0, 1100, 1103, 1097, 1106,    0, 1092,    0, 1108,
+     1098,    0,    0, 1098,    0, 1102, 1115,    0,    0, 1114,
+        0,    0,    0, 1109, 1134, 1124,    0,    0, 1134,    0,
+        0, 1127, 1143, 1129,    0,    0, 1137, 1147,    0, 1145,
+     1150, 1140, 1154, 1141,    0, 1142,    0,    0, 1325, 1207,
+
+     1217, 1227, 1237, 1247, 1251, 1254, 1260, 1270, 1280, 1290,
+     1300, 1310, 1315, 1317
     } ;
 
-static const flex_int16_t yy_def[614] =
+static yyconst flex_int16_t yy_def[615] =
     {   0,
-      599,  599,  598,    3,  600,  600,  601,  601,  602,  602,
-      603,  603,  598,  598,  598,  604,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  598,  598,  598,  598,  606,  607,
-      607,  598,  608,  608,  609,  610,  610,  598,  604,  598,
-      598,  611,  598,  598,  598,  598,  598,  598,  598,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  598,
-
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  598,  598,  606,  607,  598,  608,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  612,  610,
-      598,  611,  598,  598,  598,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  598,  598,
-      613,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  598,  598,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,  605,  605,  605,
-      605,  605,  605,  605,  605,  605,  605,    0,  598,  598,
-
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598
+      600,  600,  599,    3,  601,  601,  602,  602,  603,  603,
+      604,  604,  599,  599,  599,  605,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  599,  599,  599,  599,  607,  608,
+      608,  599,  609,  609,  610,  611,  611,  599,  605,  599,
+      599,  612,  599,  599,  599,  599,  599,  599,  599,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  599,
+
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  599,  599,  607,  608,  599,  609,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  613,  611,
+      599,  612,  599,  599,  599,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  599,  599,
+      614,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  599,  599,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,  606,  606,
+      606,  606,  606,  606,  606,  606,  606,  606,    0,  599,
+
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599
     } ;
 
-static const flex_int16_t yy_nxt[1396] =
+static yyconst flex_uint16_t yy_nxt[1397] =
     {   0,
-      598,  598,   15,   15,   61,   61,  155,  155,   67,   62,
-       62,   68,   67,  598,   70,   68,   70,   73,   73,   77,
-       78,  155,  155,   70,  598,   70,  174,  174,  598,  175,
+      599,  599,   15,   15,   61,   61,  155,  155,   67,   62,
+       62,   68,   67,  599,   70,   68,   70,   73,   73,   77,
+       78,  155,  155,   70,  599,   70,  174,  174,  599,  175,
       175,  155,  155,  269,  270,  270,  270,  175,  175,  175,
-      175,  363,  270,  598,   16,   16,   17,   18,   19,   18,
+      175,  364,  270,  599,   16,   16,   17,   18,   19,   18,
        20,   21,   22,   23,   22,   24,   25,   26,   26,   17,
        27,   28,   29,   30,   31,   32,   33,   34,   35,   36,
        37,   38,   39,   40,   41,   42,   43,   44,   45,   46,
@@ -713,33 +664,33 @@ static const flex_int16_t yy_nxt[1396] =
        38,   39,   40,   41,   42,   43,   44,   45,   46,   47,
        48,   49,   50,   51,   52,   53,   54,   17,   56,   57,
        58,   17,   17,   17,   17,   17,  111,  116,  117,   64,
-       64,   17,   17,   17,   62,   62,  270,  270,  504,   74,
+       64,   17,   17,   17,   62,   62,  270,  270,  505,   74,
        75,   75,  270,  270,   81,  154,  171,  158,  154,   79,
-       76,   72,   82,   71,   83,  111,  116,  117,  598,   84,
+       76,   72,   82,   71,   83,  111,  116,  117,  599,   84,
        17,   17,   17,   56,   57,   58,   17,   17,   17,   17,
        17,   65,   65,   81,  100,  112,   17,   17,   17,   76,
        85,   82,   95,   83,   86,  122,   96,   87,   84,  113,
-       97,  123,  134,  114,  598,  101,   98,  102,  115,   99,
+       97,  123,  134,  114,  599,  101,   98,  102,  115,   99,
 
-       88,  598,  103,  150,  112,   17,   17,  104,  598,   85,
-      598,   95,  153,   86,  122,   96,   87,  124,  113,   97,
+       88,  599,  103,  150,  112,   17,   17,  104,  599,   85,
+      599,   95,  153,   86,  122,   96,   87,  124,  113,   97,
       123,  134,  114,  125,  101,   98,  102,  115,   99,   88,
        89,  103,  150,  151,  152,  118,  104,   90,  131,  119,
       176,  153,  132,  120,   91,  126,  124,   92,   93,  121,
-       94,  598,  125,  127,  179,  133,  598,  128,  180,   89,
-      129,  130,  151,  152,  118,  598,   90,  131,  119,  176,
-      181,  132,  120,   91,  126,  598,   92,   93,  121,   94,
-      105,  598,  127,  179,  133,  139,  128,  180,  106,  129,
-      130,  107,  184,  140,  108,  598,  142,  109,  135,  181,
+       94,  599,  125,  127,  179,  133,  599,  128,  180,   89,
+      129,  130,  151,  152,  118,  599,   90,  131,  119,  176,
+      181,  132,  120,   91,  126,  599,   92,   93,  121,   94,
+      105,  599,  127,  179,  133,  139,  128,  180,  106,  129,
+      130,  107,  184,  140,  108,  599,  142,  109,  135,  181,
 
-      110,  141,  136,  143,  144,  147,  137,  148,  598,  105,
+      110,  141,  136,  143,  144,  147,  137,  148,  599,  105,
       149,  182,  138,  145,  139,  185,  146,  106,   73,   73,
       107,  184,  140,  108,  183,  142,  109,  135,   76,  110,
       141,  136,  143,  144,  147,  137,  148,  173,  173,  149,
       182,  138,  145,  177,  185,  146,  161,   76,   74,   75,
        75,  178,  186,  183,  162,  163,  187,   76,  188,   76,
       189,  164,  190,  191,  192,  165,  193,  200,  201,  194,
-      202,  203,  177,  166,  195,  598,   76,  167,  204,  168,
+      202,  203,  177,  166,  195,  599,   76,  167,  204,  168,
       178,  186,  205,  169,  208,  187,  209,  188,   76,  189,
       164,  190,  191,  192,  165,  193,  200,  201,  194,  202,
 
@@ -752,110 +703,110 @@ static const flex_int16_t yy_nxt[1396] =
       229,  230,  234,  235,  227,  220,  228,  236,  237,  231,
       221,  222,  238,  232,  239,  240,  242,  243,  244,  241,
       247,  248,  252,  245,  246,  249,  253,  254,  233,  255,
-      256,  234,  235,  257,  250,  258,  236,  237,  598,  261,
+      256,  234,  235,  257,  250,  258,  236,  237,  599,  261,
 
       262,  238,  251,  239,  240,  242,  243,  244,  241,  247,
       248,  252,  245,  246,  249,  253,  254,  265,  255,  256,
       259,  268,  257,  250,  258,  272,  263,  260,  261,  262,
       266,  251,  264,  173,  173,  273,  267,  274,  275,  276,
-      277,  279,  280,   76,  281,  282,  265,  283,  284,  259,
-      268,  285,  278,  286,  272,  263,  260,  287,  288,  266,
-      289,  264,  290,  293,  273,  267,  274,  275,  276,  277,
-      279,  280,   76,  281,  282,  291,  283,  284,  294,  292,
-      285,  278,  286,  295,  296,  297,  287,  288,  298,  289,
-      299,  290,  293,  300,  301,  302,  303,  304,  305,  306,
-
-      307,  308,  309,  310,  291,  311,  312,  294,  292,  313,
-      314,  315,  295,  296,  297,  317,  318,  298,  319,  299,
-      320,  316,  300,  301,  302,  303,  304,  305,  306,  307,
-      308,  309,  310,  321,  311,  312,  322,  323,  313,  314,
-      315,  324,  325,  326,  317,  318,  327,  319,  328,  320,
-      316,  329,  330,  331,  332,  334,  333,  335,  336,  337,
-      338,  339,  321,  340,  342,  322,  323,  343,  344,  345,
-      324,  325,  326,  341,  346,  327,  347,  328,  348,  349,
-      329,  330,  331,  332,  334,  333,  335,  336,  337,  338,
-      339,  350,  340,  342,  351,  354,  343,  344,  345,  352,
-
-      355,  353,  356,  346,  357,  347,  358,  348,  349,  360,
-      359,  361,  362,  365,  366,  367,  368,  369,  370,  371,
-      350,  372,  373,  351,  354,  374,  375,  376,  352,  355,
-      353,  356,  377,  357,  378,  358,  379,  380,  360,  359,
-      361,  362,  365,  366,  367,  368,  369,  370,  371,  381,
+      277,  281,  279,   76,  282,  283,  265,  284,  285,  259,
+      268,  286,  278,  287,  272,  263,  260,  280,  288,  266,
+      289,  264,  290,  291,  273,  267,  274,  275,  276,  277,
+      281,  279,   76,  282,  283,  292,  284,  285,  294,  293,
+      286,  278,  287,  295,  296,  297,  280,  288,  298,  289,
+      299,  290,  291,  300,  301,  302,  303,  304,  305,  306,
+
+      307,  308,  309,  310,  292,  311,  312,  294,  293,  313,
+      314,  315,  295,  296,  297,  318,  319,  298,  320,  299,
+      321,  322,  300,  301,  302,  303,  304,  305,  306,  307,
+      308,  309,  310,  316,  311,  312,  323,  324,  313,  314,
+      315,  325,  326,  317,  318,  319,  327,  320,  328,  321,
+      322,  329,  330,  331,  332,  333,  335,  334,  336,  337,
+      338,  339,  316,  340,  341,  323,  324,  343,  344,  345,
+      325,  326,  317,  346,  342,  327,  347,  328,  348,  349,
+      329,  330,  331,  332,  333,  335,  334,  336,  337,  338,
+      339,  350,  340,  341,  351,  352,  343,  344,  345,  353,
+
+      355,  354,  346,  356,  357,  347,  358,  348,  349,  359,
+      361,  362,  363,  360,  366,  367,  368,  369,  370,  371,
+      350,  372,  373,  351,  352,  374,  375,  376,  353,  355,
+      354,  377,  356,  357,  378,  358,  379,  380,  359,  361,
+      362,  363,  360,  366,  367,  368,  369,  370,  371,  381,
       372,  373,  382,  383,  374,  375,  376,  384,  385,  386,
-      387,  377,  388,  378,  389,  379,  380,  390,  391,  392,
-      393,  394,  395,  396,  397,  398,  399,  402,  381,  400,
-      403,  382,  383,  404,  405,  406,  384,  385,  386,  387,
-      401,  388,  407,  389,  408,  409,  390,  391,  392,  393,
-
-      394,  395,  396,  397,  398,  399,  402,  410,  400,  403,
-      411,  412,  404,  405,  406,  413,  414,  415,  416,  401,
-      417,  407,  418,  408,  409,  419,  420,  421,  422,  423,
+      377,  387,  388,  378,  389,  379,  380,  390,  391,  392,
+      393,  394,  395,  396,  397,  398,  399,  400,  381,  403,
+      404,  382,  383,  401,  405,  406,  384,  385,  386,  407,
+      387,  388,  408,  389,  402,  409,  390,  391,  392,  393,
+
+      394,  395,  396,  397,  398,  399,  400,  410,  403,  404,
+      411,  412,  401,  405,  406,  413,  414,  415,  407,  416,
+      417,  408,  418,  402,  409,  419,  420,  421,  422,  423,
       424,  425,  426,  427,  428,  429,  410,  430,  431,  411,
-      412,  432,  433,  434,  413,  414,  415,  416,  435,  417,
+      412,  432,  433,  434,  413,  414,  415,  435,  416,  417,
       436,  418,  437,  438,  419,  420,  421,  422,  423,  424,
-      425,  426,  427,  428,  429,  439,  430,  431,  440,  441,
-      432,  433,  434,  442,  443,  444,  445,  435,  446,  436,
+      425,  426,  427,  428,  429,  439,  430,  431,  442,  443,
+      432,  433,  434,  444,  440,  445,  435,  441,  446,  436,
       447,  437,  438,  448,  449,  450,  451,  452,  453,  454,
-      455,  456,  457,  458,  439,  459,  460,  440,  441,  461,
+      455,  456,  457,  458,  439,  459,  460,  442,  443,  461,
 
-      462,  463,  442,  443,  444,  445,  466,  446,  464,  447,
-      467,  465,  448,  449,  450,  451,  452,  453,  454,  455,
+      462,  463,  444,  440,  445,  464,  441,  446,  465,  447,
+      467,  466,  448,  449,  450,  451,  452,  453,  454,  455,
       456,  457,  458,  468,  459,  460,  469,  470,  461,  462,
-      463,  471,  472,  473,  474,  466,  475,  464,  476,  467,
-      465,  477,  478,  479,  480,  481,  482,  483,  484,  485,
+      463,  471,  472,  473,  464,  474,  475,  465,  476,  467,
+      466,  477,  478,  479,  480,  481,  482,  483,  484,  485,
       486,  487,  468,  488,  489,  469,  470,  490,  491,  492,
-      471,  472,  473,  474,  493,  475,  494,  476,  495,  496,
+      471,  472,  473,  493,  474,  475,  494,  476,  495,  496,
       477,  478,  479,  480,  481,  482,  483,  484,  485,  486,
       487,  497,  488,  489,  498,  499,  490,  491,  492,  500,
-      501,  502,  503,  493,  505,  494,  506,  495,  496,  507,
+      501,  502,  493,  503,  504,  494,  506,  495,  496,  507,
 
       508,  509,  510,  511,  512,  513,  514,  515,  516,  517,
       497,  518,  519,  498,  499,  520,  521,  522,  500,  501,
-      502,  503,  523,  505,  524,  506,  525,  526,  507,  508,
+      502,  523,  503,  504,  524,  506,  525,  526,  507,  508,
       509,  510,  511,  512,  513,  514,  515,  516,  517,  527,
       518,  519,  528,  529,  520,  521,  522,  530,  531,  532,
-      533,  523,  534,  524,  535,  525,  526,  536,  537,  538,
+      523,  533,  534,  524,  535,  525,  526,  536,  537,  538,
       539,  540,  541,  542,  543,  544,  545,  546,  527,  547,
-      548,  528,  529,  549,  550,  551,  530,  531,  532,  533,
-      552,  534,  553,  535,  554,  555,  536,  537,  538,  539,
+      548,  528,  529,  549,  550,  551,  530,  531,  532,  552,
+      533,  534,  553,  535,  554,  555,  536,  537,  538,  539,
       540,  541,  542,  543,  544,  545,  546,  556,  547,  548,
 
-      557,  558,  549,  550,  551,  559,  560,  561,  562,  552,
+      557,  558,  549,  550,  551,  559,  560,  561,  552,  562,
       563,  553,  564,  554,  555,  565,  566,  567,  568,  569,
       570,  571,  572,  573,  574,  575,  556,  576,  577,  557,
-      558,  578,  579,  580,  559,  560,  561,  562,  581,  563,
+      558,  578,  579,  580,  559,  560,  561,  581,  562,  563,
       582,  564,  583,  584,  565,  566,  567,  568,  569,  570,
       571,  572,  573,  574,  575,  585,  576,  577,  586,  587,
-      578,  579,  580,  588,  589,  590,  591,  581,  592,  582,
-      593,  583,  584,  594,  595,  596,  597,  598,  598,  598,
-      598,  598,  598,  598,  585,  598,  598,  586,  587,  598,
-      598,  598,  588,  589,  590,  591,  598,  592,  598,  593,
-
-      598,  598,  594,  595,  596,  597,   14,   14,   14,   14,
-       14,   14,   14,   14,   14,   14,   59,   59,   59,   59,
-       59,   59,   59,   59,   59,   59,   60,   60,   60,   60,
-       60,   60,   60,   60,   60,   60,   63,   63,   63,   63,
-       63,   63,   63,   63,   63,   63,   66,   66,   66,   66,
-       66,   66,   66,   66,   66,   66,   69,   69,   80,   80,
-       80,  598,   80,  156,  156,  156,  156,  598,  156,  157,
-      157,  157,  598,  157,  157,  157,  157,  157,  157,  159,
-      159,  159,  598,  159,  159,  159,  159,  598,  159,  160,
-      160,  160,  160,  160,  160,  160,  160,  160,  160,  170,
-
-      170,  598,  170,  170,  170,  170,  170,  170,  170,  172,
-      598,  172,  172,  172,  172,  172,  172,  172,  172,  271,
-      271,  364,  364,   13,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598
+      578,  579,  580,  588,  589,  590,  581,  591,  592,  582,
+      593,  583,  584,  594,  595,  596,  597,  598,  599,  599,
+      599,  599,  599,  599,  585,  599,  599,  586,  587,  599,
+      599,  599,  588,  589,  590,  599,  591,  592,  599,  593,
+
+      599,  599,  594,  595,  596,  597,  598,   14,   14,   14,
+       14,   14,   14,   14,   14,   14,   14,   59,   59,   59,
+       59,   59,   59,   59,   59,   59,   59,   60,   60,   60,
+       60,   60,   60,   60,   60,   60,   60,   63,   63,   63,
+       63,   63,   63,   63,   63,   63,   63,   66,   66,   66,
+       66,   66,   66,   66,   66,   66,   66,   69,   69,   80,
+       80,   80,  599,   80,  156,  156,  156,  156,  599,  156,
+      157,  157,  157,  599,  157,  157,  157,  157,  157,  157,
+      159,  159,  159,  599,  159,  159,  159,  159,  599,  159,
+      160,  160,  160,  160,  160,  160,  160,  160,  160,  160,
+
+      170,  170,  599,  170,  170,  170,  170,  170,  170,  170,
+      172,  599,  172,  172,  172,  172,  172,  172,  172,  172,
+      271,  271,  365,  365,   13,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599
 
     } ;
 
-static const flex_int16_t yy_chk[1396] =
+static yyconst flex_int16_t yy_chk[1397] =
     {   0,
         0,    0,    1,    2,    7,    8,   57,   57,   11,    7,
         8,   11,   12,    0,   18,   12,   18,   25,   25,   27,
@@ -871,8 +822,8 @@ static const flex_int16_t yy_chk[1396] =
         3,    3,    3,    3,    3,    3,    3,    3,    3,    3,
         3,    3,    3,    3,    3,    3,    3,    5,    5,    5,
         5,    5,    5,    5,    5,    5,   36,   39,   40,    9,
-       10,    5,    5,    5,    9,   10,  270,  270,  456,   26,
-       26,   26,  363,  363,   30,  154,   68,   62,   56,   29,
+       10,    5,    5,    5,    9,   10,  270,  270,  457,   26,
+       26,   26,  364,  364,   30,  154,   68,   62,   56,   29,
        26,   24,   30,   20,   30,   36,   39,   40,   13,   30,
         5,    5,    6,    6,    6,    6,    6,    6,    6,    6,
         6,    9,   10,   30,   34,   37,    6,    6,    6,   26,
@@ -916,105 +867,105 @@ static const flex_int16_t yy_chk[1396] =
       138,  140,  136,  136,  139,  141,  142,  151,  143,  144,
       147,  153,  145,  139,  146,  178,  150,  147,  148,  149,
       152,  139,  150,  173,  173,  180,  152,  181,  182,  183,
-      184,  185,  186,  173,  187,  188,  151,  189,  190,  147,
-      153,  191,  184,  192,  178,  150,  147,  193,  194,  152,
-      196,  150,  197,  199,  180,  152,  181,  182,  183,  184,
-      185,  186,  173,  187,  188,  198,  189,  190,  200,  198,
-      191,  184,  192,  201,  202,  203,  193,  194,  205,  196,
-      206,  197,  199,  207,  208,  209,  210,  211,  212,  213,
-
-      214,  215,  216,  217,  198,  218,  219,  200,  198,  220,
-      221,  222,  201,  202,  203,  223,  225,  205,  226,  206,
-      227,  222,  207,  208,  209,  210,  211,  212,  213,  214,
-      215,  216,  217,  228,  218,  219,  229,  230,  220,  221,
-      222,  231,  233,  235,  223,  225,  236,  226,  237,  227,
-      222,  238,  239,  240,  241,  242,  241,  243,  244,  245,
-      246,  247,  228,  248,  249,  229,  230,  250,  252,  253,
-      231,  233,  235,  248,  254,  236,  255,  237,  256,  257,
-      238,  239,  240,  241,  242,  241,  243,  244,  245,  246,
-      247,  258,  248,  249,  259,  261,  250,  252,  253,  260,
-
-      262,  260,  263,  254,  264,  255,  265,  256,  257,  266,
-      265,  267,  268,  272,  273,  274,  275,  276,  277,  278,
-      258,  280,  281,  259,  261,  282,  283,  285,  260,  262,
-      260,  263,  286,  264,  287,  265,  288,  289,  266,  265,
-      267,  268,  272,  273,  274,  275,  276,  277,  278,  290,
-      280,  281,  291,  292,  282,  283,  285,  293,  294,  295,
-      298,  286,  299,  287,  300,  288,  289,  301,  302,  303,
-      304,  305,  308,  310,  312,  313,  314,  321,  290,  315,
-      323,  291,  292,  324,  325,  326,  293,  294,  295,  298,
-      315,  299,  327,  300,  329,  330,  301,  302,  303,  304,
-
-      305,  308,  310,  312,  313,  314,  321,  331,  315,  323,
-      332,  333,  324,  325,  326,  335,  337,  338,  339,  315,
-      341,  327,  342,  329,  330,  343,  344,  345,  346,  348,
-      350,  351,  352,  353,  354,  355,  331,  356,  357,  332,
-      333,  359,  360,  362,  335,  337,  338,  339,  366,  341,
-      367,  342,  368,  369,  343,  344,  345,  346,  348,  350,
-      351,  352,  353,  354,  355,  370,  356,  357,  370,  371,
-      359,  360,  362,  372,  374,  375,  376,  366,  377,  367,
+      184,  186,  185,  173,  187,  188,  151,  189,  190,  147,
+      153,  191,  184,  192,  178,  150,  147,  185,  193,  152,
+      194,  150,  196,  197,  180,  152,  181,  182,  183,  184,
+      186,  185,  173,  187,  188,  198,  189,  190,  199,  198,
+      191,  184,  192,  200,  201,  202,  185,  193,  203,  194,
+      205,  196,  197,  206,  207,  208,  209,  210,  211,  212,
+
+      213,  214,  215,  216,  198,  217,  218,  199,  198,  219,
+      220,  221,  200,  201,  202,  223,  225,  203,  226,  205,
+      227,  228,  206,  207,  208,  209,  210,  211,  212,  213,
+      214,  215,  216,  222,  217,  218,  229,  230,  219,  220,
+      221,  231,  233,  222,  223,  225,  235,  226,  236,  227,
+      228,  237,  238,  239,  240,  241,  242,  241,  243,  244,
+      245,  246,  222,  247,  248,  229,  230,  249,  250,  252,
+      231,  233,  222,  253,  248,  235,  254,  236,  255,  256,
+      237,  238,  239,  240,  241,  242,  241,  243,  244,  245,
+      246,  257,  247,  248,  258,  259,  249,  250,  252,  260,
+
+      261,  260,  253,  262,  263,  254,  264,  255,  256,  265,
+      266,  267,  268,  265,  272,  273,  274,  275,  276,  277,
+      257,  278,  281,  258,  259,  282,  283,  284,  260,  261,
+      260,  286,  262,  263,  287,  264,  288,  289,  265,  266,
+      267,  268,  265,  272,  273,  274,  275,  276,  277,  290,
+      278,  281,  291,  292,  282,  283,  284,  293,  294,  295,
+      286,  296,  299,  287,  300,  288,  289,  301,  302,  303,
+      304,  305,  306,  309,  311,  313,  314,  315,  290,  322,
+      324,  291,  292,  316,  325,  326,  293,  294,  295,  327,
+      296,  299,  328,  300,  316,  330,  301,  302,  303,  304,
+
+      305,  306,  309,  311,  313,  314,  315,  331,  322,  324,
+      332,  333,  316,  325,  326,  334,  336,  338,  327,  339,
+      340,  328,  342,  316,  330,  343,  344,  345,  346,  347,
+      349,  351,  352,  353,  354,  355,  331,  356,  357,  332,
+      333,  358,  360,  361,  334,  336,  338,  363,  339,  340,
+      367,  342,  368,  369,  343,  344,  345,  346,  347,  349,
+      351,  352,  353,  354,  355,  370,  356,  357,  372,  373,
+      358,  360,  361,  375,  371,  376,  363,  371,  377,  367,
       378,  368,  369,  379,  380,  381,  382,  383,  384,  385,
-      386,  387,  388,  389,  370,  393,  394,  370,  371,  396,
-
-      399,  400,  372,  374,  375,  376,  403,  377,  401,  378,
-      408,  401,  379,  380,  381,  382,  383,  384,  385,  386,
-      387,  388,  389,  409,  393,  394,  410,  411,  396,  399,
-      400,  412,  414,  415,  417,  403,  418,  401,  419,  408,
-      401,  420,  421,  423,  424,  425,  427,  428,  430,  431,
-      433,  434,  409,  435,  436,  410,  411,  438,  439,  440,
-      412,  414,  415,  417,  441,  418,  442,  419,  444,  446,
-      420,  421,  423,  424,  425,  427,  428,  430,  431,  433,
-      434,  447,  435,  436,  448,  449,  438,  439,  440,  450,
-      452,  453,  454,  441,  458,  442,  459,  444,  446,  460,
-
-      463,  464,  465,  467,  468,  469,  470,  471,  472,  474,
-      447,  477,  478,  448,  449,  479,  480,  481,  450,  452,
-      453,  454,  485,  458,  487,  459,  488,  490,  460,  463,
-      464,  465,  467,  468,  469,  470,  471,  472,  474,  491,
-      477,  478,  492,  493,  479,  480,  481,  494,  495,  498,
-      501,  485,  502,  487,  503,  488,  490,  504,  506,  509,
-      510,  511,  513,  515,  516,  517,  518,  519,  491,  520,
-      521,  492,  493,  522,  524,  525,  494,  495,  498,  501,
-      526,  502,  527,  503,  528,  529,  504,  506,  509,  510,
-      511,  513,  515,  516,  517,  518,  519,  530,  520,  521,
-
-      531,  533,  522,  524,  525,  534,  536,  537,  538,  526,
-      540,  527,  541,  528,  529,  543,  544,  546,  547,  548,
-      549,  550,  552,  553,  554,  555,  530,  557,  559,  531,
-      533,  560,  563,  565,  534,  536,  537,  538,  566,  540,
-      569,  541,  573,  574,  543,  544,  546,  547,  548,  549,
-      550,  552,  553,  554,  555,  575,  557,  559,  578,  581,
-      560,  563,  565,  582,  583,  586,  587,  566,  589,  569,
-      590,  573,  574,  591,  592,  593,  595,    0,    0,    0,
-        0,    0,    0,    0,  575,    0,    0,  578,  581,    0,
-        0,    0,  582,  583,  586,  587,    0,  589,    0,  590,
-
-        0,    0,  591,  592,  593,  595,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  600,  600,  600,  600,
-      600,  600,  600,  600,  600,  600,  601,  601,  601,  601,
-      601,  601,  601,  601,  601,  601,  602,  602,  602,  602,
-      602,  602,  602,  602,  602,  602,  603,  603,  603,  603,
-      603,  603,  603,  603,  603,  603,  604,  604,  605,  605,
-      605,    0,  605,  606,  606,  606,  606,    0,  606,  607,
-      607,  607,    0,  607,  607,  607,  607,  607,  607,  608,
-      608,  608,    0,  608,  608,  608,  608,    0,  608,  609,
-      609,  609,  609,  609,  609,  609,  609,  609,  609,  610,
-
-      610,    0,  610,  610,  610,  610,  610,  610,  610,  611,
-        0,  611,  611,  611,  611,  611,  611,  611,  611,  612,
-      612,  613,  613,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598,  598,  598,  598,  598,  598,
-      598,  598,  598,  598,  598
+      386,  387,  388,  389,  370,  390,  394,  372,  373,  395,
+
+      397,  400,  375,  371,  376,  401,  371,  377,  402,  378,
+      404,  402,  379,  380,  381,  382,  383,  384,  385,  386,
+      387,  388,  389,  409,  390,  394,  410,  411,  395,  397,
+      400,  412,  413,  415,  401,  416,  418,  402,  419,  404,
+      402,  420,  421,  422,  424,  425,  426,  428,  429,  431,
+      432,  434,  409,  435,  436,  410,  411,  437,  439,  440,
+      412,  413,  415,  441,  416,  418,  442,  419,  443,  445,
+      420,  421,  422,  424,  425,  426,  428,  429,  431,  432,
+      434,  447,  435,  436,  448,  449,  437,  439,  440,  450,
+      451,  453,  441,  454,  455,  442,  459,  443,  445,  460,
+
+      461,  464,  465,  466,  468,  469,  470,  471,  472,  473,
+      447,  475,  478,  448,  449,  479,  480,  481,  450,  451,
+      453,  482,  454,  455,  486,  459,  488,  489,  460,  461,
+      464,  465,  466,  468,  469,  470,  471,  472,  473,  491,
+      475,  478,  492,  493,  479,  480,  481,  494,  495,  496,
+      482,  499,  502,  486,  503,  488,  489,  504,  505,  507,
+      510,  511,  512,  514,  516,  517,  518,  519,  491,  520,
+      521,  492,  493,  522,  523,  525,  494,  495,  496,  526,
+      499,  502,  527,  503,  528,  529,  504,  505,  507,  510,
+      511,  512,  514,  516,  517,  518,  519,  530,  520,  521,
+
+      531,  532,  522,  523,  525,  534,  535,  537,  526,  538,
+      539,  527,  541,  528,  529,  542,  544,  545,  547,  548,
+      549,  550,  551,  553,  554,  555,  530,  556,  558,  531,
+      532,  560,  561,  564,  534,  535,  537,  566,  538,  539,
+      567,  541,  570,  574,  542,  544,  545,  547,  548,  549,
+      550,  551,  553,  554,  555,  575,  556,  558,  576,  579,
+      560,  561,  564,  582,  583,  584,  566,  587,  588,  567,
+      590,  570,  574,  591,  592,  593,  594,  596,    0,    0,
+        0,    0,    0,    0,  575,    0,    0,  576,  579,    0,
+        0,    0,  582,  583,  584,    0,  587,  588,    0,  590,
+
+        0,    0,  591,  592,  593,  594,  596,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  601,  601,  601,
+      601,  601,  601,  601,  601,  601,  601,  602,  602,  602,
+      602,  602,  602,  602,  602,  602,  602,  603,  603,  603,
+      603,  603,  603,  603,  603,  603,  603,  604,  604,  604,
+      604,  604,  604,  604,  604,  604,  604,  605,  605,  606,
+      606,  606,    0,  606,  607,  607,  607,  607,    0,  607,
+      608,  608,  608,    0,  608,  608,  608,  608,  608,  608,
+      609,  609,  609,    0,  609,  609,  609,  609,    0,  609,
+      610,  610,  610,  610,  610,  610,  610,  610,  610,  610,
+
+      611,  611,    0,  611,  611,  611,  611,  611,  611,  611,
+      612,    0,  612,  612,  612,  612,  612,  612,  612,  612,
+      613,  613,  614,  614,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
+      599,  599,  599,  599,  599,  599
 
     } ;
 
 /* Table of booleans, true if rule could match eol. */
-static const flex_int32_t yy_rule_can_match_eol[164] =
+static yyconst flex_int32_t yy_rule_can_match_eol[165] =
     {   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, 
@@ -1023,8 +974,8 @@ static const flex_int32_t yy_rule_can_match_eol[164] =
     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, 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, 1, 0, 0, 0, 0, 0, 1, 1, 1, 0, 0, 0, 
-    1, 0, 0, 0,     };
+    0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 0, 1, 1, 1, 0, 0, 
+    0, 1, 0, 0, 0,     };
 
 /* The intent behind this definition is that it'll catch
  * any uses of REJECT which flex missed.
@@ -1135,14 +1086,17 @@ class UnaryOperation;
     yycolumn += yyleng;                                   \
   }
 
-#line 1138 "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 1145 "SqlLexer_gen.cpp"
+
+
+
+
+#line 1100 "SqlLexer_gen.cpp"
 
 #define INITIAL 0
 #define CONDITION_SQL 1
@@ -1176,8 +1130,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;
-    int yy_n_chars;
-    int yyleng_r;
+    yy_size_t yy_n_chars;
+    yy_size_t yyleng_r;
     char *yy_c_buf_p;
     int yy_init;
     int yy_start;
@@ -1201,7 +1155,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.*/
@@ -1211,48 +1165,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 );
 
-			int quickstep_yyget_leng ( yyscan_t yyscanner );
+yy_size_t 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.
@@ -1260,9 +1214,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
 
@@ -1271,18 +1225,19 @@ extern int quickstep_yywrap ( yyscan_t yyscanner );
 #endif
 
 #ifndef yytext_ptr
-static void yy_flex_strncpy ( char *, const char *, int , yyscan_t yyscanner);
+static void yy_flex_strncpy (char *,yyconst char *,int ,yyscan_t yyscanner);
 #endif
 
 #ifdef YY_NEED_STRLEN
-static int yy_flex_strlen ( const char * , yyscan_t yyscanner);
+static int yy_flex_strlen (yyconst 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
@@ -1302,7 +1257,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, (size_t) yyleng, 1, yyout )) {} } while (0)
+#define ECHO do { if (fwrite( yytext, yyleng, 1, yyout )) {} } while (0)
 #endif
 
 /* Gets input and stuffs it into "buf".  number of characters read, or YY_NULL,
@@ -1313,7 +1268,7 @@ static int input ( yyscan_t yyscanner );
 	if ( YY_CURRENT_BUFFER_LVALUE->yy_is_interactive ) \
 		{ \
 		int c = '*'; \
-		int n; \
+		size_t n; \
 		for ( n = 0; n < max_size && \
 			     (c = getc( yyin )) != EOF && c != '\n'; ++n ) \
 			buf[n] = (char) c; \
@@ -1326,7 +1281,7 @@ static int input ( yyscan_t yyscanner );
 	else \
 		{ \
 		errno=0; \
-		while ( (result = (int) fread(buf, 1, (yy_size_t) max_size, yyin)) == 0 && ferror(yyin)) \
+		while ( (result = fread(buf, 1, max_size, yyin))==0 && ferror(yyin)) \
 			{ \
 			if( errno != EINTR) \
 				{ \
@@ -1368,7 +1323,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)
@@ -1432,7 +1387,7 @@ YY_DECL
 #line 132 "../SqlLexer.lpp"
 
 
-#line 1435 "SqlLexer_gen.cpp"
+#line 1391 "SqlLexer_gen.cpp"
 
 	while ( /*CONSTCOND*/1 )		/* loops until end-of-file is reached */
 		{
@@ -1459,13 +1414,13 @@ yy_match:
 			while ( yy_chk[yy_base[yy_current_state] + yy_c] != yy_current_state )
 				{
 				yy_current_state = (int) yy_def[yy_current_state];
-				if ( yy_current_state >= 599 )
-					yy_c = yy_meta[yy_c];
+				if ( yy_current_state >= 600 )
+					yy_c = yy_meta[(unsigned int) yy_c];
 				}
-			yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c];
+			yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
 			++yy_cp;
 			}
-		while ( yy_current_state != 598 );
+		while ( yy_current_state != 599 );
 		yy_cp = yyg->yy_last_accepting_cpos;
 		yy_current_state = yyg->yy_last_accepting_state;
 
@@ -1476,10 +1431,10 @@ yy_find_action:
 
 		if ( yy_act != YY_END_OF_BUFFER && yy_rule_can_match_eol[yy_act] )
 			{
-			int yyl;
+			yy_size_t yyl;
 			for ( yyl = 0; yyl < yyleng; ++yyl )
 				if ( yytext[yyl] == '\n' )
-					
+					   
     do{ yylineno++;
         yycolumn=0;
     }while(0)
@@ -1629,17 +1584,17 @@ return TOKEN_CASE;
 case 21:
 YY_RULE_SETUP
 #line 187 "../SqlLexer.lpp"
-return TOKEN_CSB_TREE;
+return TOKEN_CAST;
 	YY_BREAK
 case 22:
 YY_RULE_SETUP
 #line 188 "../SqlLexer.lpp"
-return TOKEN_BY;
+return TOKEN_CSB_TREE;
 	YY_BREAK
 case 23:
 YY_RULE_SETUP
 #line 189 "../SqlLexer.lpp"
-return TOKEN_CHARACTER;
+return TOKEN_BY;
 	YY_BREAK
 case 24:
 YY_RULE_SETUP
@@ -1649,72 +1604,72 @@ return TOKEN_CHARACTER;
 case 25:
 YY_RULE_SETUP
 #line 191 "../SqlLexer.lpp"
-return TOKEN_CHECK;
+return TOKEN_CHARACTER;
 	YY_BREAK
 case 26:
 YY_RULE_SETUP
 #line 192 "../SqlLexer.lpp"
-return TOKEN_COLUMN;
+return TOKEN_CHECK;
 	YY_BREAK
 case 27:
 YY_RULE_SETUP
 #line 193 "../SqlLexer.lpp"
-return TOKEN_CONSTRAINT;
+return TOKEN_COLUMN;
 	YY_BREAK
 case 28:
 YY_RULE_SETUP
 #line 194 "../SqlLexer.lpp"
-return TOKEN_COPY;
+return TOKEN_CONSTRAINT;
 	YY_BREAK
 case 29:
 YY_RULE_SETUP
 #line 195 "../SqlLexer.lpp"
-return TOKEN_CREATE;
+return TOKEN_COPY;
 	YY_BREAK
 case 30:
 YY_RULE_SETUP
 #line 196 "../SqlLexer.lpp"
-return TOKEN_CURRENT;
+return TOKEN_CREATE;
 	YY_BREAK
 case 31:
 YY_RULE_SETUP
 #line 197 "../SqlLexer.lpp"
-return TOKEN_DATE;
+return TOKEN_CURRENT;
 	YY_BREAK
 case 32:
 YY_RULE_SETUP
 #line 198 "../SqlLexer.lpp"
-return TOKEN_DATETIME;
+return TOKEN_DATE;
 	YY_BREAK
 case 33:
 YY_RULE_SETUP
 #line 199 "../SqlLexer.lpp"
-return TOKEN_DAY;
+return TOKEN_DATETIME;
 	YY_BREAK
 case 34:
 YY_RULE_SETUP
 #line 200 "../SqlLexer.lpp"
-return TOKEN_DECIMAL;
+return TOKEN_DAY;
 	YY_BREAK
 case 35:
 YY_RULE_SETUP
 #line 201 "../SqlLexer.lpp"
-return TOKEN_DEFAULT;
+return TOKEN_DECIMAL;
 	YY_BREAK
 case 36:
 YY_RULE_SETUP
 #line 202 "../SqlLexer.lpp"
-return TOKEN_DELETE;
+return TOKEN_DEFAULT;
 	YY_BREAK
 case 37:
 YY_RULE_SETUP
 #line 203 "../SqlLexer.lpp"
-return TOKEN_DELIMITER;
+return TOKEN_DELETE;
 	YY_BREAK
 case 38:
 YY_RULE_SETUP
 #line 204 "../SqlLexer.lpp"
-return TOKEN_DESC;
+return TOKEN_DELIMITER;
 	YY_BREAK
 case 39:
 YY_RULE_SETUP
@@ -1724,127 +1679,127 @@ return TOKEN_DESC;
 case 40:
 YY_RULE_SETUP
 #line 206 "../SqlLexer.lpp"
-return TOKEN_DISTINCT;
+return TOKEN_DESC;
 	YY_BREAK
 case 41:
 YY_RULE_SETUP
 #line 207 "../SqlLexer.lpp"
-return TOKEN_DOUBLE;
+return TOKEN_DISTINCT;
 	YY_BREAK
 case 42:
 YY_RULE_SETUP
 #line 208 "../SqlLexer.lpp"
-return TOKEN_DROP;
+return TOKEN_DOUBLE;
 	YY_BREAK
 case 43:
 YY_RULE_SETUP
 #line 209 "../SqlLexer.lpp"
-return TOKEN_ELSE;
+return TOKEN_DROP;
 	YY_BREAK
 case 44:
 YY_RULE_SETUP
 #line 210 "../SqlLexer.lpp"
-return TOKEN_END;
+return TOKEN_ELSE;
 	YY_BREAK
 case 45:
 YY_RULE_SETUP
 #line 211 "../SqlLexer.lpp"
-return TOKEN_ESCAPE_STRINGS;
+return TOKEN_END;
 	YY_BREAK
 case 46:
 YY_RULE_SETUP
 #line 212 "../SqlLexer.lpp"
-return TOKEN_EXISTS;
+return TOKEN_ESCAPE_STRINGS;
 	YY_BREAK
 case 47:
 YY_RULE_SETUP
 #line 213 "../SqlLexer.lpp"
-return TOKEN_EXTRACT;
+return TOKEN_EXISTS;
 	YY_BREAK
 case 48:
 YY_RULE_SETUP
 #line 214 "../SqlLexer.lpp"
-return TOKEN_FALSE;
+return TOKEN_EXTRACT;
 	YY_BREAK
 case 49:
 YY_RULE_SETUP
 #line 215 "../SqlLexer.lpp"
-return TOKEN_FIRST;
+return TOKEN_FALSE;
 	YY_BREAK
 case 50:
 YY_RULE_SETUP
 #line 216 "../SqlLexer.lpp"
-return TOKEN_FLOAT;
+return TOKEN_FIRST;
 	YY_BREAK
 case 51:
 YY_RULE_SETUP
 #line 217 "../SqlLexer.lpp"
-return TOKEN_FOLLOWING;
+return TOKEN_FLOAT;
 	YY_BREAK
 case 52:
 YY_RULE_SETUP
 #line 218 "../SqlLexer.lpp"
-return TOKEN_FOR;
+return TOKEN_FOLLOWING;
 	YY_BREAK
 case 53:
 YY_RULE_SETUP
 #line 219 "../SqlLexer.lpp"
-return TOKEN_FOREIGN;
+return TOKEN_FOR;
 	YY_BREAK
 case 54:
 YY_RULE_SETUP
 #line 220 "../SqlLexer.lpp"
-return TOKEN_FROM;
+return TOKEN_FOREIGN;
 	YY_BREAK
 case 55:
 YY_RULE_SETUP
 #line 221 "../SqlLexer.lpp"
-return TOKEN_FULL;
+return TOKEN_FROM;
 	YY_BREAK
 case 56:
 YY_RULE_SETUP
 #line 222 "../SqlLexer.lpp"
-return TOKEN_GROUP;
+return TOKEN_FULL;
 	YY_BREAK
 case 57:
 YY_RULE_SETUP
 #line 223 "../SqlLexer.lpp"
-return TOKEN_HASH;
+return TOKEN_GROUP;
 	YY_BREAK
 case 58:
 YY_RULE_SETUP
 #line 224 "../SqlLexer.lpp"
-return TOKEN_HAVING;
+return TOKEN_HASH;
 	YY_BREAK
 case 59:
 YY_RULE_SETUP
 #line 225 "../SqlLexer.lpp"
-return TOKEN_HOUR;
+return TOKEN_HAVING;
 	YY_BREAK
 case 60:
 YY_RULE_SETUP
 #line 226 "../SqlLexer.lpp"
-return TOKEN_IN;
+return TOKEN_HOUR;
 	YY_BREAK
 case 61:
 YY_RULE_SETUP
 #line 227 "../SqlLexer.lpp"
-return TOKEN_INDEX;
+return TOKEN_IN;
 	YY_BREAK
 case 62:
 YY_RULE_SETUP
 #line 228 "../SqlLexer.lpp"
-return TOKEN_INNER;
+return TOKEN_INDEX;
 	YY_BREAK
 case 63:
 YY_RULE_SETUP
 #line 229 "../SqlLexer.lpp"
-return TOKEN_INSERT;
+return TOKEN_INNER;
 	YY_BREAK
 case 64:
 YY_RULE_SETUP
 #line 230 "../SqlLexer.lpp"
-return TOKEN_INTEGER;
+return TOKEN_INSERT;
 	YY_BREAK
 case 65:
 YY_RULE_SETUP
@@ -1854,322 +1809,322 @@ return TOKEN_INTEGER;
 case 66:
 YY_RULE_SETUP
 #line 232 "../SqlLexer.lpp"
-return TOKEN_INTERSECT;
+return TOKEN_INTEGER;
 	YY_BREAK
 case 67:
 YY_RULE_SETUP
 #line 233 "../SqlLexer.lpp"
-return TOKEN_INTERVAL;
+return TOKEN_INTERSECT;
 	YY_BREAK
 case 68:
 YY_RULE_SETUP
 #line 234 "../SqlLexer.lpp"
-return TOKEN_INTO;
+return TOKEN_INTERVAL;
 	YY_BREAK
 case 69:
 YY_RULE_SETUP
 #line 235 "../SqlLexer.lpp"
-return TOKEN_IS;
+return TOKEN_INTO;
 	YY_BREAK
 case 70:
 YY_RULE_SETUP
 #line 236 "../SqlLexer.lpp"
-return TOKEN_JOIN;
+return TOKEN_IS;
 	YY_BREAK
 case 71:
 YY_RULE_SETUP
 #line 237 "../SqlLexer.lpp"
-return TOKEN_KEY;
+return TOKEN_JOIN;
 	YY_BREAK
 case 72:
 YY_RULE_SETUP
 #line 238 "../SqlLexer.lpp"
-return TOKEN_LAST;
+return TOKEN_KEY;
 	YY_BREAK
 case 73:
 YY_RULE_SETUP
 #line 239 "../SqlLexer.lpp"
-return TOKEN_LEFT;
+return TOKEN_LAST;
 	YY_BREAK
 case 74:
 YY_RULE_SETUP
 #line 240 "../SqlLexer.lpp"
-return TOKEN_LIKE;
+return TOKEN_LEFT;
 	YY_BREAK
 case 75:
 YY_RULE_SETUP
 #line 241 "../SqlLexer.lpp"
-return TOKEN_LIMIT;
+return TOKEN_LIKE;
 	YY_BREAK
 case 76:
 YY_RULE_SETUP
 #line 242 "../SqlLexer.lpp"
-return TOKEN_LONG;
+return TOKEN_LIMIT;
 	YY_BREAK
 case 77:
 YY_RULE_SETUP
 #line 243 "../SqlLexer.lpp"
-return TOKEN_MINUTE;
+return TOKEN_LONG;
 	YY_BREAK
 case 78:
 YY_RULE_SETUP
 #line 244 "../SqlLexer.lpp"
-return TOKEN_MONTH;
+return TOKEN_MINUTE;
 	YY_BREAK
 case 79:
 YY_RULE_SETUP
 #line 245 "../SqlLexer.lpp"
-return TOKEN_NOT;
+return TOKEN_MONTH;
 	YY_BREAK
 case 80:
 YY_RULE_SETUP
 #line 246 "../SqlLexer.lpp"
-return TOKEN_NULL;
+return TOKEN_NOT;
 	YY_BREAK
 case 81:
 YY_RULE_SETUP
 #line 247 "../SqlLexer.lpp"
-return TOKEN_NULLS;
+return TOKEN_NULL;
 	YY_BREAK
 case 82:
 YY_RULE_SETUP
 #line 248 "../SqlLexer.lpp"
-return TOKEN_OFF;
+return TOKEN_NULLS;
 	YY_BREAK
 case 83:
 YY_RULE_SETUP
 #line 249 "../SqlLexer.lpp"
-return TOKEN_ON;
+return TOKEN_OFF;
 	YY_BREAK
 case 84:
 YY_RULE_SETUP
 #line 250 "../SqlLexer.lpp"
-return TOKEN_OR;
+return TOKEN_ON;
 	YY_BREAK
 case 85:
 YY_RULE_SETUP
 #line 251 "../SqlLexer.lpp"
-return TOKEN_ORDER;
+return TOKEN_OR;
 	YY_BREAK
 case 86:
 YY_RULE_SETUP
 #line 252 "../SqlLexer.lpp"
-return TOKEN_OUTER;
+return TOKEN_ORDER;
 	YY_BREAK
 case 87:
 YY_RULE_SETUP
 #line 253 "../SqlLexer.lpp"
-return TOKEN_OVER;
+return TOKEN_OUTER;
 	YY_BREAK
 case 88:
 YY_RULE_SETUP
 #line 254 "../SqlLexer.lpp"
-return TOKEN_PARTITION;
+return TOKEN_OVER;
 	YY_BREAK
 case 89:
 YY_RULE_SETUP
 #line 255 "../SqlLexer.lpp"
-return TOKEN_PARTITIONS;
+return TOKEN_PARTITION;
 	YY_BREAK
 case 90:
 YY_RULE_SETUP
 #line 256 "../SqlLexer.lpp"
-return TOKEN_PERCENT;
+return TOKEN_PARTITIONS;
 	YY_BREAK
 case 91:
 YY_RULE_SETUP
 #line 257 "../SqlLexer.lpp"
-return TOKEN_PRECEDING;
+return TOKEN_PERCENT;
 	YY_BREAK
 case 92:
 YY_RULE_SETUP
 #line 258 "../SqlLexer.lpp"
-return TOKEN_PRIMARY;
+return TOKEN_PRECEDING;
 	YY_BREAK
 case 93:
 YY_RULE_SETUP
 #line 259 "../SqlLexer.lpp"
-return TOKEN_PRIORITY;
+return TOKEN_PRIMARY;
 	YY_BREAK
 case 94:
 YY_RULE_SETUP
 #line 260 "../SqlLexer.lpp"
-return TOKEN_QUIT;
+return TOKEN_PRIORITY;
 	YY_BREAK
 case 95:
 YY_RULE_SETUP
 #line 261 "../SqlLexer.lpp"
-return TOKEN_RANGE;
+return TOKEN_QUIT;
 	YY_BREAK
 case 96:
 YY_RULE_SETUP
 #line 262 "../SqlLexer.lpp"
-return TOKEN_REAL;
+return TOKEN_RANGE;
 	YY_BREAK
 case 97:
 YY_RULE_SETUP
 #line 263 "../SqlLexer.lpp"
-return TOKEN_REFERENCES;
+return TOKEN_REAL;
 	YY_BREAK
 case 98:
 YY_RULE_SETUP
 #line 264 "../SqlLexer.lpp"
-return TOKEN_REGEXP;
+return TOKEN_REFERENCES;
 	YY_BREAK
 case 99:
 YY_RULE_SETUP
 #line 265 "../SqlLexer.lpp"
-return TOKEN_RIGHT;
+return TOKEN_REGEXP;
 	YY_BREAK
 case 100:
 YY_RULE_SETUP
 #line 266 "../SqlLexer.lpp"
-return TOKEN_ROW;
+return TOKEN_RIGHT;
 	YY_BREAK
 case 101:
 YY_RULE_SETUP
 #line 267 "../SqlLexer.lpp"
-return TOKEN_ROW_DELIMITER;
+return TOKEN_ROW;
 	YY_BREAK
 case 102:
 YY_RULE_SETUP
 #line 268 "../SqlLexer.lpp"
-return TOKEN_ROWS;
+return TOKEN_ROW_DELIMITER;
 	YY_BREAK
 case 103:
 YY_RULE_SETUP
 #line 269 "../SqlLexer.lpp"
-return TOKEN_SECOND;
+return TOKEN_ROWS;
 	YY_BREAK
 case 104:
 YY_RULE_SETUP
 #line 270 "../SqlLexer.lpp"
-return TOKEN_SELECT;
+return TOKEN_SECOND;
 	YY_BREAK
 case 105:
 YY_RULE_SETUP
 #line 271 "../SqlLexer.lpp"
-return TOKEN_SET;
+return TOKEN_SELECT;
 	YY_BREAK
 case 106:
 YY_RULE_SETUP
 #line 272 "../SqlLexer.lpp"
-return TOKEN_SMA;
+return TOKEN_SET;
 	YY_BREAK
 case 107:
 YY_RULE_SETUP
 #line 273 "../SqlLexer.lpp"
-return TOKEN_SMALLINT;
+return TOKEN_SMA;
 	YY_BREAK
 case 108:
 YY_RULE_SETUP
 #line 274 "../SqlLexer.lpp"
-return TOKEN_SUBSTRING;
+return TOKEN_SMALLINT;
 	YY_BREAK
 case 109:
 YY_RULE_SETUP
 #line 275 "../SqlLexer.lpp"
-return TOKEN_TABLE;
+return TOKEN_SUBSTRING;
 	YY_BREAK
 case 110:
 YY_RULE_SETUP
 #line 276 "../SqlLexer.lpp"
-return TOKEN_THEN;
+return TOKEN_TABLE;
 	YY_BREAK
 case 111:
 YY_RULE_SETUP
 #line 277 "../SqlLexer.lpp"
-return TOKEN_TIME;
+return TOKEN_THEN;
 	YY_BREAK
 case 112:
 YY_RULE_SETUP
 #line 278 "../SqlLexer.lpp"
-return TOKEN_TIMESTAMP;
+return TOKEN_TIME;
 	YY_BREAK
 case 113:
 YY_RULE_SETUP
 #line 279 "../SqlLexer.lpp"
-return TOKEN_TRUE;
+return TOKEN_TIMESTAMP;
 	YY_BREAK
 case 114:
 YY_RULE_SETUP
 #line 280 "../SqlLexer.lpp"
-return TOKEN_TUPLESAMPLE;
+return TOKEN_TRUE;
 	YY_BREAK
 case 115:
 YY_RULE_SETUP
 #line 281 "../SqlLexer.lpp"
-return TOKEN_UNBOUNDED;
+return TOKEN_TUPLESAMPLE;
 	YY_BREAK
 case 116:
 YY_RULE_SETUP
 #line 282 "../SqlLexer.lpp"
-return TOKEN_UNION;
+return TOKEN_UNBOUNDED;
 	YY_BREAK
 case 117:
 YY_RULE_SETUP
 #line 283 "../SqlLexer.lpp"
-return TOKEN_UNIQUE;
+return TOKEN_UNION;
 	YY_BREAK
 case 118:
 YY_RULE_SETUP
 #line 284 "../SqlLexer.lpp"
-return TOKEN_UPDATE;
+return TOKEN_UNIQUE;
 	YY_BREAK
 case 119:
 YY_RULE_SETUP
 #line 285 "../SqlLexer.lpp"
-return TOKEN_USING;
+return TOKEN_UPDATE;
 	YY_BREAK
 case 120:
 YY_RULE_SETUP
 #line 286 "../SqlLexer.lpp"
-return TOKEN_VALUES;
+return TOKEN_USING;
 	YY_BREAK
 case 121:
 YY_RULE_SETUP
 #line 287 "../SqlLexer.lpp"
-return TOKEN_VARCHAR;
+return TOKEN_VALUES;
 	YY_BREAK
 case 122:
 YY_RULE_SETUP
 #line 288 "../SqlLexer.lpp"
-return TOKEN_WHEN;
+return TOKEN_VARCHAR;
 	YY_BREAK
 case 123:
 YY_RULE_SETUP
 #line 289 "../SqlLexer.lpp"
-return TOKEN_WHERE;
+return TOKEN_WHEN;
 	YY_BREAK
 case 124:
 YY_RULE_SETUP
 #line 290 "../SqlLexer.lpp"
-return TOKEN_WINDOW;
+return TOKEN_WHERE;
 	YY_BREAK
 case 125:
 YY_RULE_SETUP
 #line 291 "../SqlLexer.lpp"
-return TOKEN_WITH;
+return TOKEN_WINDOW;
 	YY_BREAK
 case 126:
 YY_RULE_SETUP
 #line 292 "../SqlLexer.lpp"
-return TOKEN_YEAR;
+return TOKEN_WITH;
 	YY_BREAK
 case 127:
 YY_RULE_SETUP
 #line 293 "../SqlLexer.lpp"
-return TOKEN_YEARMONTH;
+return TOKEN_YEAR;
 	YY_BREAK
 case 128:
 YY_RULE_SETUP
-#line 295 "../SqlLexer.lpp"
-return TOKEN_EQ;
+#line 294 "../SqlLexer.lpp"
+return TOKEN_YEARMONTH;
 	YY_BREAK
 case 129:
 YY_RULE_SETUP
 #line 296 "../SqlLexer.lpp"
-return TOKEN_NEQ;
+return TOKEN_EQ;
 	YY_BREAK
 case 130:
 YY_RULE_SETUP
@@ -2179,56 +2134,61 @@ return TOKEN_NEQ;
 case 131:
 YY_RULE_SETUP
 #line 298 "../SqlLexer.lpp"
-return TOKEN_LT;
+return TOKEN_NEQ;
 	YY_BREAK
 case 132:
 YY_RULE_SETUP
 #line 299 "../SqlLexer.lpp"
-return TOKEN_GT;
+return TOKEN_LT;
 	YY_BREAK
 case 133:
 YY_RULE_SETUP
 #line 300 "../SqlLexer.lpp"
-return TOKEN_LEQ;
+return TOKEN_GT;
 	YY_BREAK
 case 134:
 YY_RULE_SETUP
 #line 301 "../SqlLexer.lpp"
-return TOKEN_GEQ;
+return TOKEN_LEQ;
 	YY_BREAK
 case 135:
 YY_RULE_SETUP
-#line 303 "../SqlLexer.lpp"
-return yytext[0];
+#line 302 "../SqlLexer.lpp"
+return TOKEN_GEQ;
 	YY_BREAK
 case 136:
 YY_RULE_SETUP
 #line 304 "../SqlLexer.lpp"
 return yytext[0];
 	YY_BREAK
+case 137:
+YY_RULE_SETUP
+#line 305 "../SqlLexer.lpp"
+return yytext[0];
+	YY_BREAK
 /**
     * Quoted strings. Prefacing a string with an 'e' or 'E' causes escape
     * sequences to be processed (as in PostgreSQL).
     **/
-case 137:
+case 138:
 YY_RULE_SETUP
-#line 310 "../SqlLexer.lpp"
+#line 311 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED_ESCAPED);
   }
 	YY_BREAK
-case 138:
+case 139:
 YY_RULE_SETUP
-#line 315 "../SqlLexer.lpp"
+#line 316 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED);
   }
 	YY_BREAK
-case 139:
+case 140:
 YY_RULE_SETUP
-#line 320 "../SqlLexer.lpp"
+#line 321 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_DOUBLE_QUOTED);
@@ -2240,7 +2200,7 @@ YY_RULE_SETUP
 case YY_STATE_EOF(CONDITION_STRING_SINGLE_QUOTED):
 case YY_STATE_EOF(CONDITION_STRING_SINGLE_QUOTED_ESCAPED):
 case YY_STATE_EOF(CONDITION_STRING_DOUBLE_QUOTED):
-#line 329 "../SqlLexer.lpp"
+#line 330 "../SqlLexer.lpp"
 {
     delete yylval->string_value_;
     BEGIN(INITIAL);
@@ -2251,9 +2211,9 @@ case YY_STATE_EOF(CONDITION_STRING_DOUBLE_QUOTED):
 
 /* Process escape sequences. */
 
-case 140:
+case 141:
 YY_RULE_SETUP
-#line 339 "../SqlLexer.lpp"
+#line 340 "../SqlLexer.lpp"
 {
     /* Octal code */
     unsigned int code;
@@ -2267,9 +2227,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 141:
+case 142:
 YY_RULE_SETUP
-#line 351 "../SqlLexer.lpp"
+#line 352 "../SqlLexer.lpp"
 {
     /* Hexadecimal code */
     unsigned int code;
@@ -2277,9 +2237,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 142:
+case 143:
 YY_RULE_SETUP
-#line 357 "../SqlLexer.lpp"
+#line 358 "../SqlLexer.lpp"
 {
     /* A numeric escape sequence that isn't correctly specified. */
     delete yylval->string_value_;
@@ -2288,58 +2248,58 @@ YY_RULE_SETUP
     return TOKEN_LEX_ERROR;
   }
 	YY_BREAK
-case 143:
+case 144:
 YY_RULE_SETUP
-#line 364 "../SqlLexer.lpp"
+#line 365 "../SqlLexer.lpp"
 {
     /* Backspace */
     yylval->string_value_->push_back('\b');
   }
 	YY_BREAK
-case 144:
+case 145:
 YY_RULE_SETUP
-#line 368 "../SqlLexer.lpp"
+#line 369 "../SqlLexer.lpp"
 {
     /* Form-feed */
     yylval->string_value_->push_back('\f');
   }
 	YY_BREAK
-case 145:
+case 146:
 YY_RULE_SETUP
-#line 372 "../SqlLexer.lpp"
+#line 373 "../SqlLexer.lpp"
 {
     /* Newline */
     yylval->string_value_->push_back('\n');
   }
 	YY_BREAK
-case 146:
+case 147:
 YY_RULE_SETUP
-#line 376 "../SqlLexer.lpp"
+#line 377 "../SqlLexer.lpp"
 {
     /* Carriage-return */
     yylval->string_value_->push_back('\r');
   }
 	YY_BREAK
-case 147:
+case 148:
 YY_RULE_SETUP
-#line 380 "../SqlLexer.lpp"
+#line 381 "../SqlLexer.lpp"
 {
     /* Horizontal Tab */
     yylval->string_value_->push_back('\t');
   }
 	YY_BREAK
-case 148:
-/* rule 148 can match eol */
+case 149:
+/* rule 149 can match eol */
 YY_RULE_SETUP
-#line 384 "../SqlLexer.lpp"
+#line 385 "../SqlLexer.lpp"
 {
     /* Any other character (including actual newline or carriage return) */
     yylval->string_value_->push_back(yytext[1]);
   }
 	YY_BREAK
-case 149:
+case 150:
 YY_RULE_SETUP
-#line 388 "../SqlLexer.lpp"
+#line 389 "../SqlLexer.lpp"
 {
     /* This should only be encountered right before an EOF. */
     delete yylval->string_value_;
@@ -2350,17 +2310,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 150:
+case 151:
 YY_RULE_SETUP
-#line 398 "../SqlLexer.lpp"
+#line 399 "../SqlLexer.lpp"
 {
     /* Two quotes in a row become a single quote (this is specified by the SQL standard). */
     yylval->string_value_->push_back('\'');
   }
 	YY_BREAK
-case 151:
+case 152:
 YY_RULE_SETUP
-#line 402 "../SqlLexer.lpp"
+#line 403 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2369,17 +2329,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 152:
+case 153:
 YY_RULE_SETUP
-#line 410 "../SqlLexer.lpp"
+#line 411 "../SqlLexer.lpp"
 {
     /* Two quotes in a row become a single quote (this is specified by the SQL standard). */
     yylval->string_value_->push_back('"');
   }
 	YY_BREAK
-case 153:
+case 154:
 YY_RULE_SETUP
-#line 414 "../SqlLexer.lpp"
+#line 415 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2387,94 +2347,94 @@ YY_RULE_SETUP
   }
 	YY_BREAK
 
-case 154:
-/* rule 154 can match eol */
+case 155:
+/* rule 155 can match eol */
 YY_RULE_SETUP
-#line 421 "../SqlLexer.lpp"
+#line 422 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 155:
-/* rule 155 can match eol */
+case 156:
+/* rule 156 can match eol */
 YY_RULE_SETUP
-#line 426 "../SqlLexer.lpp"
+#line 427 "../SqlLexer.lpp"
 {
   /* Scan up to a quote or escape sequence. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 156:
-/* rule 156 can match eol */
+case 157:
+/* rule 157 can match eol */
 YY_RULE_SETUP
-#line 431 "../SqlLexer.lpp"
+#line 432 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
 
-case 157:
+case 158:
 YY_RULE_SETUP
-#line 437 "../SqlLexer.lpp"
+#line 438 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(
         yylloc->first_line, yylloc->first_column, std::string(yytext, yyleng));
     return TOKEN_NAME;
   }

<TRUNCATED>


[20/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/CMathUnaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CMathUnaryOperations.hpp b/types/operations/unary_operations/CMathUnaryOperations.hpp
new file mode 100644
index 0000000..7a372e0
--- /dev/null
+++ b/types/operations/unary_operations/CMathUnaryOperations.hpp
@@ -0,0 +1,116 @@
+/**
+ * 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_TYPES_OPERATIONS_UNARY_OPERATIONS_CMATH_UNARY_OPERATIONS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_CMATH_UNARY_OPERATIONS_HPP_
+
+#include <cmath>
+#include <string>
+
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/operations/OperationUtil.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename ArgumentT, typename ResultT,
+          typename ResultT::cpptype f(typename ArgumentT::cpptype),
+          typename FunctorNameT>
+struct CMathUnaryFunctorWrapper {
+  struct Implemenation : public UnaryFunctor<ArgumentT, ResultT> {
+    inline typename ResultT::cpptype apply(
+        const typename ArgumentT::cpptype &argument) const {
+      return f(argument);
+    }
+    inline static std::string GetName() {
+      return FunctorNameT::ToString();
+    }
+  };
+
+  typedef Implemenation type;
+};
+
+template <typename ArgumentT, typename ResultT,
+          typename ResultT::cpptype f(typename ArgumentT::cpptype),
+          typename FunctorNameT>
+using CMathUnaryFunctor =
+    typename CMathUnaryFunctorWrapper<ArgumentT, ResultT, f, FunctorNameT>::type;
+
+inline std::int64_t CMathRound(const float arg) {
+  return std::llround(arg);
+}
+inline std::int64_t CMathRound(const double arg) {
+  return std::llround(arg);
+}
+
+using CMathUnaryFunctorPack = FunctorPack<
+// abs
+    CMathUnaryFunctor<IntType, IntType,
+                      std::abs, meta::StringLiteral<'a','b','s'>>,
+    CMathUnaryFunctor<LongType, LongType,
+                      std::abs, meta::StringLiteral<'a','b','s'>>,
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::fabs, meta::StringLiteral<'a','b','s'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::fabs, meta::StringLiteral<'a','b','s'>>,
+// sqrt
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::sqrt, meta::StringLiteral<'s','q','r','t'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::sqrt, meta::StringLiteral<'s','q','r','t'>>,
+// exp
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::exp, meta::StringLiteral<'e','x','p'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::exp, meta::StringLiteral<'e','x','p'>>,
+// log
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::log, meta::StringLiteral<'l','o','g'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::log, meta::StringLiteral<'l','o','g'>>,
+// ceil
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::ceil, meta::StringLiteral<'c','e','i','l'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::ceil, meta::StringLiteral<'c','e','i','l'>>,
+// floor
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::floor, meta::StringLiteral<'f','l','o','o','r'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::floor, meta::StringLiteral<'f','l','o','o','r'>>,
+// round
+    CMathUnaryFunctor<FloatType, LongType,
+                      CMathRound, meta::StringLiteral<'r','o','u','n','d'>>,
+    CMathUnaryFunctor<DoubleType, LongType,
+                      CMathRound, meta::StringLiteral<'r','o','u','n','d'>>
+>;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_CMATH_UNARY_OPERATIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/CastOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CastOperation.cpp b/types/operations/unary_operations/CastOperation.cpp
new file mode 100644
index 0000000..5091e89
--- /dev/null
+++ b/types/operations/unary_operations/CastOperation.cpp
@@ -0,0 +1,298 @@
+/**
+ * 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 "types/operations/unary_operations/CastOperation.hpp"
+
+#include <algorithm>
+#include <map>
+#include <cstdint>
+#include <cstdlib>
+#include <string>
+#include <vector>
+
+#include "types/CharType.hpp"
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeUtil.hpp"
+#include "types/TypedValue.hpp"
+#include "types/VarCharType.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
+#include "types/port/strnlen.hpp"
+#include "utility/EqualsAnyConstant.hpp"
+#include "utility/StringUtil.hpp"
+
+namespace quickstep {
+
+namespace {
+
+template <typename ArgumentT, typename ResultT>
+struct NumericCastToNumericFunctor
+    : public UnaryFunctor<ArgumentT, ResultT> {
+  inline typename ResultT::cpptype apply(
+      const typename ArgumentT::cpptype &argument) const {
+    return static_cast<typename ResultT::cpptype>(argument);
+  }
+};
+
+template <typename ArgumentT, typename ResultT>
+class CastToAsciiStringFunctor : public UnaryFunctor<ArgumentT, ResultT> {
+ public:
+  explicit CastToAsciiStringFunctor(const ArgumentT &argument_type,
+                                    const std::size_t max_string_length)
+      : argument_type_(argument_type),
+        max_string_length_(max_string_length) {}
+
+  inline void apply(const typename ArgumentT::cpptype &argument, void *result) const {
+    std::string str = argument_type_.printValueToString(TypedValue(argument));
+    const std::size_t str_len = str.length();
+
+    if (str_len < max_string_length_) {
+      std::memcpy(result, str.c_str(), str_len);
+      static_cast<char *>(result)[str_len] = 0;
+    } else {
+      std::memcpy(result, str.c_str(), max_string_length_);
+    }
+  }
+
+  inline TypedValue apply(const typename ArgumentT::cpptype &argument) const {
+    std::string str = argument_type_.printValueToString(TypedValue(argument));
+    const std::size_t len = std::min(str.length(), max_string_length_);
+    const std::size_t buf_len = len + 1;
+
+    char *buf = static_cast<char *>(std::malloc(buf_len));
+    std::memcpy(buf, str.c_str(), len);
+    buf[len] = 0;
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, buf_len);
+  }
+
+ private:
+  const ArgumentT &argument_type_;
+  const std::size_t max_string_length_;
+};
+
+template <typename ResultCppType>
+ResultCppType CastStringToNumericImpl(const char *str);
+
+template <>
+bool CastStringToNumericImpl(const char *str) {
+  const std::string lo_str = ToLower(str);
+  if (lo_str == "true") {
+    return true;
+  } else {
+    return false;
+  }
+}
+template <>
+int CastStringToNumericImpl(const char *str) {
+  return std::atoi(str);
+}
+template <>
+float CastStringToNumericImpl(const char *str) {
+  return static_cast<float>(std::atof(str));
+}
+template <>
+std::int64_t CastStringToNumericImpl(const char *str) {
+  return std::atoll(str);
+}
+template <>
+double CastStringToNumericImpl(const char *str) {
+  return std::atof(str);
+}
+
+template <typename ArgumentT, typename ResultT,
+          typename ResultT::cpptype f(const char*)>
+struct AsciiStringCastToNumericFunctor
+    : public UnaryFunctor<ArgumentT, ResultT> {
+  explicit AsciiStringCastToNumericFunctor(const std::size_t max_string_length)
+      : max_string_length_(max_string_length) {}
+
+  inline typename ResultT::cpptype apply(const TypedValue &argument) const {
+    return f(static_cast<const char*>(argument.getDataPtr()));
+  }
+
+  inline typename ResultT::cpptype apply(const void *argument) const {
+    const char *str = static_cast<const char*>(argument);
+    const std::string value(str, strnlen(str, max_string_length_));
+    return f(value.c_str());
+  }
+
+ private:
+  const std::size_t max_string_length_;
+};
+
+template <typename ArgumentT, typename ResultT>
+struct AsciiStringCastToAsciiStringFunctor
+    : public UnaryFunctor<ArgumentT, ResultT> {
+  explicit AsciiStringCastToAsciiStringFunctor(const std::size_t max_string_length)
+      : max_string_length_(max_string_length) {}
+
+  inline void apply(const void *argument, void *result) const {
+    std::memcpy(result, argument, max_string_length_);
+  }
+
+  inline void apply(const TypedValue &argument, void *result) const {
+    std::memcpy(result,
+                argument.getOutOfLineData(),
+                std::min(argument.getDataSize(), max_string_length_));
+  }
+
+  inline TypedValue apply(const void *argument) const {
+    const std::size_t len =
+        strnlen(static_cast<const char*>(argument), max_string_length_);
+
+    char *buf = static_cast<char *>(std::malloc(len+1));
+    std::memcpy(buf, argument, len);
+    buf[len] = 0;
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, len+1);
+  }
+
+  inline TypedValue apply(const TypedValue &argument) const {
+    const std::size_t len =
+        std::min(argument.getDataSize() - 1, max_string_length_);
+
+    char *buf = static_cast<char *>(std::malloc(len+1));
+    std::memcpy(buf, argument.getDataPtr(), len);
+    buf[len] = 0;
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, len+1);
+  }
+
+ private:
+  const std::size_t max_string_length_;
+};
+
+}  // namespace
+
+const re2::RE2 CastOperation::kTypePattern("([a-z]+)(\\(([0-9]+)\\))?");
+
+const std::map<std::string, TypeID> CastOperation::kNameToTypeIDMap = {
+    { "bool",    kBool },
+    { "int",     kInt },
+    { "long",    kLong },
+    { "float",   kFloat },
+    { "double",  kDouble },
+    { "char",    kChar },
+    { "varchar", kVarChar }
+};
+
+UncheckedUnaryOperator* CastOperation::makeUncheckedUnaryOperator(
+    const Type &type,
+    const std::vector<TypedValue> &static_arguments) const {
+  const Type *result_type = getResultType(type, static_arguments);
+  DCHECK(result_type != nullptr);
+
+  const TypeID argument_type_id = type.getTypeID();
+  const TypeID result_type_id = result_type->getTypeID();
+
+  if (QUICKSTEP_EQUALS_ANY_CONSTANT(argument_type_id, kBool, kInt, kLong, kFloat, kDouble)) {
+    return InvokeOnTypeID<TypeIDSelectorNumeric>(
+        argument_type_id,
+        [&](auto arg_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+      using ArgumentT = typename TypeIDTrait<decltype(arg_tid)::value>::TypeClass;
+
+      switch (result_type_id) {
+        case kBool:  // Fall through
+        case kInt:
+        case kLong:
+        case kFloat:
+        case kDouble: {
+          return InvokeOnTypeID<TypeIDSelectorNumeric>(
+              result_type_id,
+              [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+            using ResultT = typename TypeIDTrait<decltype(result_tid)::value>::TypeClass;
+
+            return new UncheckedUnaryOperatorWrapperCodegen<
+                NumericCastToNumericFunctor<ArgumentT, ResultT>>(type, *result_type);
+          });
+        }
+        case kChar:  // Fall through
+        case kVarChar: {
+          return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
+              result_type_id,
+              [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+            using ResultT = typename TypeIDTrait<decltype(result_tid)::value>::TypeClass;
+
+            return new UncheckedUnaryOperatorWrapperCodegen<
+                 CastToAsciiStringFunctor<ArgumentT, ResultT>>(
+                     type, *result_type,
+                     static_cast<const ArgumentT&>(type),
+                     static_cast<const ResultT*>(result_type)->getStringLength());
+          });
+        }
+        default:
+          LOG(FATAL) << "Unexpected result type " << result_type->getName()
+                     << " in CastOperation::makeUncheckedUnaryOperator "
+                     << "for argument type " << type.getName();
+      }
+    });
+  } else if (QUICKSTEP_EQUALS_ANY_CONSTANT(argument_type_id, kChar, kVarChar)) {
+    return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
+        argument_type_id,
+        [&](auto arg_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+      using ArgumentT = typename TypeIDTrait<decltype(arg_tid)::value>::TypeClass;
+
+      switch (result_type_id) {
+        case kBool:  // Fall through
+        case kInt:
+        case kLong:
+        case kFloat:
+        case kDouble: {
+          return InvokeOnTypeID<TypeIDSelectorNumeric>(
+              result_type_id,
+              [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+            using ResultT = typename TypeIDTrait<decltype(result_tid)::value>::TypeClass;
+
+            return new UncheckedUnaryOperatorWrapperCodegen<
+                AsciiStringCastToNumericFunctor<
+                    ArgumentT, ResultT,
+                    CastStringToNumericImpl<typename ResultT::cpptype>>>(
+                        type, *result_type,
+                        static_cast<const ArgumentT&>(type).getStringLength());
+          });
+        }
+        case kChar:  // Fall through
+        case kVarChar: {
+          return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
+              result_type_id,
+              [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+            using ResultT = typename TypeIDTrait<decltype(result_tid)::value>::TypeClass;
+
+            return new UncheckedUnaryOperatorWrapperCodegen<
+                 AsciiStringCastToAsciiStringFunctor<ArgumentT, ResultT>>(
+                     type, *result_type,
+                     std::min(static_cast<const ArgumentT&>(type).getStringLength(),
+                              static_cast<const ResultT*>(result_type)->getStringLength()));
+          });
+        }
+        default:
+          LOG(FATAL) << "Unexpected result type " << result_type->getName()
+                     << " in CastOperation::makeUncheckedUnaryOperator "
+                     << "for argument type " << type.getName();
+      }
+    });
+  }
+
+  LOG(FATAL) << "Unexpected argument type in "
+             << "CastOperation::makeUncheckedUnaryOperator: "
+             << result_type->getName();
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/CastOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CastOperation.hpp b/types/operations/unary_operations/CastOperation.hpp
new file mode 100644
index 0000000..140c152
--- /dev/null
+++ b/types/operations/unary_operations/CastOperation.hpp
@@ -0,0 +1,154 @@
+/**
+ * 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_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_
+
+#include <cstddef>
+#include <map>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "types/IntType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/operations/unary_operations/UnaryOperation.hpp"
+#include "utility/Macros.hpp"
+#include "utility/StringUtil.hpp"
+
+#include "glog/logging.h"
+
+#include "re2/stringpiece.h"
+#include "re2/re2.h"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+/**
+ * @brief UnaryOperation for CAST.
+ */
+class CastOperation : public UnaryOperation {
+ public:
+  CastOperation() {}
+
+  std::string getName() const override {
+    return "Cast";
+  }
+
+  std::string getShortName() const override {
+    return "Cast";
+  }
+
+  std::vector<OperationSignaturePtr> getSignatures() const override {
+    const std::vector<TypeID> source_type_ids =
+        { kBool, kInt, kLong, kFloat, kDouble, kChar, kVarChar };
+    const std::vector<TypeID> target_type_carrier = { kVarChar };
+
+    std::vector<OperationSignaturePtr> signatures;
+    for (const TypeID source_type_id : source_type_ids) {
+      signatures.emplace_back(
+          OperationSignature::Create(getName(), {source_type_id}, target_type_carrier));
+    }
+    return signatures;
+  }
+
+  bool canApplyTo(const Type &type,
+                  const std::vector<TypedValue> &static_arguments,
+                  std::string *message) const override {
+    DCHECK_EQ(1u, static_arguments.size());
+    if (getResultTypeInternal(type, static_arguments.front()) == nullptr) {
+      *message = "Invalid target type for CAST";
+      return false;
+    }
+    return true;
+  }
+
+  const Type* getResultType(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK_EQ(1u, static_arguments.size());
+    const Type *target_type =
+        getResultTypeInternal(type, static_arguments.front());
+    DCHECK(target_type != nullptr);
+    return target_type;
+  }
+
+  UncheckedUnaryOperator* makeUncheckedUnaryOperator(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override;
+
+ private:
+  static const Type* getResultTypeInternal(const Type &type,
+                                           const TypedValue &type_arg) {
+    DCHECK(type_arg.getTypeID() == kVarChar);
+    const std::string type_str =
+        ToLower(std::string(static_cast<const char*>(type_arg.getOutOfLineData())));
+
+    if (type_str == "text") {
+      return &TypeFactory::GetType(
+          kVarChar, type.getPrintWidth(), type.isNullable());
+    }
+
+    const re2::StringPiece type_piece(type_str);
+    std::string type_name;
+    std::string length_str;
+    if (!re2::RE2::FullMatch(type_piece,
+                             kTypePattern,
+                             &type_name,
+                             static_cast<void *>(nullptr),
+                             &length_str)) {
+      return nullptr;
+    }
+
+    auto it = kNameToTypeIDMap.find(type_name);
+    if (it == kNameToTypeIDMap.end()) {
+      return nullptr;
+    }
+
+    if (length_str.empty()) {
+      return &TypeFactory::GetType(it->second);
+    } else {
+      TypedValue length_value;
+      if (IntType::InstanceNonNullable().parseValueFromString(length_str, &length_value)) {
+        return &TypeFactory::GetType(
+            it->second,
+            static_cast<std::size_t>(length_value.getLiteral<int>()),
+            type.isNullable());
+      }
+    }
+    return nullptr;
+  }
+
+  static const re2::RE2 kTypePattern;
+  static const std::map<std::string, TypeID> kNameToTypeIDMap;
+
+  DISALLOW_COPY_AND_ASSIGN(CastOperation);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/DateExtractOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/DateExtractOperation.cpp b/types/operations/unary_operations/DateExtractOperation.cpp
index c99e403..f95e109 100644
--- a/types/operations/unary_operations/DateExtractOperation.cpp
+++ b/types/operations/unary_operations/DateExtractOperation.cpp
@@ -21,521 +21,135 @@
 
 #include <cstddef>
 #include <cstdint>
-#include <memory>
+#include <map>
 #include <string>
 #include <type_traits>
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-#include <utility>
 #include <vector>
 
-#include "storage/StorageBlockInfo.hpp"
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
+#include "types/DateType.hpp"
 #include "types/DatetimeLit.hpp"
-#include "types/IntType.hpp"
-#include "types/LongType.hpp"
+#include "types/DatetimeType.hpp"
 #include "types/Type.hpp"
-#include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
-#include "types/operations/Operation.pb.h"
-#include "utility/Macros.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
 
 #include "glog/logging.h"
 
-using std::int32_t;
-using std::int64_t;
-
 namespace quickstep {
 
-template <DateExtractUnit unit, bool argument_nullable>
-TypedValue DatetimeExtractUncheckedOperator<unit, argument_nullable>::applyToTypedValue(
-    const TypedValue &argument) const {
-  if (argument_nullable && argument.isNull()) {
-    return TypedValue(kLong);
-  }
-
-  return TypedValue(dateExtract(argument.getLiteral<DatetimeLit>()));
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-TypedValue DateExtractUncheckedOperator<unit, argument_nullable>::applyToTypedValue(
-    const TypedValue &argument) const {
-  if (argument_nullable && argument.isNull()) {
-    return TypedValue(kInt);
-  }
-
-  return TypedValue(dateExtract(argument.getLiteral<DateLit>()));
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-TypedValue DatetimeExtractUncheckedOperator<unit, argument_nullable>::applyToDataPtr(const void *argument) const {
-  if (argument_nullable && argument == nullptr) {
-    return TypedValue(kLong);
-  }
-
-  return TypedValue(dateExtract(*static_cast<const DatetimeLit*>(argument)));
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-TypedValue DateExtractUncheckedOperator<unit, argument_nullable>::applyToDataPtr(const void *argument) const {
-  if (argument_nullable && argument == nullptr) {
-    return TypedValue(kInt);
-  }
-
-  return TypedValue(dateExtract(*static_cast<const DateLit*>(argument)));
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-ColumnVector* DatetimeExtractUncheckedOperator<unit, argument_nullable>::applyToColumnVector(
-    const ColumnVector &argument) const {
-  // Datetime are usable with NativeColumnVector, so 'argument' should always
-  // be native.
-  DCHECK(argument.isNative());
-
-  const NativeColumnVector &native_argument = static_cast<const NativeColumnVector&>(argument);
-  std::unique_ptr<NativeColumnVector> result(
-      new NativeColumnVector(LongType::Instance(argument_nullable), native_argument.size()));
-
-  for (std::size_t pos = 0;
-       pos < native_argument.size();
-       ++pos) {
-    const DatetimeLit *datetime_arg =
-        static_cast<const DatetimeLit*>(native_argument.getUntypedValue<argument_nullable>(pos));
-    if (argument_nullable && (datetime_arg == nullptr)) {
-      result->appendNullValue();
-    } else {
-      *static_cast<int64_t*>(result->getPtrForDirectWrite())
-          = dateExtract(*datetime_arg);
-    }
-  }
-
-  return result.release();
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-ColumnVector* DateExtractUncheckedOperator<unit, argument_nullable>::applyToColumnVector(
-    const ColumnVector &argument) const {
-  // Date is usable with NativeColumnVector, so 'argument' should always
-  // be native.
-  DCHECK(argument.isNative());
-
-  const NativeColumnVector &native_argument = static_cast<const NativeColumnVector&>(argument);
-  std::unique_ptr<NativeColumnVector> result(
-      new NativeColumnVector(IntType::Instance(argument_nullable), native_argument.size()));
-
-  for (std::size_t pos = 0;
-       pos < native_argument.size();
-       ++pos) {
-    const DateLit *date_arg =
-        static_cast<const DateLit*>(native_argument.getUntypedValue<argument_nullable>(pos));
-    if (argument_nullable && (date_arg == nullptr)) {
-      result->appendNullValue();
-    } else {
-      *static_cast<int32_t*>(result->getPtrForDirectWrite())
-          = dateExtract(*date_arg);
-    }
-  }
-
-  return result.release();
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-template <DateExtractUnit unit, bool argument_nullable>
-ColumnVector* DatetimeExtractUncheckedOperator<unit, argument_nullable>::applyToValueAccessor(
-    ValueAccessor *accessor,
-    const attribute_id argument_attr_id) const {
-  return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-      accessor,
-      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-    std::unique_ptr<NativeColumnVector> result(
-        new NativeColumnVector(LongType::Instance(argument_nullable), accessor->getNumTuples()));
-    accessor->beginIteration();
-    while (accessor->next()) {
-      const DatetimeLit *datetime_arg =
-          static_cast<const DatetimeLit*>(
-              accessor->template getUntypedValue<argument_nullable>(argument_attr_id));
-      if (argument_nullable && (datetime_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<int64_t*>(result->getPtrForDirectWrite())
-            = this->dateExtract(*datetime_arg);
-      }
-    }
-    return result.release();
-  });
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-ColumnVector* DateExtractUncheckedOperator<unit, argument_nullable>::applyToValueAccessor(
-    ValueAccessor *accessor,
-    const attribute_id argument_attr_id) const {
-  return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-      accessor,
-      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-    std::unique_ptr<NativeColumnVector> result(
-        new NativeColumnVector(IntType::Instance(argument_nullable), accessor->getNumTuples()));
-    accessor->beginIteration();
-    while (accessor->next()) {
-      const DateLit *date_arg =
-          static_cast<const DateLit*>(
-              accessor->template getUntypedValue<argument_nullable>(argument_attr_id));
-      if (argument_nullable && (date_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<int32_t*>(result->getPtrForDirectWrite())
-            = this->dateExtract(*date_arg);
-      }
-    }
-    return result.release();
-  });
-}
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-template <DateExtractUnit unit, bool argument_nullable>
-ColumnVector* DatetimeExtractUncheckedOperator<unit, argument_nullable>::applyToValueAccessorForJoin(
-    ValueAccessor *accessor,
-    const bool use_left_relation,
-    const attribute_id argument_attr_id,
-    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
-  std::unique_ptr<NativeColumnVector> result(
-      new NativeColumnVector(LongType::Instance(argument_nullable), joined_tuple_ids.size()));
-  return InvokeOnValueAccessorNotAdapter(
-      accessor,
-      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-    for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-      const DatetimeLit *datetime_arg =
-          static_cast<const DatetimeLit*>(
-              accessor->template getUntypedValueAtAbsolutePosition<argument_nullable>(
-                  argument_attr_id,
-                  use_left_relation ? joined_pair.first : joined_pair.second));
-      if (argument_nullable && (datetime_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<int64_t*>(result->getPtrForDirectWrite())
-            = this->dateExtract(*datetime_arg);
-      }
-    }
-    return result.release();
-  });
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-ColumnVector* DateExtractUncheckedOperator<unit, argument_nullable>::applyToValueAccessorForJoin(
-    ValueAccessor *accessor,
-    const bool use_left_relation,
-    const attribute_id argument_attr_id,
-    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
-  std::unique_ptr<NativeColumnVector> result(
-      new NativeColumnVector(IntType::Instance(argument_nullable), joined_tuple_ids.size()));
-  return InvokeOnValueAccessorNotAdapter(
-      accessor,
-      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-    for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-      const DateLit *date_arg =
-          static_cast<const DateLit*>(
-              accessor->template getUntypedValueAtAbsolutePosition<argument_nullable>(
-                  argument_attr_id,
-                  use_left_relation ? joined_pair.first : joined_pair.second));
-      if (argument_nullable && (date_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<int32_t*>(result->getPtrForDirectWrite())
-            = this->dateExtract(*date_arg);
-      }
-    }
-    return result.release();
-  });
-}
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-template <DateExtractUnit unit, bool argument_nullable>
-inline int64_t
-DatetimeExtractUncheckedOperator<unit, argument_nullable>::dateExtract(
-    const DatetimeLit &argument) const {
-  switch (unit) {
-    case DateExtractUnit::kYear:
-      return argument.yearField();
-    case DateExtractUnit::kMonth:
-      return argument.monthField();
-    case DateExtractUnit::kDay:
-      return argument.dayField();
-    case DateExtractUnit::kHour:
-      return argument.hourField();
-    case DateExtractUnit::kMinute:
-      return argument.minuteField();
-    case DateExtractUnit::kSecond:
-      return argument.secondField();
-    default:
-      FATAL_ERROR("Unsupported DateExtractUnit in DatetimeExtractUncheckedOperator::dateExtract.");
-  }
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-inline int32_t DateExtractUncheckedOperator<unit, argument_nullable>::dateExtract(const DateLit &argument) const {
-  switch (unit) {
-    case DateExtractUnit::kYear:
-      return argument.yearField();
-    case DateExtractUnit::kMonth:
-      return argument.monthField();
-    default:
-      FATAL_ERROR("Unsupported DateExtractUnit in DateExtractUncheckedOperator::dateExtract.");
-  }
-}
-
-const DateExtractOperation& DateExtractOperation::Instance(const DateExtractUnit unit) {
-  switch (unit) {
-    case DateExtractUnit::kYear: {
-      static DateExtractOperation instance(DateExtractUnit::kYear);
-      return instance;
-    }
-    case DateExtractUnit::kMonth: {
-      static DateExtractOperation instance(DateExtractUnit::kMonth);
-      return instance;
-    }
-    case DateExtractUnit::kDay: {
-      static DateExtractOperation instance(DateExtractUnit::kDay);
-      return instance;
-    }
-    case DateExtractUnit::kHour: {
-      static DateExtractOperation instance(DateExtractUnit::kHour);
-      return instance;
-    }
-    case DateExtractUnit::kMinute: {
-      static DateExtractOperation instance(DateExtractUnit::kMinute);
-      return instance;
-    }
-    case DateExtractUnit::kSecond: {
-      static DateExtractOperation instance(DateExtractUnit::kSecond);
-      return instance;
-    }
-    default: {
-      FATAL_ERROR("Unsupported DateExtractUnit in DateExtractOperation::Instance.");
+struct DateExtractFunctor : public UnaryFunctor<DateType, IntType> {
+  template <typename DateExtractUnitT>
+  inline int apply(const DateLit &argument) const {
+    switch (DateExtractUnitT::value) {
+      case DateExtractUnit::kYear:
+        return argument.yearField();
+      case DateExtractUnit::kMonth:
+        return argument.monthField();
+      case DateExtractUnit::kDay:
+        return argument.dayField();
+      default:
+        DLOG(FATAL) << "Unsupported DateExtractUnit in DateExtractFunctor::apply";
     }
   }
-}
-
-serialization::UnaryOperation DateExtractOperation::getProto() const {
-  serialization::UnaryOperation proto;
-  proto.set_operation_id(serialization::UnaryOperation::DATE_EXTRACT);
-
-  switch (unit_) {
-    case DateExtractUnit::kYear:
-      proto.SetExtension(serialization::DateExtractOperation::unit, serialization::DateExtractOperation::YEAR);
-      break;
-    case DateExtractUnit::kMonth:
-      proto.SetExtension(serialization::DateExtractOperation::unit, serialization::DateExtractOperation::MONTH);
-      break;
-    case DateExtractUnit::kDay:
-      proto.SetExtension(serialization::DateExtractOperation::unit, serialization::DateExtractOperation::DAY);
-      break;
-    case DateExtractUnit::kHour:
-      proto.SetExtension(serialization::DateExtractOperation::unit, serialization::DateExtractOperation::HOUR);
-      break;
-    case DateExtractUnit::kMinute:
-      proto.SetExtension(serialization::DateExtractOperation::unit, serialization::DateExtractOperation::MINUTE);
-      break;
-    case DateExtractUnit::kSecond:
-      proto.SetExtension(serialization::DateExtractOperation::unit, serialization::DateExtractOperation::SECOND);
-      break;
-    default:
-      FATAL_ERROR("Unsupported DateExtractUnit in DateExtractOperation::getProto.");
-  }
-
-  return proto;
-}
-
-std::string DateExtractOperation::getName() const {
-  std::string name(kUnaryOperationNames[static_cast<std::size_t>(operation_id_)]);
-  name.push_back('(');
-  switch (unit_) {
-    case DateExtractUnit::kYear:
-      name.append("YEAR)");
-      break;
-    case DateExtractUnit::kMonth:
-      name.append("MONTH)");
-      break;
-    case DateExtractUnit::kDay:
-      name.append("DAY)");
-      break;
-    case DateExtractUnit::kHour:
-      name.append("HOUR)");
-      break;
-    case DateExtractUnit::kMinute:
-      name.append("MINUTE)");
-      break;
-    case DateExtractUnit::kSecond:
-      name.append("SECOND)");
-      break;
-    default:
-      name.append("UNKNOWN)");
-      break;
-  }
-  return name;
-}
-
-const Type* DateExtractOperation::pushDownTypeHint(const Type *type_hint) const {
-  if (type_hint == nullptr) {
-    return nullptr;
-  }
+};
 
-  if (type_hint->getTypeID() == kLong) {
-    switch (unit_) {
-      case DateExtractUnit::kYear:  // Fall through.
+struct DatetimeExtractFunctor : public UnaryFunctor<DatetimeType, IntType> {
+  template <typename DateExtractUnitT>
+  inline std::int64_t apply(const DatetimeLit &argument) const {
+    switch (DateExtractUnitT::value) {
+      case DateExtractUnit::kYear:
+        return argument.yearField();
       case DateExtractUnit::kMonth:
-        // There are two possibilities for the return type, based on whether we
-        // have Datetime or Date as the underlying date implementation.
-        return nullptr;
-      case DateExtractUnit::kDay:  // Fall through.
+        return argument.monthField();
+      case DateExtractUnit::kDay:
+        return argument.dayField();
       case DateExtractUnit::kHour:
+        return argument.hourField();
       case DateExtractUnit::kMinute:
+        return argument.minuteField();
       case DateExtractUnit::kSecond:
-        return &TypeFactory::GetType(kDatetime, type_hint->isNullable());
+        return argument.secondField();
       default:
-        return nullptr;
+        DLOG(FATAL) << "Unsupported DateExtractUnit in DatetimeExtractFunctor::apply";
+    }
+  }
+};
+
+const std::map<std::string, DateExtractUnit> DateExtractOperation::kNameToUnitMap = {
+    { "year",   DateExtractUnit::kYear },
+    { "month",  DateExtractUnit::kMonth },
+    { "day",    DateExtractUnit::kDay },
+    { "hour",   DateExtractUnit::kHour },
+    { "minute", DateExtractUnit::kMinute },
+    { "second", DateExtractUnit::kSecond }
+};
+
+UncheckedUnaryOperator* DateExtractOperation::makeUncheckedUnaryOperator(
+    const Type &type,
+    const std::vector<TypedValue> &static_arguments) const {
+  DCHECK_EQ(1u, static_arguments.size());
+
+  const DateExtractUnit unit = parseUnit(static_arguments.front());
+  const Type *result_type = getResultType(type, static_arguments);
+
+  if (type.getTypeID() == kDate) {
+    switch (unit) {
+      case DateExtractUnit::kYear:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DateExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kYear>>(
+                 type, *result_type);
+      case DateExtractUnit::kMonth:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DateExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kMonth>>(
+                 type, *result_type);
+      case DateExtractUnit::kDay:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DateExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kDay>>(
+                 type, *result_type);
+      default:
+        LOG(FATAL) << "Unsupported DateExtractUnit for DateType in "
+                   << "DateExtractOperation::makeUncheckedUnaryOperator";
     }
   } else {
-    return nullptr;
-  }
-}
-
-TypedValue DateExtractOperation::applyToChecked(const TypedValue &argument,
-                                                const Type &argument_type) const {
-  if (((argument.getTypeID() != TypeID::kDatetime) ||
-       (argument_type.getTypeID() != TypeID::kDatetime)) &&
-      ((argument.getTypeID() != TypeID::kDate) ||
-       (argument_type.getTypeID() != TypeID::kDate))) {
-    LOG(FATAL) << "UnaryOperation " << getName() << " is only applicable to Type "
-               << kTypeNames[TypeID::kDatetime] << ", but applyToChecked() was "
-               << "called with 'argument' of Type " << kTypeNames[argument.getTypeID()]
-               << " and explicit 'argument_type' parameter of "
-               << argument_type.getName();
-  }
-
-  if (argument.isNull()) {
-    if (argument.getTypeID() == TypeID::kDatetime) {
-      return TypedValue(kLong);
-    } else {
-      // argument type is kDate.
-      DCHECK_EQ(TypeID::kDate, argument.getTypeID());
-      return TypedValue(kInt);
-    }
-  }
-
-  switch (unit_) {
-    case DateExtractUnit::kYear: {
-      if (argument.getTypeID() == TypeID::kDatetime) {
-        return TypedValue(argument.getLiteral<DatetimeLit>().yearField());
-      } else {
-        // argument type is kDate.
-        DCHECK_EQ(TypeID::kDate, argument.getTypeID());
-        return TypedValue(argument.getLiteral<DateLit>().yearField());
-      }
-    }
-    case DateExtractUnit::kMonth: {
-      if (argument.getTypeID() == TypeID::kDatetime) {
-        return TypedValue(argument.getLiteral<DatetimeLit>().monthField());
-      } else {
-        // argument type is kDate.
-        DCHECK_EQ(TypeID::kDate, argument.getTypeID());
-        return TypedValue(argument.getLiteral<DateLit>().monthField());
-      }
-    }
-    case DateExtractUnit::kDay:
-      return TypedValue(argument.getLiteral<DatetimeLit>().dayField());
-    case DateExtractUnit::kHour:
-      return TypedValue(argument.getLiteral<DatetimeLit>().hourField());
-    case DateExtractUnit::kMinute:
-      return TypedValue(argument.getLiteral<DatetimeLit>().minuteField());
-    case DateExtractUnit::kSecond:
-      return TypedValue(argument.getLiteral<DatetimeLit>().secondField());
-    default: {
-      LOG(FATAL) << "Unsupported DateExtractUnit in DateExtractOperation::applyToChecked().";
-    }
-  }
-}
-
-UncheckedUnaryOperator* DateExtractOperation::makeUncheckedUnaryOperatorForTypeHelper(const Type &type) const {
-  switch (unit_) {
-    case DateExtractUnit::kYear: {
-      if (type.getTypeID() == TypeID::kDatetime) {
-        if (type.isNullable()) {
-          return new DatetimeExtractUncheckedOperator<DateExtractUnit::kYear, true>();
-        } else {
-          return new DatetimeExtractUncheckedOperator<DateExtractUnit::kYear, false>();
-        }
-      } else {
-        DCHECK_EQ(TypeID::kDate, type.getTypeID());
-        // type is kDate.
-        if (type.isNullable()) {
-          return new DateExtractUncheckedOperator<DateExtractUnit::kYear, true>();
-        } else {
-          return new DateExtractUncheckedOperator<DateExtractUnit::kYear, false>();
-        }
-      }
-    }
-    case DateExtractUnit::kMonth: {
-      if (type.getTypeID() == TypeID::kDatetime) {
-        if (type.isNullable()) {
-          return new DatetimeExtractUncheckedOperator<DateExtractUnit::kMonth, true>();
-        } else {
-          return new DatetimeExtractUncheckedOperator<DateExtractUnit::kMonth, false>();
-        }
-      } else {
-        // type is kDate.
-        DCHECK_EQ(TypeID::kDate, type.getTypeID());
-        if (type.isNullable()) {
-          return new DateExtractUncheckedOperator<DateExtractUnit::kMonth, true>();
-        } else {
-          return new DateExtractUncheckedOperator<DateExtractUnit::kMonth, false>();
-        }
-      }
-    }
-    case DateExtractUnit::kDay:
-      if (type.isNullable()) {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kDay, true>();
-      } else {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kDay, false>();
-      }
-    case DateExtractUnit::kHour:
-      if (type.isNullable()) {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kHour, true>();
-      } else {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kHour, false>();
-      }
-    case DateExtractUnit::kMinute:
-      if (type.isNullable()) {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kMinute, true>();
-      } else {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kMinute, false>();
-      }
-    case DateExtractUnit::kSecond:
-      if (type.isNullable()) {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kSecond, true>();
-      } else {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kSecond, false>();
-      }
-    default:
-      FATAL_ERROR("Unsupported DateExtractUnit in DateExtractOperation::makeUncheckedUnaryOperatorForTypeHelper.");
-  }
-}
-
-const Type* DateExtractOperation::resultTypeForArgumentType(const Type &type) const {
-  if (canApplyToType(type)) {
-    if (type.getTypeID() == kDatetime) {
-      return &LongType::Instance(type.isNullable());
-    } else {
-      DCHECK_EQ(kDate, type.getTypeID());
-      return &IntType::Instance(type.isNullable());
+    switch (unit) {
+      case DateExtractUnit::kYear:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DatetimeExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kYear>>(
+                 type, *result_type);
+      case DateExtractUnit::kMonth:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DatetimeExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kMonth>>(
+                 type, *result_type);
+      case DateExtractUnit::kDay:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DatetimeExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kDay>>(
+                 type, *result_type);
+      case DateExtractUnit::kHour:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DatetimeExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kHour>>(
+                 type, *result_type);
+      case DateExtractUnit::kMinute:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DatetimeExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kMinute>>(
+                 type, *result_type);
+      case DateExtractUnit::kSecond:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DatetimeExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kSecond>>(
+                 type, *result_type);
+      default:
+        LOG(FATAL) << "Unsupported DateExtractUnit for DatetimeType in "
+                   << "DateExtractOperation::makeUncheckedUnaryOperator";
     }
-  } else {
-    return nullptr;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/DateExtractOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/DateExtractOperation.hpp b/types/operations/unary_operations/DateExtractOperation.hpp
index f8c091b..577e924 100644
--- a/types/operations/unary_operations/DateExtractOperation.hpp
+++ b/types/operations/unary_operations/DateExtractOperation.hpp
@@ -23,109 +23,25 @@
 #include <cstdint>
 #include <string>
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-#include <utility>
-#include <vector>
-
-#include "storage/StorageBlockInfo.hpp"
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-#include "catalog/CatalogTypedefs.hpp"
+#include "types/IntType.hpp"
 #include "types/LongType.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
-#include "types/operations/Operation.pb.h"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
 #include "utility/Macros.hpp"
+#include "utility/StringUtil.hpp"
 
 namespace quickstep {
 
-class ColumnVector;
-class ValueAccessor;
-
-struct DatetimeLit;
-
-/** \addtogroup Types
- *  @{
- */
-
 enum class DateExtractUnit {
   kYear = 0,
   kMonth,
   kDay,
   kHour,
   kMinute,
-  kSecond
-};
-
-/**
- * @brief UncheckedUnaryOperator for Datetime Extract.
- */
-template <DateExtractUnit unit, bool argument_nullable>
-class DatetimeExtractUncheckedOperator : public UncheckedUnaryOperator {
- public:
-  DatetimeExtractUncheckedOperator()
-      : UncheckedUnaryOperator() {}
-
-  TypedValue applyToTypedValue(const TypedValue &argument) const override;
-
-  TypedValue applyToDataPtr(const void *argument) const override;
-
-  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
-                                     const attribute_id argument_attr_id) const override;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  ColumnVector* applyToValueAccessorForJoin(
-      ValueAccessor *accessor,
-      const bool use_left_relation,
-      const attribute_id argument_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
- private:
-  inline std::int64_t dateExtract(const DatetimeLit &argument) const;
-
-  DISALLOW_COPY_AND_ASSIGN(DatetimeExtractUncheckedOperator);
-};
-
-/**
- * @brief UncheckedUnaryOperator for Date Extract.
- */
-template <DateExtractUnit unit, bool argument_nullable>
-class DateExtractUncheckedOperator : public UncheckedUnaryOperator {
- public:
-  DateExtractUncheckedOperator()
-      : UncheckedUnaryOperator() {}
-
-  TypedValue applyToTypedValue(const TypedValue &argument) const override;
-
-  TypedValue applyToDataPtr(const void *argument) const override;
-
-  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
-                                     const attribute_id argument_attr_id) const override;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  ColumnVector* applyToValueAccessorForJoin(
-      ValueAccessor *accessor,
-      const bool use_left_relation,
-      const attribute_id argument_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
- private:
-  inline std::int32_t dateExtract(const DateLit &argument) const;
-
-  DISALLOW_COPY_AND_ASSIGN(DateExtractUncheckedOperator);
+  kSecond,
+  kInvalid
 };
 
 /**
@@ -133,55 +49,82 @@ class DateExtractUncheckedOperator : public UncheckedUnaryOperator {
  */
 class DateExtractOperation : public UnaryOperation {
  public:
-  /**
-   * @brief Get a reference to the singleton instance of this Operation for a
-   *        particular DateExtractUnit.
-   *
-   * @param unit The date unit to extract.
-   * @return A reference to the singleton instance of this Operation for the
-   *         specified DateExtractUnit.
-   **/
-  static const DateExtractOperation& Instance(const DateExtractUnit unit);
-
-  serialization::UnaryOperation getProto() const override;
-
-  std::string getName() const override;
-
-  bool canApplyToType(const Type &type) const override {
-    return type.getTypeID() == TypeID::kDatetime || type.getTypeID() == kDate;
-  }
-
-  const Type* resultTypeForArgumentType(const Type &type) const override;
+  DateExtractOperation() {}
 
-  const Type* fixedNullableResultType() const override {
-    return nullptr;
+  std::string getName() const override {
+    return "DateExtract";
   }
 
-  bool resultTypeIsPlausible(const Type &result_type) const override {
-    return result_type.getTypeID() == kLong || result_type.getTypeID() == kInt;
+  std::string getShortName() const override {
+    return "DateExtract";
   }
 
-  const Type* pushDownTypeHint(const Type *type_hint) const override;
-
-  TypedValue applyToChecked(const TypedValue &argument,
-                            const Type &argument_type) const override;
+  std::vector<OperationSignaturePtr> getSignatures() const override {
+    const std::vector<TypeID> unit_carrier = { kVarChar };
+    return {
+        OperationSignature::Create(getName(), {kDate}, unit_carrier),
+        OperationSignature::Create(getName(), {kDatetime}, unit_carrier)
+    };
+  }
 
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorForType(const Type &type) const override {
-    DCHECK(canApplyToType(type));
+  bool canApplyTo(const Type &type,
+                  const std::vector<TypedValue> &static_arguments,
+                  std::string *message) const override {
+    DCHECK(type.getTypeID() == kDate || type.getTypeID() == kDatetime);
+    DCHECK_EQ(1u, static_arguments.size());
+
+    const DateExtractUnit unit = parseUnit(static_arguments.front());
+    switch (unit) {
+      case DateExtractUnit::kYear:  // Fall through
+      case DateExtractUnit::kMonth:
+      case DateExtractUnit::kDay:
+        return true;
+      case DateExtractUnit::kHour:  // Fall through
+      case DateExtractUnit::kMinute:
+      case DateExtractUnit::kSecond:
+        if (type.getTypeID() == kDate) {
+          *message = "Invalid extraction unit for argument of DATE type";
+        } else {
+          return true;
+        }
+      default:
+        *message = "Invalid extraction unit for DateExtract";
+        return false;
+    }
+  }
 
-    return makeUncheckedUnaryOperatorForTypeHelper(type);
+  const Type* getResultType(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(UnaryOperation::canApplyTo(type, static_arguments));
+    if (type.getTypeID() == kDatetime) {
+      return &LongType::Instance(type.isNullable());
+    } else {
+      DCHECK_EQ(kDate, type.getTypeID());
+      return &IntType::Instance(type.isNullable());
+    }
   }
 
- private:
-  explicit DateExtractOperation(const DateExtractUnit unit)
-      : UnaryOperation(UnaryOperationID::kDateExtract),
-        unit_(unit) {}
+  UncheckedUnaryOperator* makeUncheckedUnaryOperator(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override;
 
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorForTypeHelper(const Type &type) const;
+ private:
+  static DateExtractUnit parseUnit(const TypedValue &unit_arg) {
+    DCHECK(unit_arg.getTypeID() == kVarChar);
+    const std::string unit_str =
+        ToLower(std::string(static_cast<const char*>(unit_arg.getOutOfLineData())));
+
+    auto it = kNameToUnitMap.find(unit_str);
+    if (it != kNameToUnitMap.end()) {
+      return it->second;
+    } else {
+      return DateExtractUnit::kInvalid;
+    }
+  }
 
-  const DateExtractUnit unit_;
+  static const std::map<std::string, DateExtractUnit> kNameToUnitMap;
 
- private:
   DISALLOW_COPY_AND_ASSIGN(DateExtractOperation);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/NumericCastOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/NumericCastOperation.hpp b/types/operations/unary_operations/NumericCastOperation.hpp
deleted file mode 100644
index 1c5e3d4..0000000
--- a/types/operations/unary_operations/NumericCastOperation.hpp
+++ /dev/null
@@ -1,313 +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_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_
-
-#include <cstddef>
-#include <string>
-#include <utility>
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-#include <vector>
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
-#include "types/DoubleType.hpp"
-#include "types/FloatType.hpp"
-#include "types/IntType.hpp"
-#include "types/LongType.hpp"
-#include "types/Type.hpp"
-#include "types/TypeID.hpp"
-#include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
-#include "types/operations/Operation.pb.h"
-#include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
-#include "utility/Macros.hpp"
-#include "utility/PtrMap.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-// TODO(quickstep-team): Support CAST on Datetime, YearMonthInterval, DatetimeInterval, VarChar, Char.
-//                       Currently we only need cast operations on numeric values.
-
-/**
- * @brief UncheckedUnaryOperator for CAST.
- */
-template <class SourceType, bool source_nullability, class TargetType, bool target_nullability>
-class UncheckedNumericCastOperator : public UncheckedUnaryOperator {
- public:
-  UncheckedNumericCastOperator()
-      : UncheckedUnaryOperator(),
-        target_type_(TargetType::Instance(target_nullability)) {
-  }
-
-  TypedValue applyToTypedValue(const TypedValue &argument) const override {
-    if (source_nullability && argument.isNull()) {
-      return TypedValue(TargetType::kStaticTypeID);
-    }
-
-    return TypedValue(static_cast<typename TargetType::cpptype>(
-        argument.getLiteral<typename SourceType::cpptype>()));
-  }
-
-  TypedValue applyToDataPtr(const void *argument) const override {
-    if (source_nullability && argument == nullptr) {
-      return TypedValue(TargetType::kStaticTypeID);
-    }
-
-    return TypedValue(
-        static_cast<const typename TargetType::cpptype>(
-            *static_cast<const typename SourceType::cpptype*>(argument)));
-  }
-
-  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override {
-    DCHECK(NativeColumnVector::UsableForType(target_type_));
-    DCHECK(argument.isNative());
-    const NativeColumnVector &native_argument = static_cast<const NativeColumnVector&>(argument);
-    NativeColumnVector *result = new NativeColumnVector(
-        target_type_,
-        native_argument.size());
-    for (std::size_t pos = 0;
-         pos < native_argument.size();
-         ++pos) {
-      const typename SourceType::cpptype *scalar_arg
-          = static_cast<const typename SourceType::cpptype*>(
-              native_argument.getUntypedValue<source_nullability>(pos));
-      if (source_nullability && (scalar_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<typename TargetType::cpptype*>(result->getPtrForDirectWrite())
-            = static_cast<typename TargetType::cpptype>(*scalar_arg);
-      }
-    }
-    return result;
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
-                                     const attribute_id argument_attr_id) const override {
-    DCHECK(NativeColumnVector::UsableForType(target_type_));
-    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        accessor,
-        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-      NativeColumnVector *result = new NativeColumnVector(
-          target_type_,
-          accessor->getNumTuples());
-      accessor->beginIteration();
-      while (accessor->next()) {
-        const typename SourceType::cpptype *scalar_arg
-            = static_cast<const typename SourceType::cpptype*>(
-                accessor->template getUntypedValue<source_nullability>(argument_attr_id));
-        if (source_nullability && (scalar_arg == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename TargetType::cpptype*>(result->getPtrForDirectWrite())
-              = static_cast<typename TargetType::cpptype>(*scalar_arg);
-        }
-      }
-      return result;
-    });
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  ColumnVector* applyToValueAccessorForJoin(
-      ValueAccessor *accessor,
-      const bool use_left_relation,
-      const attribute_id argument_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    DCHECK(NativeColumnVector::UsableForType(target_type_));
-    NativeColumnVector *result = new NativeColumnVector(target_type_,
-                                                        joined_tuple_ids.size());
-    InvokeOnValueAccessorNotAdapter(
-        accessor,
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-      for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-        const typename SourceType::cpptype *scalar_arg
-            = static_cast<const typename SourceType::cpptype*>(
-                accessor->template getUntypedValueAtAbsolutePosition<source_nullability>(
-                    argument_attr_id,
-                    use_left_relation ? joined_pair.first : joined_pair.second));
-        if (source_nullability && (scalar_arg == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename TargetType::cpptype*>(result->getPtrForDirectWrite())
-              = static_cast<typename TargetType::cpptype>(*scalar_arg);
-        }
-      }
-    });
-    return result;
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
- private:
-  const Type &target_type_;
-
-  DISALLOW_COPY_AND_ASSIGN(UncheckedNumericCastOperator);
-};
-
-/**
- * @brief UnaryOperation for CAST.
- */
-class NumericCastOperation : public UnaryOperation {
- public:
-  serialization::UnaryOperation getProto() const override {
-    serialization::UnaryOperation proto;
-    proto.set_operation_id(serialization::UnaryOperation::CAST);
-    proto.MutableExtension(serialization::CastOperation::target_type)
-        ->CopyFrom(getTargetType().getProto());
-
-    return proto;
-  }
-
-  /**
-   * @brief Get a reference to the singleton instance of this Operation.
-   *
-   * @param target_type The target type to coerce input values to.
-   * @return A reference to the singleton instance of this Operation.
-   **/
-  static const NumericCastOperation& Instance(const Type &target_type) {
-    static PtrMap<const Type*, NumericCastOperation> instance_map;
-    PtrMap<const Type*, NumericCastOperation>::iterator instance_map_it =
-        instance_map.find(&target_type);
-    if (instance_map_it == instance_map.end()) {
-      instance_map_it = instance_map.insert(&target_type,
-                                            new NumericCastOperation(target_type)).first;
-    }
-    return *(instance_map_it->second);
-  }
-
-  /**
-   * @return The target type for coercion.
-   */
-  const Type& getTargetType() const {
-    return target_type_;
-  }
-
-  std::string getName() const override {
-    return std::string(kUnaryOperationNames[static_cast<std::size_t>(operation_id_)])
-        .append("(")
-        .append(target_type_.getName())
-        .append(")");
-  }
-
-  bool canApplyToType(const Type &type) const override {
-    return target_type_.isCoercibleFrom(type);
-  }
-
-  const Type* resultTypeForArgumentType(const Type &type) const override {
-    if (canApplyToType(type)) {
-      return &target_type_;
-    } else {
-      return nullptr;
-    }
-  }
-
-  const Type* fixedNullableResultType() const override {
-    return &target_type_.getNullableVersion();
-  }
-
-  bool resultTypeIsPlausible(const Type &result_type) const override {
-    return result_type.equals(target_type_)
-           || result_type.equals(target_type_.getNullableVersion());
-  }
-
-  const Type* pushDownTypeHint(const Type *type_hint) const override {
-    return &target_type_;
-  }
-
-  TypedValue applyToChecked(const TypedValue &argument,
-                            const Type &argument_type) const override {
-    return target_type_.coerceValue(argument, argument_type);
-  }
-
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorForType(const Type &type) const override {
-    switch (type.getTypeID()) {
-      case kInt:
-        return makeUncheckedUnaryOperatorHelperForSourceNullability<IntType>(type);
-      case kLong:
-        return makeUncheckedUnaryOperatorHelperForSourceNullability<LongType>(type);
-      case kFloat:
-        return makeUncheckedUnaryOperatorHelperForSourceNullability<FloatType>(type);
-      case kDouble:
-        return makeUncheckedUnaryOperatorHelperForSourceNullability<DoubleType>(type);
-      default:
-        FATAL_ERROR("Unhandled type " << kTypeNames[type.getTypeID()]);
-    }
-  }
-
- private:
-  explicit NumericCastOperation(const Type &target_type)
-      : UnaryOperation(UnaryOperationID::kCast),
-        target_type_(target_type) {}
-
-  template <class SourceType>
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorHelperForSourceNullability(const Type &type) const {
-    if (type.isNullable()) {
-      return makeUncheckedUnaryOperatorHelperForTargetType<SourceType, true>();
-    } else {
-      return makeUncheckedUnaryOperatorHelperForTargetType<SourceType, false>();
-    }
-  }
-
-  template <class SourceType, bool source_nullability>
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorHelperForTargetType() const {
-    switch (target_type_.getTypeID()) {
-      case kInt:
-        return makeUncheckedUnaryOperatorHelperForTargetNullability<SourceType, source_nullability, IntType>();
-      case kLong:
-        return makeUncheckedUnaryOperatorHelperForTargetNullability<SourceType, source_nullability, LongType>();
-      case kFloat:
-        return makeUncheckedUnaryOperatorHelperForTargetNullability<SourceType, source_nullability, FloatType>();
-      case kDouble:
-        return makeUncheckedUnaryOperatorHelperForTargetNullability<SourceType, source_nullability, DoubleType>();
-      default:
-        FATAL_ERROR("Unhandled type " << kTypeNames[target_type_.getTypeID()]);
-    }
-  }
-
-  template <class SourceType, bool source_nullability, class TargetType>
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorHelperForTargetNullability() const {
-    if (target_type_.isNullable()) {
-      return new UncheckedNumericCastOperator<SourceType, source_nullability, TargetType, true>();
-    } else {
-      return new UncheckedNumericCastOperator<SourceType, source_nullability, TargetType, false>();
-    }
-  }
-
-  const Type& target_type_;
-
-  DISALLOW_COPY_AND_ASSIGN(NumericCastOperation);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/SubstringOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/SubstringOperation.cpp b/types/operations/unary_operations/SubstringOperation.cpp
index 84f1c8d..1cc8912 100644
--- a/types/operations/unary_operations/SubstringOperation.cpp
+++ b/types/operations/unary_operations/SubstringOperation.cpp
@@ -19,198 +19,48 @@
 
 #include "types/operations/unary_operations/SubstringOperation.hpp"
 
-#include <algorithm>
-#include <tuple>
-#include <utility>
+#include <cstddef>
 #include <vector>
 
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
-#include "types/containers/ColumnVectorUtil.hpp"
-#include "types/operations/Operation.pb.h"
-#include "types/port/strnlen.hpp"
-#include "utility/TemplateUtil.hpp"
+#include "utility/meta/Dispatchers.hpp"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
-serialization::UnaryOperation SubstringOperation::getProto() const {
-  serialization::UnaryOperation proto;
-  proto.set_operation_id(serialization::UnaryOperation::SUBSTRING);
-  proto.SetExtension(serialization::SubstringOperation::start_position,
-                     start_position_);
-  proto.SetExtension(serialization::SubstringOperation::substring_length,
-                     substring_length_);
-  return proto;
-}
+UncheckedUnaryOperator* SubstringOperation::makeUncheckedUnaryOperator(
+    const Type &type,
+    const std::vector<TypedValue> &static_arguments) const {
+  DCHECK(UnaryOperation::canApplyTo(type, static_arguments));
 
-UncheckedUnaryOperator* SubstringOperation::makeUncheckedUnaryOperatorForType(
-    const Type &type) const {
-  DCHECK(type.getSuperTypeID() == Type::kAsciiString);
+  std::size_t start_position;
+  std::size_t substring_length;
+  ExtractStaticArguments(static_arguments, &start_position, &substring_length);
 
   const std::size_t input_maximum_length =
-      static_cast<const AsciiStringSuperType&>(type).getStringLength();
+      type.getTypeID() == kChar
+          ? static_cast<const CharType&>(type).getStringLength()
+          : static_cast<const VarCharType&>(type).getStringLength();
   const bool input_null_terminated = (type.getTypeID() == TypeID::kVarChar);
 
-  const Type *result_type = resultTypeForArgumentType(type);
+  const Type *result_type = getResultType(type, static_arguments);
   DCHECK(result_type != nullptr);
 
-  return CreateBoolInstantiatedInstance<SubstringUncheckedOperator, UncheckedUnaryOperator>(
-      std::forward_as_tuple(start_position_,
-                            computeMaximumSubstringLength(type),
-                            input_maximum_length,
-                            *result_type),
-      input_null_terminated, type.isNullable());
-}
-
-template <bool null_terminated, bool input_nullable>
-inline void SubstringUncheckedOperator<null_terminated, input_nullable>
-    ::computeSubstring(const char *input,
-                       char *output) const {
-  std::size_t string_length =
-      (null_terminated ? strlen(input) : strnlen(input, maximum_input_length_));
-
-  if (start_position_ >= string_length) {
-    *output = '\0';
-    return;
-  }
-
-  const std::size_t actual_substring_length =
-      std::min(string_length - start_position_, substring_length_);
-  std::memcpy(output, input + start_position_, actual_substring_length);
-
-  if (actual_substring_length < substring_length_) {
-    output[actual_substring_length] = '\0';
-  }
-}
-
-template <bool null_terminated, bool input_nullable>
-TypedValue SubstringUncheckedOperator<null_terminated,
-                                      input_nullable>
-    ::applyToTypedValue(const TypedValue& argument) const {
-  if (input_nullable && argument.isNull()) {
-    return TypedValue(result_type_.getTypeID());
-  }
-
-  char *output_ptr = static_cast<char*>(std::malloc(substring_length_));
-  computeSubstring(static_cast<const char*>(argument.getOutOfLineData()),
-                   output_ptr);
-
-  return TypedValue::CreateWithOwnedData(result_type_.getTypeID(),
-                                         output_ptr,
-                                         substring_length_);
-}
-
-template <bool null_terminated, bool input_nullable>
-TypedValue SubstringUncheckedOperator<null_terminated,
-                                      input_nullable>
-    ::applyToDataPtr(const void *argument) const {
-  if (input_nullable && argument == nullptr) {
-    return TypedValue(result_type_.getTypeID());
-  }
-
-  char *output_ptr = static_cast<char*>(std::malloc(substring_length_));
-  computeSubstring(static_cast<const char*>(argument),
-                   output_ptr);
-
-  return TypedValue::CreateWithOwnedData(result_type_.getTypeID(),
-                                         output_ptr,
-                                         substring_length_);
-}
-
-template <bool null_terminated, bool input_nullable>
-ColumnVector* SubstringUncheckedOperator<null_terminated,
-                                         input_nullable>
-    ::applyToColumnVector(const ColumnVector &argument) const {
-  return InvokeOnColumnVector(
-      argument,
-      [&](const auto &column_vector) -> ColumnVector* {  // NOLINT(build/c++11)
-    NativeColumnVector *result =
-        new NativeColumnVector(result_type_, column_vector.size());
-
-    for (std::size_t cv_pos = 0;
-         cv_pos < column_vector.size();
-         ++cv_pos) {
-      const char *input_ptr = static_cast<const char *>(
-          column_vector.template getUntypedValue<input_nullable>(cv_pos));
-
-      if (input_nullable && input_ptr == nullptr) {
-        result->appendNullValue();
-      } else {
-        this->computeSubstring(input_ptr,
-                               static_cast<char *>(result->getPtrForDirectWrite()));
-      }
-    }
-    return result;
-  });
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-template <bool null_terminated, bool input_nullable>
-ColumnVector* SubstringUncheckedOperator<null_terminated,
-                                         input_nullable>
-    ::applyToValueAccessor(ValueAccessor *accessor,
-                           const attribute_id argument_attr_id) const {
-  return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-      accessor,
-      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-    NativeColumnVector *result =
-        new NativeColumnVector(result_type_, accessor->getNumTuples());
-
-    accessor->beginIteration();
-    while (accessor->next()) {
-      const char *input_ptr = static_cast<const char *>(
-          accessor->template getUntypedValue<input_nullable>(argument_attr_id));
-
-      if (input_nullable && (input_ptr == nullptr)) {
-        result->appendNullValue();
-      } else {
-        this->computeSubstring(input_ptr,
-                               static_cast<char *>(result->getPtrForDirectWrite()));
-      }
-    }
-    return result;
-  });
-}
-#endif
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-template <bool null_terminated, bool input_nullable>
-ColumnVector* SubstringUncheckedOperator<null_terminated,
-                                         input_nullable>
-    ::applyToValueAccessorForJoin(
-        ValueAccessor *accessor,
-        const bool use_left_relation,
-        const attribute_id argument_attr_id,
-        const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
-  return InvokeOnValueAccessorNotAdapter(
-      accessor,
-      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-    NativeColumnVector *result =
-        new NativeColumnVector(result_type_, accessor->getNumTuples());
-
-    for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-      const char *input_ptr = static_cast<const char *>(
-          accessor->template getUntypedValueAtAbsolutePosition<input_nullable>(
-              argument_attr_id,
-              use_left_relation ? joined_pair.first : joined_pair.second));
-
-      if (input_nullable && input_ptr == nullptr) {
-        result->appendNullValue();
-      } else {
-        this->computeSubstring(input_ptr,
-                               static_cast<char *>(result->getPtrForDirectWrite()));
-      }
-    }
-    return result;
+  return meta::InvokeOnBools(
+      input_null_terminated, type.isNullable(),
+      [&](auto is_null_terminated,  // NOLINT(build/c++11)
+          auto is_nullable) -> UncheckedUnaryOperator* {
+    return new SubstringUncheckedOperator<
+        decltype(is_null_terminated)::value,
+        decltype(is_nullable)::value>(
+            start_position,
+            ComputeMaximumSubstringLength(type, start_position, substring_length),
+            input_maximum_length,
+            *result_type);
   });
 }
-#endif
 
 }  // namespace quickstep


[29/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/parser/preprocessed/SqlLexer_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.hpp b/parser/preprocessed/SqlLexer_gen.hpp
index 86e36f0..7ff9fe4 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 5 "SqlLexer_gen.hpp"
+#line 6 "SqlLexer_gen.hpp"
 
-#line 7 "SqlLexer_gen.hpp"
+#line 8 "SqlLexer_gen.hpp"
 
 #define  YY_INT_ALIGNED short int
 
@@ -13,89 +13,11 @@
 #define FLEX_SCANNER
 #define YY_FLEX_MAJOR_VERSION 2
 #define YY_FLEX_MINOR_VERSION 6
-#define YY_FLEX_SUBMINOR_VERSION 3
+#define YY_FLEX_SUBMINOR_VERSION 0
 #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. */
@@ -170,13 +92,25 @@ typedef unsigned int flex_uint32_t;
 
 #endif /* ! FLEXINT_H */
 
-/* TODO: this is always defined, so inline it */
-#define yyconst const
+#ifdef __cplusplus
+
+/* The "const" storage-class-modifier is valid. */
+#define YY_USE_CONST
+
+#else	/* ! __cplusplus */
 
-#if defined(__GNUC__) && __GNUC__ >= 3
-#define yynoreturn __attribute__((__noreturn__))
+/* C99 requires __STDC__ to be defined as 1. */
+#if defined (__STDC__)
+
+#define YY_USE_CONST
+
+#endif	/* defined (__STDC__) */
+#endif	/* ! __cplusplus */
+
+#ifdef YY_USE_CONST
+#define yyconst const
 #else
-#define yynoreturn
+#define yyconst
 #endif
 
 /* An opaque pointer. */
@@ -231,12 +165,12 @@ struct yy_buffer_state
 	/* Size of input buffer in bytes, not including room for EOB
 	 * characters.
 	 */
-	int yy_buf_size;
+	yy_size_t yy_buf_size;
 
 	/* Number of characters read into yy_ch_buf, not including EOB
 	 * characters.
 	 */
-	int yy_n_chars;
+	yy_size_t 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
@@ -259,7 +193,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.
 	 */
@@ -270,21 +204,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 ( const char *yy_str , yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_scan_bytes ( const char *bytes, int 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 (yyconst char *yy_str ,yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_bytes (yyconst char *bytes,yy_size_t 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 */
 
@@ -317,48 +251,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 );
 
-			int quickstep_yyget_leng ( yyscan_t yyscanner );
+yy_size_t 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.
@@ -366,18 +300,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 *, const char *, int , yyscan_t yyscanner);
+static void yy_flex_strncpy (char *,yyconst char *,int ,yyscan_t yyscanner);
 #endif
 
 #ifdef YY_NEED_STRLEN
-static int yy_flex_strlen ( const char * , yyscan_t yyscanner);
+static int yy_flex_strlen (yyconst char * ,yyscan_t yyscanner);
 #endif
 
 #ifndef YY_NO_INPUT
@@ -406,7 +340,7 @@ static int yy_flex_strlen ( const 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)
@@ -426,9 +360,9 @@ extern int quickstep_yylex \
 #undef YY_DECL
 #endif
 
-#line 469 "../SqlLexer.lpp"
+#line 470 "../SqlLexer.lpp"
 
 
-#line 432 "SqlLexer_gen.hpp"
+#line 367 "SqlLexer_gen.hpp"
 #undef quickstep_yyIN_HEADER
 #endif /* quickstep_yyHEADER_H */


[19/32] incubator-quickstep git commit: Refactor type system and operations.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/SubstringOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/SubstringOperation.hpp b/types/operations/unary_operations/SubstringOperation.hpp
index 66f311f..afaf74d 100644
--- a/types/operations/unary_operations/SubstringOperation.hpp
+++ b/types/operations/unary_operations/SubstringOperation.hpp
@@ -22,22 +22,27 @@
 
 #include <algorithm>
 #include <cstddef>
+#include <cstdint>
 #include <cstdlib>
 #include <cstring>
+#include <limits>
 #include <memory>
-#include <unordered_map>
 #include <utility>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/CharType.hpp"
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
-#include "types/operations/Operation.pb.h"
+#include "types/VarCharType.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/containers/ColumnVectorUtil.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
-#include "utility/HashPair.hpp"
+#include "types/port/strnlen.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
@@ -53,116 +58,81 @@ class ValueAccessor;
  */
 class SubstringOperation : public UnaryOperation {
  public:
-  /**
-   * @brief Get a reference to the singleton instance of this Operation for
-   *        the given (start_position, substring_length) pair.
-   **/
-  static const SubstringOperation& Instance(const std::size_t start_position,
-                                            const std::size_t substring_length) {
-    // TODO(jianqiao): This is a temporary solution that creates a new instance
-    // for each distinct pair of start_position and substring_length arguments.
-    // The number of instances may be unbounded if quickstep continuously accepts
-    // queries that call SUBSTRING with different arguments. It still remains to
-    // design a better long-term solution.
-    const auto hash = [](const auto &pair) {
-      return hash_combine_detail::HashCombiner<std::size_t>::CombineHashes(pair.first, pair.second);
-    };
-    static std::unordered_map<std::pair<std::size_t, std::size_t>,
-                              std::unique_ptr<const SubstringOperation>,
-                              decltype(hash)> instance_map(10, hash);
-
-    const std::pair<std::size_t, std::size_t> key_pair =
-        std::make_pair(start_position, substring_length);
-    auto imit = instance_map.find(key_pair);
-    if (imit != instance_map.end()) {
-      return *imit->second;
-    } else {
-      const SubstringOperation *instance =
-          new SubstringOperation(start_position, substring_length);
-      instance_map.emplace(key_pair,
-                           std::unique_ptr<const SubstringOperation>(instance));
-      return *instance;
-    }
-  }
-
-  serialization::UnaryOperation getProto() const override;
+  SubstringOperation() {}
 
-  bool canApplyToType(const Type &type) const override {
-    return (type.getSuperTypeID() == Type::kAsciiString);
+  std::string getName() const override {
+    return "Substring";
   }
 
-  const Type *resultTypeForArgumentType(const Type &type) const override {
-    if (type.getSuperTypeID() == Type::kAsciiString) {
-      // Result is a Char string.
-      return &TypeFactory::GetType(TypeID::kChar,
-                                   computeMaximumSubstringLength(type),
-                                   type.isNullable());
-    }
-    return nullptr;
+  std::string getShortName() const override {
+    return "Substring";
   }
 
-  const Type* fixedNullableResultType() const override {
-    // Result type is not fixed (i.e. can have various lengths).
-    return nullptr;
+  std::vector<OperationSignaturePtr> getSignatures() const override {
+    return {
+        OperationSignature::Create(getName(), {kChar}, {kLong, kLong}),
+        OperationSignature::Create(getName(), {kVarChar}, {kLong, kLong})
+    };
   }
 
-  bool resultTypeIsPlausible(const Type &result_type) const override {
-    // Result can be coerced to Char or VarChar.
-    return (result_type.getSuperTypeID() == Type::kAsciiString);
-  }
+  bool canApplyTo(const Type &type,
+                  const std::vector<TypedValue> &static_arguments,
+                  std::string *message) const override {
+    DCHECK(type.getTypeID() == kChar || type.getTypeID() == kVarChar);
+    DCHECK(!static_arguments.empty() && static_arguments[0].getTypeID() == kLong);
+    DCHECK(static_arguments.size() <= 2);
 
-  const Type* pushDownTypeHint(const Type *type_hint) const override {
-    // Input can only be a string, but we don't know the length.
-    return nullptr;
-  }
+    if (static_arguments[0].getLiteral<std::int64_t>() <= 0) {
+      *message = "The start position must be greater than 0";
+      return false;
+    }
 
-  TypedValue applyToChecked(const TypedValue &argument,
-                            const Type &argument_type) const override {
-    DCHECK(canApplyToType(argument_type));
+    if (static_arguments.size() == 2) {
+      DCHECK(static_arguments[1].getTypeID() == kLong);
+      if (static_arguments[1].getLiteral<std::int64_t>() <= 0) {
+        *message = "The substring length must be greater than 0";
+        return false;
+      }
+    }
 
-    const Type *result_type = resultTypeForArgumentType(argument_type);
-    DCHECK(result_type != nullptr);
+    return true;
+  }
 
-    if (argument_type.isNullable() && argument.isNull()) {
-      return result_type->makeNullValue();
-    } else {
-      const std::size_t result_length = computeMaximumSubstringLength(argument_type);
-      char *output_ptr = static_cast<char*>(std::malloc(result_length));
-      const char *input_ptr = static_cast<const char*>(argument.getOutOfLineData());
+  const Type* getResultType(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(UnaryOperation::canApplyTo(type, static_arguments));
 
-      const std::size_t string_length = argument.getAsciiStringLength();
-      if (start_position_ >= string_length) {
-        *output_ptr = '\0';
-      } else {
-        const std::size_t actual_substring_length =
-            std::min(string_length - start_position_, substring_length_);
-        std::memcpy(output_ptr, input_ptr + start_position_, actual_substring_length);
-        if (actual_substring_length < result_length) {
-          output_ptr[actual_substring_length] = '\0';
-        }
-      }
+    std::size_t start_position;
+    std::size_t substring_length;
+    ExtractStaticArguments(static_arguments, &start_position, &substring_length);
 
-      return TypedValue::CreateWithOwnedData(result_type->getTypeID(),
-                                             output_ptr,
-                                             result_length);
-    }
+    return &TypeFactory::GetType(TypeID::kChar,
+                                 ComputeMaximumSubstringLength(type, start_position, substring_length),
+                                 type.isNullable());
   }
 
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorForType(const Type &type) const override;
+  UncheckedUnaryOperator* makeUncheckedUnaryOperator(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override;
 
  private:
-  /**
-   * @brief Constructor.
-   *
-   * @param input_type The data type of the input argument for substring.
-   * @param start_position The 0-base starting position of the substring.
-   * @param substring_length The substring length.
-   */
-  SubstringOperation(const std::size_t start_position,
-                     const std::size_t substring_length)
-      : UnaryOperation(UnaryOperationID::kSubstring),
-        start_position_(start_position),
-        substring_length_(substring_length) {
+  inline static void ExtractStaticArguments(
+      const std::vector<TypedValue> &static_arguments,
+      std::size_t *start_position,
+      std::size_t *substring_length) {
+    DCHECK_LE(1u, static_arguments.size());
+    DCHECK_GE(2u, static_arguments.size());
+
+    DCHECK(static_arguments[0].getTypeID() == kLong);
+    *start_position =
+        static_cast<std::size_t>(static_arguments[0].getLiteral<std::int64_t>() - 1);
+
+    DCHECK(static_arguments.size() < 2u || static_arguments[1].getTypeID() == kLong);
+    *substring_length =
+        static_arguments.size() < 2u
+            ? std::numeric_limits<std::size_t>::max()
+            : static_cast<std::size_t>(static_arguments[1].getLiteral<std::int64_t>());
   }
 
   /**
@@ -171,19 +141,23 @@ class SubstringOperation : public UnaryOperation {
    *
    * @param type The type of the input, must be either CharType or VarCharType.
    */
-  inline std::size_t computeMaximumSubstringLength(const Type& type) const {
-      DCHECK(type.getSuperTypeID() == Type::kAsciiString);
-
-      // Substring result should have length no greater than the minimum of
-      // (1) the input string length subtract the start position, and
-      // (2) the specified substring length.
-     return std::min(static_cast<const AsciiStringSuperType&>(type).getStringLength() - start_position_,
-                     substring_length_);
+  inline static std::size_t ComputeMaximumSubstringLength(
+      const Type& type,
+      const std::size_t start_position,
+      const std::size_t substring_length) {
+    DCHECK(type.getTypeID() == kChar || type.getTypeID() == kVarChar);
+
+    const std::size_t input_maximum_length =
+        type.getTypeID() == kChar
+            ? static_cast<const CharType&>(type).getStringLength()
+            : static_cast<const VarCharType&>(type).getStringLength();
+
+    // Substring result should have length no greater than the minimum of
+    // (1) the input string length subtract the start position, and
+    // (2) the specified substring length.
+    return std::min(input_maximum_length - start_position, substring_length);
   }
 
-  const std::size_t start_position_;
-  const std::size_t substring_length_;
-
  private:
   DISALLOW_COPY_AND_ASSIGN(SubstringOperation);
 };
@@ -203,8 +177,6 @@ class SubstringUncheckedOperator : public UncheckedUnaryOperator {
 
   TypedValue applyToTypedValue(const TypedValue& argument) const override;
 
-  TypedValue applyToDataPtr(const void *argument) const override;
-
   ColumnVector* applyToColumnVector(const ColumnVector &argument) const override;
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -231,6 +203,100 @@ class SubstringUncheckedOperator : public UncheckedUnaryOperator {
   DISALLOW_COPY_AND_ASSIGN(SubstringUncheckedOperator);
 };
 
+template <bool null_terminated, bool input_nullable>
+inline void SubstringUncheckedOperator<null_terminated, input_nullable>
+    ::computeSubstring(const char *input,
+                       char *output) const {
+  std::size_t string_length =
+      (null_terminated ? strlen(input) : strnlen(input, maximum_input_length_));
+
+  if (start_position_ >= string_length) {
+    *output = '\0';
+    return;
+  }
+
+  const std::size_t actual_substring_length =
+      std::min(string_length - start_position_, substring_length_);
+  std::memcpy(output, input + start_position_, actual_substring_length);
+
+  if (actual_substring_length < substring_length_) {
+    output[actual_substring_length] = '\0';
+  }
+}
+
+template <bool null_terminated, bool input_nullable>
+TypedValue SubstringUncheckedOperator<null_terminated,
+                                      input_nullable>
+    ::applyToTypedValue(const TypedValue& argument) const {
+  if (input_nullable && argument.isNull()) {
+    return TypedValue(result_type_.getTypeID());
+  }
+
+  char *output_ptr = static_cast<char*>(std::malloc(substring_length_));
+  computeSubstring(static_cast<const char*>(argument.getOutOfLineData()),
+                   output_ptr);
+
+  return TypedValue::CreateWithOwnedData(result_type_.getTypeID(),
+                                         output_ptr,
+                                         substring_length_);
+}
+
+template <bool null_terminated, bool input_nullable>
+ColumnVector* SubstringUncheckedOperator<null_terminated,
+                                         input_nullable>
+    ::applyToColumnVector(const ColumnVector &argument) const {
+  return InvokeOnColumnVector(
+      argument,
+      [&](const auto &column_vector) -> ColumnVector* {  // NOLINT(build/c++11)
+    NativeColumnVector *result =
+        new NativeColumnVector(result_type_, column_vector.size());
+
+    for (std::size_t cv_pos = 0;
+         cv_pos < column_vector.size();
+         ++cv_pos) {
+      const char *input_ptr = static_cast<const char *>(
+          column_vector.template getUntypedValue<input_nullable>(cv_pos));
+
+      if (input_nullable && input_ptr == nullptr) {
+        result->appendNullValue();
+      } else {
+        this->computeSubstring(input_ptr,
+                               static_cast<char *>(result->getPtrForDirectWrite()));
+      }
+    }
+    return result;
+  });
+}
+
+#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
+template <bool null_terminated, bool input_nullable>
+ColumnVector* SubstringUncheckedOperator<null_terminated,
+                                         input_nullable>
+    ::applyToValueAccessor(ValueAccessor *accessor,
+                           const attribute_id argument_attr_id) const {
+  return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+      accessor,
+      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+    NativeColumnVector *result =
+        new NativeColumnVector(result_type_, accessor->getNumTuples());
+
+    accessor->beginIteration();
+    while (accessor->next()) {
+      const char *input_ptr = static_cast<const char *>(
+          accessor->template getUntypedValue<input_nullable>(argument_attr_id));
+
+      if (input_nullable && (input_ptr == nullptr)) {
+        result->appendNullValue();
+      } else {
+        this->computeSubstring(input_ptr,
+                               static_cast<char *>(result->getPtrForDirectWrite()));
+      }
+    }
+    return result;
+  });
+}
+#endif
+
 }  // namespace quickstep
 
 #endif /* QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_SUBSTRING_OPERATION_HPP_ */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/UnaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperation.cpp b/types/operations/unary_operations/UnaryOperation.cpp
index af150b3..09b27d9 100644
--- a/types/operations/unary_operations/UnaryOperation.cpp
+++ b/types/operations/unary_operations/UnaryOperation.cpp
@@ -20,28 +20,8 @@
 #include "types/operations/unary_operations/UnaryOperation.hpp"
 
 #include "types/operations/Operation.pb.h"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
-serialization::UnaryOperation UnaryOperation::getProto() const {
-  serialization::UnaryOperation proto;
-  switch (operation_id_) {
-    case UnaryOperationID::kNegate:
-      proto.set_operation_id(serialization::UnaryOperation::NEGATE);
-      break;
-    case UnaryOperationID::kCast:
-      FATAL_ERROR("Must use the overridden NumericCastOperation::getProto");
-    case UnaryOperationID::kDateExtract:
-      FATAL_ERROR("Must use the overridden DateExtractOperation::getProto");
-    case UnaryOperationID::kSubstring:
-      FATAL_ERROR("Must use the overridden SubstringOperation::getProto");
-    default:
-      FATAL_ERROR("Unrecognized UnaryOperationID in UnaryOperation::getProto");
-  }
-
-  return proto;
-}
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/UnaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperation.hpp b/types/operations/unary_operations/UnaryOperation.hpp
index 30a2961..70cb6f9 100644
--- a/types/operations/unary_operations/UnaryOperation.hpp
+++ b/types/operations/unary_operations/UnaryOperation.hpp
@@ -24,18 +24,10 @@
 #include <string>
 #include <type_traits>
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-#include <utility>
-#include <vector>
-
-#include "storage/StorageBlockInfo.hpp"
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
 #include "catalog/CatalogTypedefs.hpp"
-#include "types/operations/Operation.hpp"
-#include "types/operations/Operation.pb.h"
 #include "types/TypedValue.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
+#include "types/operations/Operation.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -48,6 +40,9 @@ class ValueAccessor;
  *  @{
  */
 
+class UnaryOperation;
+typedef std::shared_ptr<const UnaryOperation> UnaryOperationPtr;
+
 /**
  * @brief A unary operator which can be quickly applied to data items WITHOUT
  *        checking their type.
@@ -69,14 +64,6 @@ class UncheckedUnaryOperator {
   virtual TypedValue applyToTypedValue(const TypedValue &argument) const = 0;
 
   /**
-   * @brief Apply to a data item via a pointer without type-checking.
-   *
-   * @param argument The data item to apply to.
-   * @return The literal result of the operation.
-   **/
-  virtual TypedValue applyToDataPtr(const void *argument) const = 0;
-
-  /**
    * @brief Apply to a vector of values without type-checking.
    *
    * @param argument The argument ColumnVector to apply to.
@@ -96,27 +83,6 @@ class UncheckedUnaryOperator {
                                              const attribute_id argument_attr_id) const = 0;
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  /**
-   * @brief Apply to an attribute of a list of joined tuples in a
-   *        ValueAccessor.
-   *
-   * @param accessor The ValueAccessor to apply to.
-   * @param use_left_relation If true, this UnaryOperation's argument is
-   *        assumed to be taken from the left relation in the pairs of
-   *        joined_tuple_ids. If false, the right relation.
-   * @param argument_attr_id The attribute ID of the argument in accessor.
-   * @param joined_tuple_ids A series of pairs of tuple ids from the left and
-   *        right relations in a join.
-   * @return A ColumnVector of literal results of the operation.
-   **/
-  virtual ColumnVector* applyToValueAccessorForJoin(
-      ValueAccessor *accessor,
-      const bool use_left_relation,
-      const attribute_id argument_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
  protected:
   UncheckedUnaryOperator() {
   }
@@ -134,30 +100,11 @@ class UncheckedUnaryOperator {
 class UnaryOperation : public Operation {
  public:
   /**
-   * @brief Generate a serialized Protocol Buffer representation of
-   *        this UnaryOperation.
-   *
-   * @return The serialized Protocol Buffer representation of this UnaryOperation.
-   **/
-  virtual serialization::UnaryOperation getProto() const;
-
-  /**
-   * @brief Determine the ID of this UnaryOperation.
-   *
-   * @return The ID of this UnaryOperation.
-   **/
-  inline UnaryOperationID getUnaryOperationID() const {
-    return operation_id_;
-  }
-
-  /**
    * @brief Get a human-readable name for this UnaryOperation.
    *
    * @return A human-readable name for this UnaryOperation.
    **/
-  virtual std::string getName() const {
-    return kUnaryOperationNames[static_cast<std::size_t>(operation_id_)];
-  }
+  virtual std::string getName() const = 0;
 
   /**
    * @brief Get a human-readable short name (e.g. "-") for this UnaryOperation.
@@ -165,113 +112,32 @@ class UnaryOperation : public Operation {
    * @return A human-readable short name for this BinaryOperation.
    **/
   virtual std::string getShortName() const {
-    return kUnaryOperationShortNames[static_cast<std::size_t>(operation_id_)];
+    return getName();
   }
 
-  /**
-   * @brief Determine whether this UnaryOperation can apply to the specified
-   *        Type.
-   *
-   * @param type The argument Type to check.
-   * @return Whether this UnaryOperation can apply to type.
-   **/
-  virtual bool canApplyToType(const Type &type) const = 0;
+  virtual bool canApplyTo(const Type &argument_type,
+                          const std::vector<TypedValue> &static_arguments,
+                          std::string *message) const = 0;
 
-  /**
-   * @brief Determine the Type of the result from applying this UnaryOperation
-   *        to an argument of the specified Type.
-   *
-   * @param type The argument Type to check.
-   * @return The Type of the result from applying this UnaryOperation to type
-   *         (NULL if not applicable).
-   **/
-  virtual const Type* resultTypeForArgumentType(const Type &type) const = 0;
-
-  /**
-   * @brief If this UnaryOperation always yields the same Type (or if the ONLY
-   *        difference between 2 possible return Types is nullability), return
-   *        that Type, otherwise return NULL.
-   *
-   * @return The nullable version of this UnaryOperation's fixed result Type,
-   *         if applicable.
-   **/
-  virtual const Type* fixedNullableResultType() const = 0;
+  virtual const Type* getResultType(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments) const = 0;
 
-  /**
-   * @brief Check if a particular Type might possibly be returned by this
-   *        UnaryOperation, assuming an appropriate argument type.
-   * @note A nullable result type may be considered plausible even if a
-   *       particular UnaryOperation never actually returns NULL values, so
-   *       long as the non-nullable version of the type would otherwise be
-   *       plausible.
-   *
-   * @param result_type Check whether this Type can possibly be returned by
-   *        this UnaryOperation.
-   * @return true if result_type can be returned by this UnaryOperation, false
-   *         otherwise.
-   **/
-  virtual bool resultTypeIsPlausible(const Type &result_type) const = 0;
+  virtual UncheckedUnaryOperator* makeUncheckedUnaryOperator(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments) const = 0;
 
-  /**
-   * @brief Get a "hint" Type for the argument to this UnaryOperation based on
-   *        a hint for this UnaryOperation's result type. If possible, returns
-   *        a pointer to a Type that, when given to this UnaryOperation as an
-   *        argument, yields values of the desired type (i.e. calling
-   *        resultTypeForArgumentType() on the returned type should return the
-   *        original type_hint).
-   * @note In some cases (e.g. NumericCastOperation) there may be multiple
-   *       types that can be used as arguments to this UnaryOperation that will
-   *       all yield the desired type_hint. In such cases, this method will
-   *       pick one Type based on its own implementation-specific preference.
-   *
-   * @param type_hint A hint about what Type the result of this UnaryOperation
-   *        should have. May be NULL to indicate no preference.
-   * @return A type hint for the argument to this UnaryOperation based on
-   *         type_hint, or NULL if no suitable Type exists.
-   **/
-  virtual const Type* pushDownTypeHint(const Type *type_hint) const = 0;
-
-  /**
-   * @brief Apply this UnaryOperation to a TypedValue.
-   * @warning It is an error to call this method if this UnaryOperation can not
-   *          be applied to argument_type. If in doubt, check canApplyToType()
-   *          first.
-   *
-   * @param argument The TypedValue to apply to.
-   * @param argument_type The Type that argument belongs to.
-   * @return The literal result of the operation.
-   **/
-  virtual TypedValue applyToChecked(const TypedValue &argument,
-                                    const Type &argument_type) const = 0;
-
-  /**
-   * @brief Create an UncheckedUnaryOperator which can apply to items of the
-   *        specified type.
-   * @warning The resulting UncheckedUnaryOperator performs no type-checking
-   *          whatsoever. Nonetheless, it is useful in situations where many
-   *          data items of the same, known type are to be operated on (for
-   *          example, over many tuples in the same relation).
-   *
-   * @param type The Type of argument to apply to.
-   * @return An UncheckedUnaryOperator which applies this UnaryOperation to
-   *         the specified Type.
-   * @exception OperationInapplicableToType This UnaryOperation is not
-   *            applicable to type.
-   **/
-  virtual UncheckedUnaryOperator* makeUncheckedUnaryOperatorForType(const Type &type) const = 0;
+  bool canApplyTo(const Type &argument_type,
+                  const std::vector<TypedValue> &static_arguments) const {
+    std::string message;
+    return canApplyTo(argument_type, static_arguments, &message);
+  }
 
  protected:
-  explicit UnaryOperation(const UnaryOperationID operation_id)
-      : Operation(Operation::kUnaryOperation,
-                  kUnaryOperationNames[
-                      static_cast<typename std::underlying_type<UnaryOperationID>::type>(operation_id)],
-                  kUnaryOperationShortNames[
-                      static_cast<typename std::underlying_type<UnaryOperationID>::type>(operation_id)]),
-        operation_id_(operation_id) {
+  UnaryOperation()
+      : Operation(Operation::kUnaryOperation) {
   }
 
-  const UnaryOperationID operation_id_;
-
  private:
   DISALLOW_COPY_AND_ASSIGN(UnaryOperation);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/UnaryOperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationFactory.cpp b/types/operations/unary_operations/UnaryOperationFactory.cpp
deleted file mode 100644
index b306061..0000000
--- a/types/operations/unary_operations/UnaryOperationFactory.cpp
+++ /dev/null
@@ -1,120 +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 "types/operations/unary_operations/UnaryOperationFactory.hpp"
-
-#include <string>
-
-#include "types/TypeFactory.hpp"
-#include "types/operations/Operation.pb.h"
-#include "types/operations/unary_operations/ArithmeticUnaryOperations.hpp"
-#include "types/operations/unary_operations/NumericCastOperation.hpp"
-#include "types/operations/unary_operations/DateExtractOperation.hpp"
-#include "types/operations/unary_operations/SubstringOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-const UnaryOperation& UnaryOperationFactory::GetUnaryOperation(const UnaryOperationID id) {
-  switch (id) {
-    case UnaryOperationID::kNegate:
-      return NegateUnaryOperation::Instance();
-    case UnaryOperationID::kCast:
-      FATAL_ERROR("Getting a CastOperation through GetUnaryOperation is not supported");
-    case UnaryOperationID::kDateExtract:
-      FATAL_ERROR("Getting a DateExtractOperation through GetUnaryOperation is not supported");
-    case UnaryOperationID::kSubstring:
-      FATAL_ERROR("Getting a SubstringOperation through GetUnaryOperation is not supported");
-    default:
-      FATAL_ERROR("Unknown UnaryOperationID");
-  }
-}
-
-bool UnaryOperationFactory::ProtoIsValid(const serialization::UnaryOperation &proto) {
-  // Check that UnaryOperation is fully initialized.
-  if (!proto.IsInitialized()) {
-    return false;
-  }
-
-  // Check that the operation_id is a valid UnaryOperation.
-  if (!proto.UnaryOperationID_IsValid(proto.operation_id())) {
-    return false;
-  }
-
-  switch (proto.operation_id()) {
-    case serialization::UnaryOperation::NEGATE:
-      return true;
-    case serialization::UnaryOperation::CAST:
-      return proto.HasExtension(serialization::CastOperation::target_type)
-          && TypeFactory::ProtoIsValid(proto.GetExtension(serialization::CastOperation::target_type));
-    case serialization::UnaryOperation::DATE_EXTRACT:
-      return proto.HasExtension(serialization::DateExtractOperation::unit)
-          && DateExtractOperation_Unit_IsValid(proto.GetExtension(serialization::DateExtractOperation::unit));
-    case serialization::UnaryOperation::SUBSTRING:
-      return proto.HasExtension(serialization::SubstringOperation::start_position)
-          && proto.HasExtension(serialization::SubstringOperation::substring_length);
-    default:
-      return false;
-  }
-}
-
-const UnaryOperation& UnaryOperationFactory::ReconstructFromProto(
-    const serialization::UnaryOperation &proto) {
-  DCHECK(ProtoIsValid(proto))
-      << "Attempted to create UnaryOperation from an invalid proto description:\n"
-      << proto.DebugString();
-
-  switch (proto.operation_id()) {
-    case serialization::UnaryOperation::NEGATE:
-      return GetUnaryOperation(UnaryOperationID::kNegate);
-    case serialization::UnaryOperation::CAST:
-      return NumericCastOperation::Instance(
-          TypeFactory::ReconstructFromProto(
-              proto.GetExtension(
-                  serialization::CastOperation::target_type)));
-    case serialization::UnaryOperation::DATE_EXTRACT:
-      switch (proto.GetExtension(serialization::DateExtractOperation::unit)) {
-        case serialization::DateExtractOperation::YEAR:
-          return DateExtractOperation::Instance(DateExtractUnit::kYear);
-        case serialization::DateExtractOperation::MONTH:
-          return DateExtractOperation::Instance(DateExtractUnit::kMonth);
-        case serialization::DateExtractOperation::DAY:
-          return DateExtractOperation::Instance(DateExtractUnit::kDay);
-        case serialization::DateExtractOperation::HOUR:
-          return DateExtractOperation::Instance(DateExtractUnit::kHour);
-        case serialization::DateExtractOperation::MINUTE:
-          return DateExtractOperation::Instance(DateExtractUnit::kMinute);
-        case serialization::DateExtractOperation::SECOND:
-          return DateExtractOperation::Instance(DateExtractUnit::kSecond);
-        default:
-          FATAL_ERROR("Unrecognized DateExtractOperation unit in UnaryOperation::ReconstructFromProto");
-      }
-    case serialization::UnaryOperation::SUBSTRING:
-      return SubstringOperation::Instance(
-          proto.GetExtension(serialization::SubstringOperation::start_position),
-          proto.GetExtension(serialization::SubstringOperation::substring_length));
-    default:
-      FATAL_ERROR("Unrecognized UnaryOperationID in UnaryOperation::ReconstructFromProto");
-  }
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/UnaryOperationFactory.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationFactory.hpp b/types/operations/unary_operations/UnaryOperationFactory.hpp
deleted file mode 100644
index 2ce83d4..0000000
--- a/types/operations/unary_operations/UnaryOperationFactory.hpp
+++ /dev/null
@@ -1,79 +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_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_FACTORY_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_FACTORY_HPP_
-
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-class UnaryOperation;
-namespace serialization { class UnaryOperation; }
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief All-static factory object that provides access to UnaryOperations.
- **/
-class UnaryOperationFactory {
- public:
-  /**
-   * @brief Convenience factory method to get a pointer to a UnaryOperation
-   *        from that UnaryOperation's ID.
-   *
-   * @param id The ID of the desired UnaryOperation.
-   * @return The UnaryOperation corresponding to id.
-   **/
-  static const UnaryOperation& GetUnaryOperation(const UnaryOperationID id);
-
-  /**
-   * @brief Get a reference to a UnaryOperation from that UnaryOperation's
-   *        serialized Protocol Buffer representation.
-   *
-   * @param proto A serialized Protocol Buffer representation of a UnaryOperation,
-   *        originally generated by getProto().
-   * @return The UnaryOperation described by proto.
-   **/
-  static const UnaryOperation& ReconstructFromProto(const serialization::UnaryOperation &proto);
-
-  /**
-   * @brief Check whether a serialization::UnaryOperation is fully-formed and
-   *        all parts are valid.
-   *
-   * @param proto A serialized Protocol Buffer representation of a UnaryOperation,
-   *        originally generated by getProto().
-   * @return Whether proto is fully-formed and valid.
-   **/
-  static bool ProtoIsValid(const serialization::UnaryOperation &proto);
-
- private:
-  UnaryOperationFactory();
-
-  DISALLOW_COPY_AND_ASSIGN(UnaryOperationFactory);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/UnaryOperationID.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationID.cpp b/types/operations/unary_operations/UnaryOperationID.cpp
deleted file mode 100644
index b47a848..0000000
--- a/types/operations/unary_operations/UnaryOperationID.cpp
+++ /dev/null
@@ -1,32 +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 "types/operations/unary_operations/UnaryOperationID.hpp"
-
-namespace quickstep {
-
-const char *kUnaryOperationNames[] = {
-  "Negate", "Cast", "DateExtract", "Substring"
-};
-
-const char *kUnaryOperationShortNames[] = {
-  "-", "Cast", "DateExtract", "Substring"
-};
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/UnaryOperationID.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationID.hpp b/types/operations/unary_operations/UnaryOperationID.hpp
deleted file mode 100644
index fa50f50..0000000
--- a/types/operations/unary_operations/UnaryOperationID.hpp
+++ /dev/null
@@ -1,63 +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_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_ID_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_ID_HPP_
-
-#include <type_traits>
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief Concrete UnaryOperations.
- **/
-enum class UnaryOperationID {
-  kNegate = 0,
-  kCast,
-  kDateExtract,
-  kSubstring,
-  kNumUnaryOperationIDs  // Not a real UnaryOperationID, exists for counting purposes.
-};
-
-/**
- * @brief Names of comparisons in the same order as UnaryOperationID.
- * @note Defined out-of-line in UnaryOperation.cpp
- **/
-extern const char *kUnaryOperationNames[
-    static_cast<typename std::underlying_type<UnaryOperationID>::type>(
-        UnaryOperationID::kNumUnaryOperationIDs)];
-
-/**
- * @brief Short names (i.e. mathematical symbols) of comparisons in the same
- *        order as UnaryOperationID.
- * @note Defined out-of-line in UnaryOperation.cpp
- **/
-extern const char *kUnaryOperationShortNames[
-    static_cast<typename std::underlying_type<UnaryOperationID>::type>(
-        UnaryOperationID::kNumUnaryOperationIDs)];
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_ID_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/unary_operations/UnaryOperationWrapper.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationWrapper.hpp b/types/operations/unary_operations/UnaryOperationWrapper.hpp
new file mode 100644
index 0000000..59b2cf0
--- /dev/null
+++ b/types/operations/unary_operations/UnaryOperationWrapper.hpp
@@ -0,0 +1,250 @@
+/**
+ * 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_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_WRAPPER_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_WRAPPER_HPP_
+
+#include <cstddef>
+#include <string>
+#include <type_traits>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/operations/OperationSignature.hpp"
+#include "types/operations/OperationUtil.hpp"
+#include "types/operations/unary_operations/UnaryOperation.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename ArgumentT, typename ResultT>
+struct UnaryFunctor {
+  typedef ArgumentT ArgumentType;
+  typedef ResultT ResultType;
+
+  static constexpr Operation
+      ::OperationSuperTypeID kOperationSuperTypeID = Operation::kUnaryOperation;
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+class UncheckedUnaryOperatorWrapperCodegen : public UncheckedUnaryOperator {
+ public:
+  template <typename ...ConstructorArgs>
+  UncheckedUnaryOperatorWrapperCodegen(const Type &argument_type,
+                                       const Type &result_type,
+                                       ConstructorArgs &&...args)
+      : functor_(std::forward<ConstructorArgs>(args)...),
+        impl_(functor_, argument_type, result_type) {}
+
+  TypedValue applyToTypedValue(const TypedValue &argument) const override {
+    return impl_.applyToTypedValue(argument);
+  }
+
+  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override {
+    using ArgumentCVT = typename ArgumentGen::ColumnVectorType;
+    DCHECK_EQ(argument.isNative(), ArgumentCVT::kNative);
+
+    using ArgumentAccessorT = ColumnVectorValueAccessor<ArgumentCVT>;
+    ArgumentAccessorT accessor(static_cast<const ArgumentCVT&>(argument));
+    return impl_.applyToValueAccessor(&accessor, 0);
+  }
+
+  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
+                                     const attribute_id attr_id) const override {
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        accessor,
+        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      return impl_.applyToValueAccessor(accessor, attr_id);
+    });
+  }
+
+ private:
+  using ArgumentType = typename FunctorT::ArgumentType;
+  using ResultType = typename FunctorT::ResultType;
+
+  using FuncSpec = typename FunctorSpecializer<FunctorT, SpecArgs...>::type;
+  using ArgumentGen = Codegen<FuncSpec, ArgumentType>;
+  using ResultGen = Codegen<FuncSpec, ResultType>;
+
+  template <bool argument_nullable>
+  struct Implementation;
+
+  const FunctorT functor_;
+  const Implementation<true> impl_;
+
+  DISALLOW_COPY_AND_ASSIGN(UncheckedUnaryOperatorWrapperCodegen);
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <bool argument_nullable>
+struct UncheckedUnaryOperatorWrapperCodegen<FunctorT, SpecArgs...>
+    ::Implementation {
+  Implementation(const FunctorT &functor_in,
+                 const Type &argument_type_in,
+                 const Type &result_type_in)
+      : functor(functor_in),
+        argument_type(argument_type_in),
+        result_type(result_type_in) {}
+
+  inline TypedValue applyToTypedValue(const TypedValue &argument) const {
+    if (argument_nullable && argument.isNull()) {
+      return TypedValue(ResultType::kStaticTypeID);
+    }
+
+    return ResultGen::template ApplyUnaryTypedValue<ArgumentGen>(
+        ArgumentGen::ToNativeValueConst(argument),
+        result_type,
+        functor);
+  }
+
+  template <typename AccessorT>
+  inline ColumnVector* applyToValueAccessor(AccessorT *accessor,
+                                            const attribute_id attr_id) const {
+    using ResultCVT = typename ResultGen::ColumnVectorType;
+    ResultCVT *result_cv = new ResultCVT(result_type, accessor->getNumTuples());
+
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename ArgumentGen::NativeTypeConstPtr arg_value =
+          ArgumentGen::template GetValuePtr<
+              argument_nullable, AccessorT>(accessor, attr_id);
+
+      if (argument_nullable && ArgumentGen::IsNull(arg_value)) {
+        result_cv->appendNullValue();
+      } else {
+        ResultGen::template ApplyUnaryColumnVector<ArgumentGen>(
+            ArgumentGen::Dereference(arg_value), functor, result_cv);
+      }
+    }
+    return result_cv;
+  }
+
+  const FunctorT &functor;
+  const Type &argument_type;
+  const Type &result_type;
+};
+
+template <typename FunctorT>
+class UnaryOperationWrapper : public UnaryOperation {
+ public:
+  UnaryOperationWrapper()
+      : UnaryOperation(),
+        operation_name_(FunctorT::GetName()) {}
+
+  std::string getName() const override {
+    return operation_name_;
+  }
+
+  std::string getShortName() const override {
+    return getName();
+  }
+
+  std::vector<OperationSignaturePtr> getSignatures() const override {
+    return {
+        OperationSignature::Create(getName(), {ArgumentType::kStaticTypeID}, 0)
+    };
+  }
+
+  bool canApplyTo(const Type &argument_type,
+                  const std::vector<TypedValue> &static_arguments,
+                  std::string *message) const override {
+    DCHECK(argument_type.getTypeID() == ArgumentType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return true;
+  }
+
+  const Type* getResultType(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(argument_type.getTypeID() == ArgumentType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return getResultTypeImpl<ResultType::kParameterized>(
+        argument_type, static_arguments);
+  }
+
+  UncheckedUnaryOperator* makeUncheckedUnaryOperator(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(argument_type.getTypeID() == ArgumentType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return makeUncheckedUnaryOperatorImpl<
+        std::is_default_constructible<FunctorT>::value>(
+            argument_type, static_arguments);
+  }
+
+ private:
+  using ArgumentType = typename FunctorT::ArgumentType;
+  using ResultType = typename FunctorT::ResultType;
+
+  template <bool functor_use_default_constructor>
+  inline UncheckedUnaryOperator* makeUncheckedUnaryOperatorImpl(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<functor_use_default_constructor>* = 0) const {
+    return new UncheckedUnaryOperatorWrapperCodegen<FunctorT>(
+        argument_type, *getResultType(argument_type, static_arguments));
+  }
+
+  template <bool functor_use_default_constructor>
+  inline UncheckedUnaryOperator* makeUncheckedUnaryOperatorImpl(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<!functor_use_default_constructor>* = 0) const {
+    return new UncheckedUnaryOperatorWrapperCodegen<FunctorT>(
+        argument_type, *getResultType(argument_type, static_arguments),
+        static_cast<const ArgumentType&>(argument_type));
+  }
+
+  template <bool result_type_has_parameter>
+  inline const Type* getResultTypeImpl(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<!result_type_has_parameter>* = 0) const {
+    return &TypeFactory::GetType(ResultType::kStaticTypeID,
+                                 argument_type.isNullable());
+  }
+
+  template <bool result_type_has_parameter>
+  inline const Type* getResultTypeImpl(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<result_type_has_parameter>* = 0) const {
+    return FunctorT::GetResultType(argument_type);
+  }
+
+  const std::string operation_name_;
+
+  DISALLOW_COPY_AND_ASSIGN(UnaryOperationWrapper);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_WRAPPER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index e1fb770..0108ff9 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -159,6 +159,7 @@ QS_PROTOBUF_GENERATE_CPP(quickstep_utility_SortConfiguration_proto_srcs
                          SortConfiguration.proto)
 
 add_subdirectory(lip_filter)
+add_subdirectory(meta)
 
 # Declare micro-libs:
 add_library(quickstep_utility_Alignment ../empty_src.cpp Alignment.hpp)
@@ -202,7 +203,6 @@ add_library(quickstep_utility_SortConfiguration_proto
             ${quickstep_utility_SortConfiguration_proto_hdrs})
 add_library(quickstep_utility_SqlError SqlError.cpp SqlError.hpp)
 add_library(quickstep_utility_StringUtil StringUtil.cpp StringUtil.hpp)
-add_library(quickstep_utility_TemplateUtil ../empty_src.cpp TemplateUtil.hpp)
 # Note that TextBasedTest.{hpp, cpp} are not in this static library.
 # Any tests that use them need to include them in the
 # executable.
@@ -323,7 +323,6 @@ target_link_libraries(quickstep_utility_ShardedLockManager
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_utility_StringUtil
                       glog)
-target_link_libraries(quickstep_utility_TemplateUtil)
 target_link_libraries(quickstep_utility_TextBasedTestDriver
                       glog
                       gtest
@@ -373,7 +372,6 @@ target_link_libraries(quickstep_utility
                       quickstep_utility_SortConfiguration_proto
                       quickstep_utility_SqlError
                       quickstep_utility_StringUtil
-                      quickstep_utility_TemplateUtil
                       quickstep_utility_TextBasedTestDriver
                       quickstep_utility_ThreadSafeQueue
                       quickstep_utility_TreeStringSerializable
@@ -485,14 +483,6 @@ target_link_libraries(TreeStringSerializable_unittest
                       quickstep_utility_TreeStringSerializable)
 add_test(TreeStringSerializable_unittest TreeStringSerializable_unittest)
 
-add_executable(TemplateUtil_unittest "${CMAKE_CURRENT_SOURCE_DIR}/tests/TemplateUtil_unittest.cpp")
-target_link_libraries(TemplateUtil_unittest
-                      gtest
-                      gtest_main
-                      quickstep_utility_Macros
-                      quickstep_utility_TemplateUtil)
-add_test(TemplateUtil_unittest TemplateUtil_unittest)
-
 # Benchmarks:
 if (UNIX)
   add_executable(EqualsAnyConstant_benchmark

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/utility/StringUtil.cpp
----------------------------------------------------------------------
diff --git a/utility/StringUtil.cpp b/utility/StringUtil.cpp
index 2745457..9fca695 100644
--- a/utility/StringUtil.cpp
+++ b/utility/StringUtil.cpp
@@ -52,6 +52,12 @@ std::string ToLower(const std::string& str) {
   return lower_str;
 }
 
+std::string ToUpper(const std::string& str) {
+  std::string upper_str(str.size(), ' ');
+  std::transform(str.begin(), str.end(), upper_str.begin(), toupper);
+  return upper_str;
+}
+
 std::string EscapeSpecialChars(const std::string& text) {
   std::string new_text;
   for (const char& c : text) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/utility/StringUtil.hpp
----------------------------------------------------------------------
diff --git a/utility/StringUtil.hpp b/utility/StringUtil.hpp
index abda8f3..e928225 100644
--- a/utility/StringUtil.hpp
+++ b/utility/StringUtil.hpp
@@ -35,11 +35,21 @@ namespace quickstep {
  * @brief Convert a string \p str to lower case.
  *
  * @param str The string to be converted.
- * @return The converted string with all lower case characters bing converted to upper case characters.
+ * @return The converted string with all lower case characters being converted
+ *         to upper case characters.
  */
 extern std::string ToLower(const std::string &str);
 
 /**
+ * @brief Convert a string \p str to upper case.
+ *
+ * @param str The string to be converted.
+ * @return The converted string with all upper case characters being converted
+ *         to lower case characters.
+ */
+extern std::string ToUpper(const std::string &str);
+
+/**
  * @brief Converts special characters to escape characters.
  *
  * @param text The string to be unescaped.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/utility/TemplateUtil.hpp
----------------------------------------------------------------------
diff --git a/utility/TemplateUtil.hpp b/utility/TemplateUtil.hpp
index dfae8e4..587336d 100644
--- a/utility/TemplateUtil.hpp
+++ b/utility/TemplateUtil.hpp
@@ -30,204 +30,6 @@ namespace quickstep {
  *  @{
  */
 
-namespace template_util_inner {
-
-/**
- * @brief Represents a compile-time sequence of integers.
- *
- * Sequence is defined here for C++11 compatibility. For C++14 and above,
- * std::integer_sequence can be used to achieve the same functionality.
- *
- * TODO(jianqiao): directly use std::integer_sequence if having C++14 support.
- */
-template<std::size_t ...>
-struct Sequence {};
-
-/**
- * @brief The helper class for creating Sequence. MakeSequence<N>::type is
- *        equivalent to Sequence<1,2,...,N>.
- *
- * MakeSequence is defined here for C++11 compatibility. For C++14 and above,
- * std::make_index_sequence can be used to achieve the same functionality.
- *
- * TODO(jianqiao): directly use std::make_index_sequence if having C++14 support.
- */
-template<std::size_t N, std::size_t ...S>
-struct MakeSequence : MakeSequence<N-1, N-1, S...> {};
-
-template<std::size_t ...S>
-struct MakeSequence<0, S...> {
-  typedef Sequence<S...> type;
-};
-
-/**
- * @brief Final step of CreateBoolInstantiatedInstance. Now all bool_values are
- *        ready. Instantiate the template and create (i.e. new) an instance.
- */
-template <template <bool ...> class T, class ReturnT,
-          bool ...bool_values, std::size_t ...i,
-          typename Tuple>
-inline ReturnT* CreateBoolInstantiatedInstanceInner(Tuple &&args,
-                                                    Sequence<i...> &&indices) {
-  return new T<bool_values...>(std::get<i>(std::forward<Tuple>(args))...);
-}
-
-/**
- * @brief Invoke the functor with the compile-time bool values wrapped as
- *        integral_constant types.
- */
-template <typename FunctorT, bool ...bool_values>
-inline auto InvokeOnBoolsInner(const FunctorT &functor) {
-  return functor(std::integral_constant<bool, bool_values>()...);
-}
-
-/**
- * @brief Recursive dispatching.
- */
-template <typename FunctorT, bool ...bool_values, typename ...Bools>
-inline auto InvokeOnBoolsInner(const FunctorT &functor,
-                               const bool tparam,
-                               const Bools ...rest_params) {
-  if (tparam) {
-    return InvokeOnBoolsInner<FunctorT, bool_values..., true>(
-        functor, rest_params...);
-  } else {
-    return InvokeOnBoolsInner<FunctorT, bool_values..., false>(
-        functor, rest_params...);
-  }
-}
-
-/**
- * @brief Move the functor to the first position in argument list.
- */
-template <std::size_t last, std::size_t ...i, typename TupleT>
-inline auto InvokeOnBoolsInner(TupleT &&args, Sequence<i...> &&indices) {
-  return InvokeOnBoolsInner(std::get<last>(std::forward<TupleT>(args)),
-                            std::get<i>(std::forward<TupleT>(args))...);
-}
-
-}  // namespace template_util_inner
-
-/**
- * @brief Edge case of the recursive CreateBoolInstantiatedInstance function
- *        when all bool variables have been branched and replaced with compile-time
- *        bool constants.
- */
-template <template <bool ...> class T, class ReturnT,
-          bool ...bool_values,
-          typename Tuple>
-inline ReturnT* CreateBoolInstantiatedInstance(Tuple &&args) {
-  // Note that the constructor arguments have been forwarded as a tuple (args).
-  // Here we generate a compile-time index sequence (i.e. typename MakeSequence<n_args>::type())
-  // for the tuple, so that the tuple can be unpacked as a sequence of constructor
-  // parameters in CreateBoolInstantiatedInstanceInner.
-  constexpr std::size_t n_args = std::tuple_size<Tuple>::value;
-  return template_util_inner::CreateBoolInstantiatedInstanceInner<
-      T, ReturnT, bool_values...>(
-          std::forward<Tuple>(args),
-          typename template_util_inner::MakeSequence<n_args>::type());
-}
-
-/**
- * @brief A helper function for creating bool branched templates.
- *
- * The scenario for using this helper function is that, suppose we have a class
- * where all template parameters are bools:
- * --
- * template <bool c1, bool c2, bool c3>
- * class SomeClass : public BaseClass {
- *   // This simple function will be invoked in computationally-intensive loops.
- *   inline SomeType someSimpleFunction(...) {
- *     if (c1) {
- *       doSomeThing1();
- *     }
- *     if (c2) {
- *       doSomeThing2();
- *     }
- *     if (c3) {
- *       doSomeThing3();
- *     }
- *   }
- * };
- * --
- * Typically, this bool-paramterized template is for performance consideration.
- * That is, we would like to make a copy of code for each configuration of bool
- * values, so that there will be no branchings in someSimpleFunction().
- *
- * The problem is that, to conditionally instantiate the template, given bool
- * variables c1, c2, c3, we have to do something like this:
- * --
- * if (c1) {
- *   if (c2) {
- *     if (c3) {
- *       return new SomeClass<true, true, true>(some_args...);
- *     } else {
- *       return new SomeClass<true, true, false>(some_args...);
- *     }
- *   } else {
- *     if (c3) {
- *       return new SomeClass<true, false, true>(some_args...);
- *     } else {
- *       return new SomeClass<true, false, false>(some_args...);
- *     }
- * } else {
- *   ...
- * }
- * --
- * Then there will be power(2,N) branches if the template has N bool parameters,
- * making it tedious to do the instantiating.
- *
- * Now, this helper function can achieve the branched instantiation in one
- * statement as:
- * --
- * return CreateBoolInstantiatedInstance<SomeClass,BaseClass>(
- *     std::forward_as_tuple(some_args...), c1, c2, c3);
- * --
- */
-template <template <bool ...> class T, class ReturnT,
-          bool ...bool_values, typename ...Bools,
-          typename Tuple>
-inline ReturnT* CreateBoolInstantiatedInstance(Tuple &&args,
-                                               const bool tparam,
-                                               const Bools ...rest_tparams) {
-  if (tparam) {
-    return CreateBoolInstantiatedInstance<T, ReturnT, bool_values..., true>(
-        std::forward<Tuple>(args), rest_tparams...);
-  } else {
-    return CreateBoolInstantiatedInstance<T, ReturnT, bool_values..., false>(
-        std::forward<Tuple>(args), rest_tparams...);
-  }
-}
-
-/**
- * @brief A helper function for bool branched template specialization.
- *
- * Usage example:
- * --
- * bool c1 = true, c2 = false;
- *
- * InvokeOnBools(
- *     c1, c2,
- *     [&](auto c1, auto c2) -> SomeBaseClass* {
- *   using T1 = decltype(c1);  // T1 == std::true_type
- *   using T2 = decltype(c2);  // T2 == std::false_type
- *
- *   constexpr bool cv1 = T1::value;  // cv1 == true
- *   constexpr bool cv2 = T2::value;  // cv2 == false
- *
- *   SomeFunction<cv1, cv2>(...);
- *   return new SomeClass<cv1, cv2>(...);
- * });
- * --
- */
-template <typename ...ArgTypes>
-inline auto InvokeOnBools(ArgTypes ...args) {
-  constexpr std::size_t last = sizeof...(args) - 1;
-  return template_util_inner::InvokeOnBoolsInner<last>(
-      std::forward_as_tuple(args...),
-      typename template_util_inner::MakeSequence<last>::type());
-}
-
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/utility/meta/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/meta/CMakeLists.txt b/utility/meta/CMakeLists.txt
new file mode 100644
index 0000000..1b72dd9
--- /dev/null
+++ b/utility/meta/CMakeLists.txt
@@ -0,0 +1,41 @@
+# 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.
+
+# Declare micro-libs:
+add_library(quickstep_utility_meta_Common ../../empty_src.cpp Common.hpp)
+add_library(quickstep_utility_meta_Dispatchers ../../empty_src.cpp Dispatchers.hpp)
+add_library(quickstep_utility_meta_TMP ../../empty_src.cpp TMP.hpp)
+add_library(quickstep_utility_meta_TransitiveClosure ../../empty_src.cpp TransitiveClosure.hpp)
+add_library(quickstep_utility_meta_TypeList ../../empty_src.cpp TypeList.hpp)
+add_library(quickstep_utility_meta_TypeListMetaFunctions ../../empty_src.cpp TypeListMetaFunctions.hpp)
+
+# Link dependencies:
+target_link_libraries(quickstep_utility_meta_Dispatchers
+                      quickstep_utility_meta_Common)
+target_link_libraries(quickstep_utility_meta_TMP
+                      quickstep_utility_meta_Common
+                      quickstep_utility_meta_Dispatchers
+                      quickstep_utility_meta_TransitiveClosure
+                      quickstep_utility_meta_TypeList)
+target_link_libraries(quickstep_utility_meta_TransitiveClosure
+                      quickstep_utility_meta_TypeList)
+target_link_libraries(quickstep_utility_meta_TypeList
+                      quickstep_utility_meta_Common
+                      quickstep_utility_meta_TypeListMetaFunctions)
+target_link_libraries(quickstep_utility_meta_TypeListMetaFunctions
+                      quickstep_utility_meta_Common)
+

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/utility/meta/Common.hpp
----------------------------------------------------------------------
diff --git a/utility/meta/Common.hpp b/utility/meta/Common.hpp
new file mode 100644
index 0000000..39c513e
--- /dev/null
+++ b/utility/meta/Common.hpp
@@ -0,0 +1,143 @@
+/**
+ * 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_META_COMMON_HPP_
+#define QUICKSTEP_UTILITY_META_COMMON_HPP_
+
+namespace quickstep {
+namespace meta {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+template <typename T, T ...s>
+struct Sequence {
+  template <template <typename ...> class Host>
+  using bind_to = Host<std::integral_constant<T, s>...>;
+
+  template <template <T ...> class Host>
+  using bind_values_to = Host<s...>;
+
+  template <typename U>
+  using cast_to = Sequence<U, static_cast<U>(s)...>;
+
+  template <typename CollectionT>
+  inline static CollectionT Instantiate() {
+    return { s... };
+  }
+};
+
+template <std::size_t ...s>
+using IntegerSequence = Sequence<std::size_t, s...>;
+
+
+template <std::size_t n, std::size_t ...s>
+struct MakeSequence : MakeSequence<n-1, n-1, s...> {};
+
+template <std::size_t ...s>
+struct MakeSequence<0, s...> {
+  using type = IntegerSequence<s...>;
+};
+
+
+template <typename ...> struct Conjunction : std::true_type {};
+template <typename B> struct Conjunction<B> : B {};
+template <typename B, typename ...Bs>
+struct Conjunction<B, Bs...>
+    : std::conditional_t<B::value, Conjunction<Bs...>, B> {};
+
+template <typename ...> struct Disjunction : std::false_type {};
+template <typename B> struct Disjunction<B> : B {};
+template <typename B, typename ...Bs>
+struct Disjunction<B, Bs...>
+    : std::conditional_t<B::value, B, Disjunction<Bs...>> {};
+
+template <typename check, typename ...cases>
+struct EqualsAny {
+  static constexpr bool value =
+     Disjunction<std::is_same<check, cases>...>::value;
+};
+
+
+template <typename T, typename Enable = void>
+struct IsTrait {
+  static constexpr bool value = false;
+};
+
+template <typename T>
+struct IsTrait<T, std::enable_if_t<
+    std::is_same<typename T::type, typename T::type>::value>> {
+  static constexpr bool value = true;
+};
+
+template <typename T, template <typename> class Op, typename Enable = void>
+struct IsWellFormed {
+  static constexpr bool value = false;
+};
+
+template <typename T, template <typename> class Op>
+struct IsWellFormed<T, Op, std::enable_if_t<std::is_same<Op<T>, Op<T>>::value>> {
+  static constexpr bool value = true;
+};
+
+
+template <typename LeftT, typename RightT>
+struct PairSelectorLeft {
+  typedef LeftT type;
+};
+
+template <typename LeftT, typename RightT>
+struct PairSelectorRight {
+  typedef RightT type;
+};
+
+
+template <char ...c>
+struct StringLiteral {
+  inline static std::string ToString() {
+    return std::string({c...});
+  }
+};
+
+template <template <typename ...> class Op>
+class TraitWrapper {
+ private:
+  template <typename ...ArgTypes>
+  struct Implemenation {
+    using type = Op<ArgTypes...>;
+  };
+
+ public:
+  template <typename ...ArgTypes>
+  using type = Implemenation<ArgTypes...>;
+};
+
+template <template <typename ...> class Op>
+struct TraitUnwrapper {
+  template <typename ...ArgTypes>
+  using type = typename Op<ArgTypes...>::type;
+};
+
+/** @} */
+
+}  // namespace meta
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_META_COMMON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/utility/meta/Dispatchers.hpp
----------------------------------------------------------------------
diff --git a/utility/meta/Dispatchers.hpp b/utility/meta/Dispatchers.hpp
new file mode 100644
index 0000000..5b0ee48
--- /dev/null
+++ b/utility/meta/Dispatchers.hpp
@@ -0,0 +1,107 @@
+/**
+ * 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_META_DISPATCHERS_HPP_
+#define QUICKSTEP_UTILITY_META_DISPATCHERS_HPP_
+
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+namespace meta {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+/**
+ * @brief A helper function for bool branched template specialization.
+ *
+ * Usage example:
+ * --
+ * bool c1 = true, c2 = false;
+ *
+ * InvokeOnBools(
+ *     c1, c2,
+ *     [&](auto c1, auto c2) -> SomeBaseClass* {
+ *   using T1 = decltype(c1);  // T1 == std::true_type
+ *   using T2 = decltype(c2);  // T2 == std::false_type
+ *
+ *   constexpr bool cv1 = T1::value;  // cv1 == true
+ *   constexpr bool cv2 = T2::value;  // cv2 == false
+ *
+ *   SomeFunction<cv1, cv2>(...);
+ *   return new SomeClass<cv1, cv2>(...);
+ * });
+ * --
+ */
+template <typename ...ArgTypes>
+inline auto InvokeOnBools(ArgTypes ...args);
+
+
+namespace internal {
+
+/**
+ * @brief Invoke the functor with the compile-time bool values wrapped as
+ *        integral_constant types.
+ */
+template <typename FunctorT, bool ...bool_values>
+inline auto InvokeOnBoolsInner(const FunctorT &functor) {
+  return functor(std::integral_constant<bool, bool_values>()...);
+}
+
+/**
+ * @brief Recursive dispatching.
+ */
+template <typename FunctorT, bool ...bool_values, typename ...Bools>
+inline auto InvokeOnBoolsInner(const FunctorT &functor,
+                               const bool tparam,
+                               const Bools ...rest_params) {
+  if (tparam) {
+    return InvokeOnBoolsInner<FunctorT, bool_values..., true>(
+        functor, rest_params...);
+  } else {
+    return InvokeOnBoolsInner<FunctorT, bool_values..., false>(
+        functor, rest_params...);
+  }
+}
+
+/**
+ * @brief Move the functor to the first position in argument list.
+ */
+template <std::size_t last, std::size_t ...i, typename TupleT>
+inline auto InvokeOnBoolsInner(TupleT &&args, IntegerSequence<i...> &&indices) {
+  return InvokeOnBoolsInner(std::get<last>(std::forward<TupleT>(args)),
+                            std::get<i>(std::forward<TupleT>(args))...);
+}
+
+}  // namespace internal
+
+template <typename ...ArgTypes>
+inline auto InvokeOnBools(ArgTypes ...args) {
+  constexpr std::size_t last = sizeof...(args) - 1;
+  return internal::InvokeOnBoolsInner<last>(std::forward_as_tuple(args...),
+                                            typename MakeSequence<last>::type());
+}
+
+/** @} */
+
+}  // namespace meta
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_META_DISPATCHERS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/utility/meta/TMP.hpp
----------------------------------------------------------------------
diff --git a/utility/meta/TMP.hpp b/utility/meta/TMP.hpp
new file mode 100644
index 0000000..5456479
--- /dev/null
+++ b/utility/meta/TMP.hpp
@@ -0,0 +1,28 @@
+/**
+ * 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_META_TMP_HPP_
+#define QUICKSTEP_UTILITY_META_TMP_HPP_
+
+#include "utility/meta/Common.hpp"
+#include "utility/meta/Dispatchers.hpp"
+#include "utility/meta/TransitiveClosure.hpp"
+#include "utility/meta/TypeList.hpp"
+
+#endif  // QUICKSTEP_UTILITY_META_TMP_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/utility/meta/TransitiveClosure.hpp
----------------------------------------------------------------------
diff --git a/utility/meta/TransitiveClosure.hpp b/utility/meta/TransitiveClosure.hpp
new file mode 100644
index 0000000..a5362bb
--- /dev/null
+++ b/utility/meta/TransitiveClosure.hpp
@@ -0,0 +1,97 @@
+/**
+ * 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_META_TRANSITIVE_CLOSURE_HPP_
+#define QUICKSTEP_UTILITY_META_TRANSITIVE_CLOSURE_HPP_
+
+#include "utility/meta/TypeList.hpp"
+
+namespace quickstep {
+namespace meta {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+template <typename Edges>
+struct TransitiveClosure;
+
+
+namespace internal {
+
+template <typename TL, typename Enable = void>
+struct EdgeMatcher {};
+
+template <typename TL>
+struct EdgeMatcher<
+    TL,
+    std::enable_if_t<std::is_same<typename TL::template at<0, 1>,
+                                  typename TL::template at<1, 0>>::value>> {
+  using type = TypeList<typename TL::template at<0, 0>,
+                        typename TL::template at<1, 1>>;
+};
+
+template <typename LeftEdges, typename RightEdges>
+struct JoinPath {
+  using type = typename LeftEdges::template cartesian_product<RightEdges>
+                                 ::template filtermap<EdgeMatcher>;
+};
+
+// Semi-naive
+template <typename Out, typename WorkSet, typename Edges, typename Enable = void>
+struct TransitiveClosureInner;
+
+template <typename Out, typename WorkSet, typename Edges>
+struct TransitiveClosureInner<Out, WorkSet, Edges,
+                              std::enable_if_t<WorkSet::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename WorkSet, typename Edges>
+struct TransitiveClosureInner<Out, WorkSet, Edges,
+                              std::enable_if_t<WorkSet::length != 0>>
+    : TransitiveClosureInner<typename Out::template append<WorkSet>,
+                             typename JoinPath<WorkSet, Edges>::type::template subtract<
+                                 typename Out::template append<WorkSet>>::template unique<>,
+                             Edges> {};
+
+template <typename Edge>
+struct TransitiveClosureInitializer {
+  using type = TypeList<TypeList<typename Edge::head, typename Edge::head>,
+                        TypeList<typename Edge::tail::head, typename Edge::tail::head>>;
+};
+
+template <typename Edges>
+using TransitiveClosureHelper =
+    typename TransitiveClosureInner<TypeList<>,
+                                    typename Edges::template flatmap<
+                                        TransitiveClosureInitializer>::template unique<>,
+                                    Edges>::type;
+
+}  // namespace internal
+
+template <typename Edges>
+struct TransitiveClosure : internal::TransitiveClosureHelper<Edges> {};
+
+/** @} */
+
+}  // namespace meta
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_META_TRANSITIVE_CLOSURE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/utility/meta/TypeList.hpp
----------------------------------------------------------------------
diff --git a/utility/meta/TypeList.hpp b/utility/meta/TypeList.hpp
new file mode 100644
index 0000000..fac3ce5
--- /dev/null
+++ b/utility/meta/TypeList.hpp
@@ -0,0 +1,124 @@
+/**
+ * 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_META_TYPE_LIST_HPP_
+#define QUICKSTEP_UTILITY_META_TYPE_LIST_HPP_
+
+#include "utility/meta/Common.hpp"
+#include "utility/meta/TypeListMetaFunctions.hpp"
+
+namespace quickstep {
+namespace meta {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+template <typename ...Ts>
+class TypeList;
+
+template <typename ...Ts>
+class TypeListCommon {
+ private:
+  template <typename ...Tail> struct AppendHelper {
+    using type = TypeList<Ts..., Tail...>;
+  };
+
+ public:
+  static constexpr std::size_t length = sizeof...(Ts);
+
+  using type = TypeList<Ts...>;
+
+  template <template <typename ...> class Host>
+  using bind_to = Host<Ts...>;
+
+  template <std::size_t ...pos>
+  using at = typename internal::ElementAtImpl<
+      TypeList<Ts...>, TypeList<std::integral_constant<std::size_t, pos>...>>::type;
+
+  template <typename T>
+  using push_front = TypeList<T, Ts...>;
+
+  template <typename T>
+  using push_back = TypeList<Ts..., T>;
+
+  template <typename T>
+  using contains = EqualsAny<T, Ts...>;
+
+  template <typename ...DumbT>
+  using unique = typename internal::UniqueImpl<TypeList<>, TypeList<Ts...>, DumbT...>::type;
+
+  template <typename TL>
+  using append = typename TL::template bind_to<AppendHelper>::type;
+
+  template <typename TL>
+  using cartesian_product = typename internal::CartesianProductImpl<TypeList<Ts...>, TL>::type;
+
+  template <typename Subtrahend>
+  using subtract = typename internal::SubtractImpl<TypeList<>, TypeList<Ts...>, Subtrahend>::type;
+
+  template <template <typename ...> class Op>
+  using map = TypeList<typename Op<Ts>::type...>;
+
+  template <template <typename ...> class Op>
+  using flatmap = typename internal::FlatmapImpl<TypeList<>, TypeList<Ts...>, Op>::type;
+
+  template <template <typename ...> class Op>
+  using filter = typename internal::FilterImpl<TypeList<>, TypeList<Ts...>, Op>::type;
+
+  template <template <typename ...> class Op>
+  using filtermap = typename internal::FiltermapImpl<TypeList<>, TypeList<Ts...>, Op>::type;
+
+  template <typename ...DumbT>
+  using flatten_once = typename internal::FlattenOnceImpl<TypeList<>, TypeList<Ts...>, DumbT...>::type;
+
+  template <typename TL>
+  using zip = typename internal::ZipImpl<TypeList<>, TypeList<Ts...>, TL>::type;
+
+  template <typename TL, template <typename ...> class Op>
+  using zip_with = typename internal::ZipWithImpl<TypeList<>, TypeList<Ts...>, TL, Op>::type;
+
+  template <typename T>
+  using as_sequence = typename internal::AsSequenceImpl<T, Ts...>::type;
+};
+
+template <typename ...Ts>
+class TypeList : public TypeListCommon<Ts...> {
+ private:
+  template <typename Head, typename ...Tail>
+  struct HeadTailHelper {
+    using head = Head;
+    using tail = TypeList<Tail...>;
+  };
+
+ public:
+  using head = typename HeadTailHelper<Ts...>::head;
+  using tail = typename HeadTailHelper<Ts...>::tail;
+};
+
+template <>
+class TypeList<> : public TypeListCommon<> {
+};
+
+/** @} */
+
+}  // namespace meta
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_META_TYPE_LIST_HPP_


[08/32] incubator-quickstep git commit: Add common-subexpression support.

Posted by ji...@apache.org.
Add common-subexpression support.


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

Branch: refs/heads/new-op
Commit: 8169306c2923d68235ba3c0c8df4c53f5eee9a68
Parents: b19bb94
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Apr 20 15:28:12 2017 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Sun Apr 23 14:57:02 2017 -0500

----------------------------------------------------------------------
 expressions/CMakeLists.txt                      |   6 +
 expressions/Expression.hpp                      |  53 ++
 expressions/ExpressionFactories.cpp             |  10 +
 expressions/Expressions.proto                   |   8 +
 expressions/predicate/CMakeLists.txt            |   4 +-
 expressions/predicate/ComparisonPredicate.cpp   |  87 ++-
 expressions/predicate/ComparisonPredicate.hpp   |  12 +
 expressions/predicate/NegationPredicate.cpp     |  21 +
 expressions/predicate/NegationPredicate.hpp     |  12 +
 expressions/predicate/Predicate.cpp             |  16 +
 expressions/predicate/Predicate.hpp             |  20 +-
 expressions/predicate/PredicateWithList.cpp     |  54 ++
 expressions/predicate/PredicateWithList.hpp     |  11 +
 expressions/scalar/CMakeLists.txt               |  18 +
 expressions/scalar/Scalar.cpp                   |  15 +
 expressions/scalar/Scalar.hpp                   |  40 +-
 expressions/scalar/ScalarAttribute.cpp          |  42 +-
 expressions/scalar/ScalarAttribute.hpp          |  22 +-
 expressions/scalar/ScalarBinaryExpression.cpp   | 257 +++---
 expressions/scalar/ScalarBinaryExpression.hpp   |  23 +-
 expressions/scalar/ScalarCaseExpression.cpp     | 124 ++-
 expressions/scalar/ScalarCaseExpression.hpp     |  33 +-
 expressions/scalar/ScalarLiteral.cpp            |  48 +-
 expressions/scalar/ScalarLiteral.hpp            |  23 +-
 expressions/scalar/ScalarSharedExpression.cpp   | 141 ++++
 expressions/scalar/ScalarSharedExpression.hpp   | 127 +++
 expressions/scalar/ScalarUnaryExpression.cpp    |  84 +-
 expressions/scalar/ScalarUnaryExpression.hpp    |  22 +-
 .../tests/ScalarCaseExpression_unittest.cpp     |  33 +-
 query_optimizer/CMakeLists.txt                  |   3 +
 query_optimizer/PhysicalGenerator.cpp           |  35 +-
 .../expressions/AttributeReference.cpp          |  19 +
 .../expressions/AttributeReference.hpp          |   6 +
 .../expressions/BinaryExpression.cpp            |  37 +
 .../expressions/BinaryExpression.hpp            |   4 +
 query_optimizer/expressions/CMakeLists.txt      |  32 +-
 query_optimizer/expressions/Cast.cpp            |  17 +
 query_optimizer/expressions/Cast.hpp            |   4 +
 .../expressions/CommonSubexpression.cpp         |  72 ++
 .../expressions/CommonSubexpression.hpp         | 141 ++++
 query_optimizer/expressions/ExpressionType.hpp  |   3 +-
 query_optimizer/expressions/ExpressionUtil.hpp  |   6 +-
 query_optimizer/expressions/NamedExpression.hpp |  13 -
 query_optimizer/expressions/PatternMatcher.hpp  |   9 +-
 query_optimizer/expressions/Scalar.hpp          |  41 +
 query_optimizer/expressions/ScalarLiteral.cpp   |  23 +
 query_optimizer/expressions/ScalarLiteral.hpp   |   5 +
 query_optimizer/expressions/SimpleCase.cpp      |  46 ++
 query_optimizer/expressions/SimpleCase.hpp      |   5 +
 query_optimizer/expressions/UnaryExpression.cpp |  17 +
 query_optimizer/expressions/UnaryExpression.hpp |   5 +
 query_optimizer/rules/CMakeLists.txt            |  66 ++
 query_optimizer/rules/CollapseSelection.cpp     |  59 ++
 query_optimizer/rules/CollapseSelection.hpp     |  62 ++
 .../rules/ExtractCommonSubexpression.cpp        | 376 +++++++++
 .../rules/ExtractCommonSubexpression.hpp        | 179 +++++
 .../rules/ReuseAggregateExpressions.cpp         | 349 +++++++++
 .../rules/ReuseAggregateExpressions.hpp         | 154 ++++
 .../tests/execution_generator/CMakeLists.txt    |  12 +
 .../CommonSubexpression.test                    |  52 ++
 .../tests/physical_generator/CMakeLists.txt     |   4 +
 .../physical_generator/CommonSubexpression.test | 772 +++++++++++++++++++
 .../tests/physical_generator/Select.test        | 112 +--
 relational_operators/CMakeLists.txt             |   2 +
 relational_operators/HashJoinOperator.cpp       |  51 +-
 .../NestedLoopsJoinOperator.cpp                 |   6 +-
 storage/AggregationOperationState.cpp           |   4 +-
 storage/CMakeLists.txt                          |   3 +
 storage/StorageBlock.cpp                        |  10 +-
 storage/WindowAggregationOperationState.cpp     |   8 +-
 types/containers/CMakeLists.txt                 |   3 +-
 types/containers/ColumnVector.hpp               |  14 +-
 types/containers/ColumnVectorsValueAccessor.hpp |  23 +-
 .../binary_operations/AddBinaryOperation.hpp    |   4 +
 .../binary_operations/BinaryOperation.hpp       |  13 +
 .../MultiplyBinaryOperation.hpp                 |   4 +
 utility/CMakeLists.txt                          |   7 +
 utility/ColumnVectorCache.hpp                   |  89 +++
 utility/HashError.hpp                           |  62 ++
 79 files changed, 4040 insertions(+), 369 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/CMakeLists.txt b/expressions/CMakeLists.txt
index b1f1fb1..33606cd 100644
--- a/expressions/CMakeLists.txt
+++ b/expressions/CMakeLists.txt
@@ -25,12 +25,16 @@ QS_PROTOBUF_GENERATE_CPP(expressions_Expressions_proto_srcs
                          expressions_Expressions_proto_hdrs
                          Expressions.proto)
 
+add_library(quickstep_expressions_Expression ../empty_src.cpp Expression.hpp)
 add_library(quickstep_expressions_ExpressionFactories
             ExpressionFactories.cpp
             ExpressionFactories.hpp)
 add_library(quickstep_expressions_Expressions_proto
             ${expressions_Expressions_proto_srcs})
 
+target_link_libraries(quickstep_expressions_Expression
+                      quickstep_utility_Macros
+                      quickstep_utility_TreeStringSerializable)
 target_link_libraries(quickstep_expressions_ExpressionFactories
                       glog
                       quickstep_catalog_CatalogDatabaseLite
@@ -48,6 +52,7 @@ target_link_libraries(quickstep_expressions_ExpressionFactories
                       quickstep_expressions_scalar_ScalarBinaryExpression
                       quickstep_expressions_scalar_ScalarCaseExpression
                       quickstep_expressions_scalar_ScalarLiteral
+                      quickstep_expressions_scalar_ScalarSharedExpression
                       quickstep_expressions_scalar_ScalarUnaryExpression
                       quickstep_types_TypeFactory
                       quickstep_types_TypedValue
@@ -64,6 +69,7 @@ target_link_libraries(quickstep_expressions_Expressions_proto
 # Module all-in-one library:
 add_library(quickstep_expressions ../empty_src.cpp ExpressionsModule.hpp)
 target_link_libraries(quickstep_expressions
+                      quickstep_expressions_Expression
                       quickstep_expressions_ExpressionFactories
                       quickstep_expressions_Expressions_proto
                       quickstep_expressions_aggregation

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/Expression.hpp
----------------------------------------------------------------------
diff --git a/expressions/Expression.hpp b/expressions/Expression.hpp
new file mode 100644
index 0000000..39815c3
--- /dev/null
+++ b/expressions/Expression.hpp
@@ -0,0 +1,53 @@
+/**
+ * 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_EXPRESSIONS_EXPRESSION_HPP_
+#define QUICKSTEP_EXPRESSIONS_EXPRESSION_HPP_
+
+#include "utility/Macros.hpp"
+#include "utility/TreeStringSerializable.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Expressions
+ *  @{
+ */
+
+/**
+ * @brief Base class for all expressions (scalars and predicates) in the backend.
+ */
+class Expression : public TreeStringSerializable<const Expression*> {
+ public:
+  /**
+   * @brief Virtual destructor.
+   **/
+  virtual ~Expression() {}
+
+ protected:
+  Expression() {}
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(Expression);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_EXPRESSIONS_EXPRESSION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/ExpressionFactories.cpp
----------------------------------------------------------------------
diff --git a/expressions/ExpressionFactories.cpp b/expressions/ExpressionFactories.cpp
index 01d22a0..871db50 100644
--- a/expressions/ExpressionFactories.cpp
+++ b/expressions/ExpressionFactories.cpp
@@ -39,6 +39,7 @@
 #include "expressions/scalar/ScalarBinaryExpression.hpp"
 #include "expressions/scalar/ScalarCaseExpression.hpp"
 #include "expressions/scalar/ScalarLiteral.hpp"
+#include "expressions/scalar/ScalarSharedExpression.hpp"
 #include "expressions/scalar/ScalarUnaryExpression.hpp"
 #include "types/TypeFactory.hpp"
 #include "types/TypedValue.hpp"
@@ -210,6 +211,11 @@ Scalar* ScalarFactory::ReconstructFromProto(const serialization::Scalar &proto,
                                       std::move(result_expressions),
                                       else_result_expression.release());
     }
+    case serialization::Scalar::SHARED_EXPRESSION: {
+      return new ScalarSharedExpression(
+          proto.GetExtension(serialization::ScalarSharedExpression::share_id),
+          ReconstructFromProto(proto.GetExtension(serialization::ScalarSharedExpression::operand), database));
+    }
     default:
       FATAL_ERROR("Unknown Scalar data source in ScalarFactory::ReconstructFromProto");
   }
@@ -302,6 +308,10 @@ bool ScalarFactory::ProtoIsValid(const serialization::Scalar &proto,
       // Everything checks out.
       return true;
     }
+    case serialization::Scalar::SHARED_EXPRESSION: {
+      return proto.HasExtension(serialization::ScalarSharedExpression::share_id)
+             && proto.HasExtension(serialization::ScalarSharedExpression::operand);
+    }
     default: {
       break;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/Expressions.proto
----------------------------------------------------------------------
diff --git a/expressions/Expressions.proto b/expressions/Expressions.proto
index 8d923c5..8b4611e 100644
--- a/expressions/Expressions.proto
+++ b/expressions/Expressions.proto
@@ -50,6 +50,7 @@ message Scalar {
     UNARY_EXPRESSION = 2;
     BINARY_EXPRESSION = 3;
     CASE_EXPRESSION = 4;
+    SHARED_EXPRESSION = 5;
   }
 
   required ScalarDataSource data_source = 1;
@@ -123,3 +124,10 @@ message ScalarCaseExpression {
     optional Scalar else_result_expression = 163;
   }
 }
+
+message ScalarSharedExpression {
+  extend Scalar {
+    optional int32 share_id = 196;
+    optional Scalar operand = 197;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/predicate/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/predicate/CMakeLists.txt b/expressions/predicate/CMakeLists.txt
index b90562c..04abfc7 100644
--- a/expressions/predicate/CMakeLists.txt
+++ b/expressions/predicate/CMakeLists.txt
@@ -35,7 +35,7 @@ add_library(quickstep_expressions_predicate_PredicateCost
             ../../empty_src.cpp
             PredicateCost.hpp)
 add_library(quickstep_expressions_predicate_PredicateWithList
-            ../../empty_src.cpp
+            PredicateWithList.cpp
             PredicateWithList.hpp)
 add_library(quickstep_expressions_predicate_TrivialPredicates
             ../../empty_src.cpp
@@ -61,6 +61,7 @@ target_link_libraries(quickstep_expressions_predicate_ComparisonPredicate
                       quickstep_types_containers_ColumnVector
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_comparisons_Comparison
+                      quickstep_types_operations_comparisons_ComparisonID
                       quickstep_utility_Macros
                       quickstep_utility_PtrVector)
 target_link_libraries(quickstep_expressions_predicate_ConjunctionPredicate
@@ -92,6 +93,7 @@ target_link_libraries(quickstep_expressions_predicate_NegationPredicate
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_predicate_Predicate
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_Expression
                       quickstep_expressions_Expressions_proto
                       quickstep_storage_StorageBlockInfo
                       quickstep_storage_TupleIdSequence

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/predicate/ComparisonPredicate.cpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/ComparisonPredicate.cpp b/expressions/predicate/ComparisonPredicate.cpp
index 5f8612e..2f7b84b 100644
--- a/expressions/predicate/ComparisonPredicate.cpp
+++ b/expressions/predicate/ComparisonPredicate.cpp
@@ -23,6 +23,7 @@
 #include <limits>
 #include <memory>
 #include <string>
+#include <type_traits>
 #include <utility>
 #include <vector>
 
@@ -41,6 +42,7 @@
 #include "types/containers/ColumnVector.hpp"
 #include "types/operations/Operation.pb.h"
 #include "types/operations/comparisons/Comparison.hpp"
+#include "types/operations/comparisons/ComparisonID.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrVector.hpp"
 
@@ -190,18 +192,20 @@ TupleIdSequence* ComparisonPredicate::getAllMatches(
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
     if (short_circuit_adapter) {
-      std::unique_ptr<ColumnVector> right_values(right_operand_->getAllValues(
+      ColumnVectorPtr right_values(right_operand_->getAllValues(
           short_circuit_adapter.get(),
-          sub_blocks_ref));
+          sub_blocks_ref,
+          nullptr /* cv_cache */));
       return fast_comparator_->compareStaticValueAndColumnVector(
           left_operand_->getStaticValue(),
           *right_values,
           nullptr,
           filter);
     } else {
-      std::unique_ptr<ColumnVector> right_values(right_operand_->getAllValues(
+      ColumnVectorPtr right_values(right_operand_->getAllValues(
           accessor,
-          sub_blocks_ref));
+          sub_blocks_ref,
+          nullptr /* cv_cache */));
       return fast_comparator_->compareStaticValueAndColumnVector(
           left_operand_->getStaticValue(),
           *right_values,
@@ -222,18 +226,20 @@ TupleIdSequence* ComparisonPredicate::getAllMatches(
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
     if (short_circuit_adapter) {
-      std::unique_ptr<ColumnVector> left_values(left_operand_->getAllValues(
+      ColumnVectorPtr left_values(left_operand_->getAllValues(
           short_circuit_adapter.get(),
-          sub_blocks_ref));
+          sub_blocks_ref,
+          nullptr /* cv_cache */));
       return fast_comparator_->compareColumnVectorAndStaticValue(
           *left_values,
           right_operand_->getStaticValue(),
           nullptr,
           filter);
     } else {
-      std::unique_ptr<ColumnVector> left_values(left_operand_->getAllValues(
+      ColumnVectorPtr left_values(left_operand_->getAllValues(
           accessor,
-          sub_blocks_ref));
+          sub_blocks_ref,
+          nullptr /* cv_cache */));
       return fast_comparator_->compareColumnVectorAndStaticValue(
           *left_values,
           right_operand_->getStaticValue(),
@@ -255,9 +261,10 @@ TupleIdSequence* ComparisonPredicate::getAllMatches(
                                                             filter);
       } else {
         if (short_circuit_adapter) {
-          std::unique_ptr<ColumnVector> right_values(right_operand_->getAllValues(
+          ColumnVectorPtr right_values(right_operand_->getAllValues(
               short_circuit_adapter.get(),
-              sub_blocks_ref));
+              sub_blocks_ref,
+              nullptr /* cv_cache */));
           return fast_comparator_->compareValueAccessorAndColumnVector(
               short_circuit_adapter.get(),
               left_operand_attr_id,
@@ -265,9 +272,10 @@ TupleIdSequence* ComparisonPredicate::getAllMatches(
               nullptr,
               filter);
         } else {
-          std::unique_ptr<ColumnVector> right_values(right_operand_->getAllValues(
+          ColumnVectorPtr right_values(right_operand_->getAllValues(
               accessor,
-              sub_blocks_ref));
+              sub_blocks_ref,
+              nullptr /* cv_cache */));
           return fast_comparator_->compareValueAccessorAndColumnVector(accessor,
                                                                        left_operand_attr_id,
                                                                        *right_values,
@@ -277,9 +285,10 @@ TupleIdSequence* ComparisonPredicate::getAllMatches(
       }
     } else if (right_operand_attr_id != -1) {
       if (short_circuit_adapter) {
-        std::unique_ptr<ColumnVector> left_values(left_operand_->getAllValues(
+        ColumnVectorPtr left_values(left_operand_->getAllValues(
             short_circuit_adapter.get(),
-            sub_blocks_ref));
+            sub_blocks_ref,
+            nullptr /* cv_cache */));
         return fast_comparator_->compareColumnVectorAndValueAccessor(
             *left_values,
             short_circuit_adapter.get(),
@@ -287,9 +296,10 @@ TupleIdSequence* ComparisonPredicate::getAllMatches(
             nullptr,
             filter);
       } else {
-        std::unique_ptr<ColumnVector> left_values(left_operand_->getAllValues(
+        ColumnVectorPtr left_values(left_operand_->getAllValues(
             accessor,
-            sub_blocks_ref));
+            sub_blocks_ref,
+            nullptr /* cv_cache */));
         return fast_comparator_->compareColumnVectorAndValueAccessor(*left_values,
                                                                      accessor,
                                                                      right_operand_attr_id,
@@ -300,23 +310,27 @@ TupleIdSequence* ComparisonPredicate::getAllMatches(
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
     if (short_circuit_adapter) {
-      std::unique_ptr<ColumnVector> left_values(left_operand_->getAllValues(
+      ColumnVectorPtr left_values(left_operand_->getAllValues(
           short_circuit_adapter.get(),
-          sub_blocks_ref));
-      std::unique_ptr<ColumnVector> right_values(right_operand_->getAllValues(
+          sub_blocks_ref,
+          nullptr /* cv_cache */));
+      ColumnVectorPtr right_values(right_operand_->getAllValues(
           short_circuit_adapter.get(),
-          sub_blocks_ref));
+          sub_blocks_ref,
+          nullptr /* cv_cache */));
       return fast_comparator_->compareColumnVectors(*left_values,
                                                     *right_values,
                                                     nullptr,
                                                     filter);
     } else {
-      std::unique_ptr<ColumnVector> left_values(left_operand_->getAllValues(
+      ColumnVectorPtr left_values(left_operand_->getAllValues(
           accessor,
-          sub_blocks_ref));
-      std::unique_ptr<ColumnVector> right_values(right_operand_->getAllValues(
+          sub_blocks_ref,
+          nullptr /* cv_cache */));
+      ColumnVectorPtr right_values(right_operand_->getAllValues(
           accessor,
-          sub_blocks_ref));
+          sub_blocks_ref,
+          nullptr /* cv_cache */));
       return fast_comparator_->compareColumnVectors(*left_values,
                                                     *right_values,
                                                     filter,
@@ -373,4 +387,29 @@ void ComparisonPredicate::initHelper(bool own_children) {
   }
 }
 
+void ComparisonPredicate::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<const Expression*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const Expression*>> *container_child_fields) const {
+  Predicate::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->emplace_back("comparison");
+  inline_field_values->emplace_back(
+      kComparisonNames[static_cast<std::underlying_type<ComparisonID>::type>(
+          comparison_.getComparisonID())]);
+
+  non_container_child_field_names->emplace_back("left_operand");
+  non_container_child_fields->emplace_back(left_operand_.get());
+  non_container_child_field_names->emplace_back("right_operand");
+  non_container_child_fields->emplace_back(right_operand_.get());
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/predicate/ComparisonPredicate.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/ComparisonPredicate.hpp b/expressions/predicate/ComparisonPredicate.hpp
index 9030857..1ef2cb1 100644
--- a/expressions/predicate/ComparisonPredicate.hpp
+++ b/expressions/predicate/ComparisonPredicate.hpp
@@ -21,7 +21,9 @@
 #define QUICKSTEP_EXPRESSIONS_PREDICATE_COMPARISON_PREDICATE_HPP_
 
 #include <memory>
+#include <string>
 #include <utility>
+#include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/Expressions.pb.h"
@@ -35,6 +37,7 @@
 
 namespace quickstep {
 
+class Expression;
 class TupleIdSequence;
 class ValueAccessor;
 
@@ -137,6 +140,15 @@ class ComparisonPredicate : public Predicate {
    **/
   std::pair<bool, attribute_id> getAttributeFromAttributeLiteralComparison() const;
 
+ 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<const Expression*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const Expression*>> *container_child_fields) const override;
+
  private:
   const Comparison &comparison_;
   std::unique_ptr<Scalar> left_operand_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/predicate/NegationPredicate.cpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/NegationPredicate.cpp b/expressions/predicate/NegationPredicate.cpp
index bee1c8d..92a8411 100644
--- a/expressions/predicate/NegationPredicate.cpp
+++ b/expressions/predicate/NegationPredicate.cpp
@@ -19,6 +19,9 @@
 
 #include "expressions/predicate/NegationPredicate.hpp"
 
+#include <string>
+#include <vector>
+
 #include "expressions/Expressions.pb.h"
 #include "expressions/predicate/Predicate.hpp"
 #include "storage/TupleIdSequence.hpp"
@@ -120,4 +123,22 @@ Predicate* NegationPredicate::NegatePredicate(Predicate *operand) {
   }
 }
 
+void NegationPredicate::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<const Expression*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const Expression*>> *container_child_fields) const {
+  Predicate::getFieldStringItems(inline_field_names,
+                                 inline_field_values,
+                                 non_container_child_field_names,
+                                 non_container_child_fields,
+                                 container_child_field_names,
+                                 container_child_fields);
+
+  non_container_child_field_names->emplace_back("operand");
+  non_container_child_fields->emplace_back(operand_.get());
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/predicate/NegationPredicate.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/NegationPredicate.hpp b/expressions/predicate/NegationPredicate.hpp
index 33c6df8..ec005ea 100644
--- a/expressions/predicate/NegationPredicate.hpp
+++ b/expressions/predicate/NegationPredicate.hpp
@@ -21,6 +21,8 @@
 #define QUICKSTEP_EXPRESSIONS_PREDICATE_NEGATION_PREDICATE_HPP_
 
 #include <memory>
+#include <string>
+#include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
 #include "expressions/Expressions.pb.h"
@@ -30,6 +32,7 @@
 
 namespace quickstep {
 
+class Expression;
 class TupleIdSequence;
 class ValueAccessor;
 
@@ -105,6 +108,15 @@ class NegationPredicate : public Predicate {
 
   bool getStaticResult() const override;
 
+ 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<const Expression*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const Expression*>> *container_child_fields) const override;
+
  private:
   void initHelper();
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/predicate/Predicate.cpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/Predicate.cpp b/expressions/predicate/Predicate.cpp
index 006e8f1..1f61d62 100644
--- a/expressions/predicate/Predicate.cpp
+++ b/expressions/predicate/Predicate.cpp
@@ -19,6 +19,9 @@
 
 #include "expressions/predicate/Predicate.hpp"
 
+#include <string>
+#include <vector>
+
 #include "storage/TupleIdSequence.hpp"
 #include "storage/ValueAccessor.hpp"
 #include "utility/Macros.hpp"
@@ -57,4 +60,17 @@ TupleIdSequence* Predicate::GenerateSequenceForStaticResult(
   return result;
 }
 
+void Predicate::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<const Expression*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const Expression*>> *container_child_fields) const {
+  if (hasStaticResult()) {
+    inline_field_names->emplace_back("static_result");
+    inline_field_values->emplace_back(getStaticResult() ? "true" : "false");
+  }
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/predicate/Predicate.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/Predicate.hpp b/expressions/predicate/Predicate.hpp
index 5fb3ef5..6a2ba6d 100644
--- a/expressions/predicate/Predicate.hpp
+++ b/expressions/predicate/Predicate.hpp
@@ -20,7 +20,12 @@
 #ifndef QUICKSTEP_EXPRESSIONS_PREDICATE_PREDICATE_HPP_
 #define QUICKSTEP_EXPRESSIONS_PREDICATE_PREDICATE_HPP_
 
+#include <string>
+#include <type_traits>
+#include <vector>
+
 #include "catalog/CatalogTypedefs.hpp"
+#include "expressions/Expression.hpp"
 #include "expressions/Expressions.pb.h"
 #include "storage/StorageBlockInfo.hpp"
 #include "utility/Macros.hpp"
@@ -39,7 +44,7 @@ struct SubBlocksReference;
 /**
  * @brief Base class for all predicates.
  **/
-class Predicate {
+class Predicate : public Expression {
  public:
   /**
    * @brief The possible types of predicates.
@@ -67,6 +72,11 @@ class Predicate {
   virtual ~Predicate() {
   }
 
+  std::string getName() const override {
+    return kPredicateTypeNames[
+        static_cast<std::underlying_type<PredicateType>::type>(getPredicateType())];
+  }
+
   /**
    * @brief Serialize this predicate in Protocol Buffer form.
    *
@@ -189,6 +199,14 @@ class Predicate {
   virtual bool getStaticResult() const;
 
  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<const Expression*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const Expression*>> *container_child_fields) const override;
+
   Predicate() {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/predicate/PredicateWithList.cpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/PredicateWithList.cpp b/expressions/predicate/PredicateWithList.cpp
new file mode 100644
index 0000000..6b3d7a7
--- /dev/null
+++ b/expressions/predicate/PredicateWithList.cpp
@@ -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.
+ **/
+
+#include "expressions/predicate/PredicateWithList.hpp"
+
+#include <string>
+#include <vector>
+
+namespace quickstep {
+
+void PredicateWithList::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<const Expression*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const Expression*>> *container_child_fields) const {
+  Predicate::getFieldStringItems(inline_field_names,
+                                 inline_field_values,
+                                 non_container_child_field_names,
+                                 non_container_child_fields,
+                                 container_child_field_names,
+                                 container_child_fields);
+
+  container_child_field_names->emplace_back("static_operand_list");
+  container_child_fields->emplace_back();
+  for (const auto &static_operand : static_operand_list_) {
+    container_child_fields->back().emplace_back(&static_operand);
+  }
+
+  container_child_field_names->emplace_back("dynamic_operand_list");
+  container_child_fields->emplace_back();
+  for (const auto &dynamic_operand : dynamic_operand_list_) {
+    container_child_fields->back().emplace_back(&dynamic_operand);
+  }
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/predicate/PredicateWithList.hpp
----------------------------------------------------------------------
diff --git a/expressions/predicate/PredicateWithList.hpp b/expressions/predicate/PredicateWithList.hpp
index b1bf7e5..c6fb99d 100644
--- a/expressions/predicate/PredicateWithList.hpp
+++ b/expressions/predicate/PredicateWithList.hpp
@@ -20,6 +20,9 @@
 #ifndef QUICKSTEP_EXPRESSIONS_PREDICATE_PREDICATE_WITH_LIST_HPP_
 #define QUICKSTEP_EXPRESSIONS_PREDICATE_PREDICATE_WITH_LIST_HPP_
 
+#include <string>
+#include <vector>
+
 #include "expressions/predicate/Predicate.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrList.hpp"
@@ -58,6 +61,14 @@ class PredicateWithList : public Predicate {
   }
 
  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<const Expression*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const Expression*>> *container_child_fields) const override;
+
   PtrList<Predicate> static_operand_list_;
   PtrList<Predicate> dynamic_operand_list_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/scalar/CMakeLists.txt b/expressions/scalar/CMakeLists.txt
index 8f509da..6b52231 100644
--- a/expressions/scalar/CMakeLists.txt
+++ b/expressions/scalar/CMakeLists.txt
@@ -29,6 +29,9 @@ add_library(quickstep_expressions_scalar_ScalarCaseExpression
 add_library(quickstep_expressions_scalar_ScalarLiteral
             ScalarLiteral.cpp
             ScalarLiteral.hpp)
+add_library(quickstep_expressions_scalar_ScalarSharedExpression
+            ScalarSharedExpression.cpp
+            ScalarSharedExpression.hpp)
 add_library(quickstep_expressions_scalar_ScalarUnaryExpression
             ScalarUnaryExpression.cpp
             ScalarUnaryExpression.hpp)
@@ -36,9 +39,11 @@ add_library(quickstep_expressions_scalar_ScalarUnaryExpression
 # Link dependencies:
 target_link_libraries(quickstep_expressions_scalar_Scalar
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_Expression
                       quickstep_expressions_Expressions_proto
                       quickstep_storage_StorageBlockInfo
                       quickstep_types_TypedValue
+                      quickstep_types_containers_ColumnVector
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_scalar_ScalarAttribute
                       quickstep_catalog_CatalogAttribute
@@ -65,6 +70,7 @@ target_link_libraries(quickstep_expressions_scalar_ScalarBinaryExpression
                       quickstep_types_containers_ColumnVector
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_binaryoperations_BinaryOperation
+                      quickstep_types_operations_binaryoperations_BinaryOperationID
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_scalar_ScalarCaseExpression
                       quickstep_catalog_CatalogTypedefs
@@ -92,6 +98,16 @@ target_link_libraries(quickstep_expressions_scalar_ScalarLiteral
                       quickstep_types_TypedValue_proto
                       quickstep_types_containers_ColumnVector
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_expressions_scalar_ScalarSharedExpression
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_expressions_Expressions_proto
+                      quickstep_expressions_scalar_Scalar
+                      quickstep_storage_StorageBlockInfo
+                      quickstep_storage_ValueAccessor
+                      quickstep_types_TypedValue
+                      quickstep_types_containers_ColumnVector
+                      quickstep_utility_ColumnVectorCache
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_expressions_scalar_ScalarUnaryExpression
                       quickstep_catalog_CatalogTypedefs
                       quickstep_expressions_Expressions_proto
@@ -104,6 +120,7 @@ target_link_libraries(quickstep_expressions_scalar_ScalarUnaryExpression
                       quickstep_types_containers_ColumnVector
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_unaryoperations_UnaryOperation
+                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_Macros)
 
 # Submodule all-in-one library:
@@ -114,6 +131,7 @@ target_link_libraries(quickstep_expressions_scalar
                       quickstep_expressions_scalar_ScalarBinaryExpression
                       quickstep_expressions_scalar_ScalarCaseExpression
                       quickstep_expressions_scalar_ScalarLiteral
+                      quickstep_expressions_scalar_ScalarSharedExpression
                       quickstep_expressions_scalar_ScalarUnaryExpression)
 
 # Tests:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/Scalar.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/Scalar.cpp b/expressions/scalar/Scalar.cpp
index a1c436c..c552d8b 100644
--- a/expressions/scalar/Scalar.cpp
+++ b/expressions/scalar/Scalar.cpp
@@ -19,6 +19,9 @@
 
 #include "expressions/scalar/Scalar.hpp"
 
+#include <string>
+#include <vector>
+
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -28,6 +31,7 @@ const char *Scalar::kScalarDataSourceNames[] = {
   "Attribute",
   "UnaryExpression",
   "BinaryExpression",
+  "SharedExpression",
   "SimpleCase"
 };
 
@@ -35,4 +39,15 @@ const TypedValue& Scalar::getStaticValue() const {
   FATAL_ERROR("Called getStaticValue() on a Scalar which does not have a static value");
 }
 
+void Scalar::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<const Expression*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const Expression*>> *container_child_fields) const {
+  inline_field_names->emplace_back("result_type");
+  inline_field_values->emplace_back(type_.getName());
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/Scalar.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/Scalar.hpp b/expressions/scalar/Scalar.hpp
index 2db850a..472b71c 100644
--- a/expressions/scalar/Scalar.hpp
+++ b/expressions/scalar/Scalar.hpp
@@ -20,18 +20,22 @@
 #ifndef QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_HPP_
 #define QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_HPP_
 
+#include <string>
+#include <type_traits>
 #include <utility>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
+#include "expressions/Expression.hpp"
 #include "expressions/Expressions.pb.h"
 #include "storage/StorageBlockInfo.hpp"
 #include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
-class ColumnVector;
+class ColumnVectorCache;
 class Type;
 class ValueAccessor;
 
@@ -44,7 +48,7 @@ struct SubBlocksReference;
 /**
  * @brief Base class for anything which evaluates to a Scalar value.
  **/
-class Scalar {
+class Scalar : public Expression {
  public:
   /**
    * @brief The possible provenance of Scalar values.
@@ -55,6 +59,7 @@ class Scalar {
     kUnaryExpression,
     kBinaryExpression,
     kCaseExpression,
+    kSharedExpression,
     kNumScalarDataSources  // Not a real ScalarDataSource, exists for counting purposes.
   };
 
@@ -70,6 +75,11 @@ class Scalar {
   virtual ~Scalar() {
   }
 
+  std::string getName() const override {
+    return kScalarDataSourceNames[
+        static_cast<std::underlying_type<ScalarDataSource>::type>(getDataSource())];
+  }
+
   /**
    * @brief Serialize this scalar in Protocol Buffer form.
    *
@@ -197,11 +207,14 @@ class Scalar {
    *        fast-path (non-scan) evaluation of Predicates embedded in this
    *        scalar (e.g. WHEN predicates in ScalarCaseExpression). May be NULL,
    *        in which case scan-based evaluation is always used.
+   * @param cv_cache If non-NULL, used as memoization table that is updated and
+   *        looked up during evaluation, for results of common subexpressions.
    * @return A ColumnVector of this Scalar's values for each tuple accesible
    *         via accessor.
    **/
-  virtual ColumnVector* getAllValues(ValueAccessor *accessor,
-                                     const SubBlocksReference *sub_blocks_ref) const = 0;
+  virtual ColumnVectorPtr getAllValues(ValueAccessor *accessor,
+                                       const SubBlocksReference *sub_blocks_ref,
+                                       ColumnVectorCache *cv_cache) const = 0;
 
   /**
    * @brief Get this Scalar's value for all specified joined tuples from two
@@ -215,19 +228,30 @@ class Scalar {
    *        from the right relation.
    * @param joined_tuple_ids A series of pairs of tuple ids from the left and
    *        right relations that will be joined.
+   * @param cv_cache If non-NULL, used as memoization table that is updated and
+   *        looked up during evaluation, for results of common subexpressions.
    * @return A ColumnVector of this Scalar's values for all the joined tuples
    *         specified by joined_tuple_ids.
    **/
-  virtual ColumnVector* getAllValuesForJoin(
+  virtual ColumnVectorPtr getAllValuesForJoin(
       const relation_id left_relation_id,
       ValueAccessor *left_accessor,
       const relation_id right_relation_id,
       ValueAccessor *right_accessor,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
+      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+      ColumnVectorCache *cv_cache) const = 0;
 
  protected:
-  explicit Scalar(const Type &type) : type_(type) {
-  }
+  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<const Expression*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const Expression*>> *container_child_fields) const override;
+
+  explicit Scalar(const Type &type)
+      : Expression(), type_(type) {}
 
   const Type &type_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/ScalarAttribute.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarAttribute.cpp b/expressions/scalar/ScalarAttribute.cpp
index cc42084..036eeba 100644
--- a/expressions/scalar/ScalarAttribute.cpp
+++ b/expressions/scalar/ScalarAttribute.cpp
@@ -19,6 +19,7 @@
 
 #include "expressions/scalar/ScalarAttribute.hpp"
 
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -88,13 +89,15 @@ relation_id ScalarAttribute::getRelationIdForValueAccessor() const {
   return attribute_.getParent().getID();
 }
 
-ColumnVector* ScalarAttribute::getAllValues(ValueAccessor *accessor,
-                                            const SubBlocksReference *sub_blocks_ref) const {
+ColumnVectorPtr ScalarAttribute::getAllValues(
+    ValueAccessor *accessor,
+    const SubBlocksReference *sub_blocks_ref,
+    ColumnVectorCache *cv_cache) const {
   const attribute_id attr_id = attribute_.getID();
   const Type &result_type = attribute_.getType();
   return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
       accessor,
-      [&attr_id, &result_type](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      [&attr_id, &result_type](auto *accessor) -> ColumnVectorPtr {  // NOLINT(build/c++11)
     if (NativeColumnVector::UsableForType(result_type)) {
       NativeColumnVector *result = new NativeColumnVector(result_type,
                                                           accessor->getNumTuples());
@@ -139,7 +142,7 @@ ColumnVector* ScalarAttribute::getAllValues(ValueAccessor *accessor,
           }
         }
       }
-      return result;
+      return ColumnVectorPtr(result);
     } else {
       IndirectColumnVector *result = new IndirectColumnVector(result_type,
                                                               accessor->getNumTuples());
@@ -147,17 +150,18 @@ ColumnVector* ScalarAttribute::getAllValues(ValueAccessor *accessor,
       while (accessor->next()) {
         result->appendTypedValue(accessor->getTypedValue(attr_id));
       }
-      return result;
+      return ColumnVectorPtr(result);
     }
   });
 }
 
-ColumnVector* ScalarAttribute::getAllValuesForJoin(
+ColumnVectorPtr ScalarAttribute::getAllValuesForJoin(
     const relation_id left_relation_id,
     ValueAccessor *left_accessor,
     const relation_id right_relation_id,
     ValueAccessor *right_accessor,
-    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
+    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+    ColumnVectorCache *cv_cache) const {
   DCHECK((attribute_.getParent().getID() == left_relation_id)
          || (attribute_.getParent().getID() == right_relation_id));
 
@@ -173,7 +177,7 @@ ColumnVector* ScalarAttribute::getAllValuesForJoin(
       [&joined_tuple_ids,
        &attr_id,
        &result_type,
-       &using_left_relation](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+       &using_left_relation](auto *accessor) -> ColumnVectorPtr {  // NOLINT(build/c++11)
     if (NativeColumnVector::UsableForType(result_type)) {
       NativeColumnVector *result = new NativeColumnVector(result_type,
                                                           joined_tuple_ids.size());
@@ -196,7 +200,7 @@ ColumnVector* ScalarAttribute::getAllValuesForJoin(
                   using_left_relation ? joined_pair.first : joined_pair.second));
         }
       }
-      return result;
+      return ColumnVectorPtr(result);
     } else {
       IndirectColumnVector *result = new IndirectColumnVector(result_type,
                                                               joined_tuple_ids.size());
@@ -206,9 +210,27 @@ ColumnVector* ScalarAttribute::getAllValuesForJoin(
                   attr_id,
                   using_left_relation ? joined_pair.first : joined_pair.second));
       }
-      return result;
+      return ColumnVectorPtr(result);
     }
   });
 }
 
+void ScalarAttribute::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<const Expression*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const Expression*>> *container_child_fields) const {
+  Scalar::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->emplace_back("attribute");
+  inline_field_values->emplace_back(std::to_string(attribute_.getID()));
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/ScalarAttribute.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarAttribute.hpp b/expressions/scalar/ScalarAttribute.hpp
index c6a41df..4d30fe9 100644
--- a/expressions/scalar/ScalarAttribute.hpp
+++ b/expressions/scalar/ScalarAttribute.hpp
@@ -20,6 +20,7 @@
 #ifndef QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_ATTRIBUTE_HPP_
 #define QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_ATTRIBUTE_HPP_
 
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -28,12 +29,13 @@
 #include "expressions/scalar/Scalar.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
 class CatalogAttribute;
-class ColumnVector;
+class ColumnVectorCache;
 class ValueAccessor;
 
 struct SubBlocksReference;
@@ -77,21 +79,31 @@ class ScalarAttribute : public Scalar {
 
   relation_id getRelationIdForValueAccessor() const override;
 
-  ColumnVector* getAllValues(ValueAccessor *accessor,
-                             const SubBlocksReference *sub_blocks_ref) const override;
+  ColumnVectorPtr getAllValues(ValueAccessor *accessor,
+                               const SubBlocksReference *sub_blocks_ref,
+                               ColumnVectorCache *cv_cache) const override;
 
-  ColumnVector* getAllValuesForJoin(
+  ColumnVectorPtr getAllValuesForJoin(
       const relation_id left_relation_id,
       ValueAccessor *left_accessor,
       const relation_id right_relation_id,
       ValueAccessor *right_accessor,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override;
+      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+      ColumnVectorCache *cv_cache) const override;
 
   const CatalogAttribute& getAttribute() const {
     return attribute_;
   }
 
  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<const Expression*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const Expression*>> *container_child_fields) const override;
+
   const CatalogAttribute &attribute_;
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/ScalarBinaryExpression.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarBinaryExpression.cpp b/expressions/scalar/ScalarBinaryExpression.cpp
index 5fe6cde..b3568f8 100644
--- a/expressions/scalar/ScalarBinaryExpression.cpp
+++ b/expressions/scalar/ScalarBinaryExpression.cpp
@@ -21,6 +21,7 @@
 
 #include <memory>
 #include <string>
+#include <type_traits>
 #include <utility>
 #include <vector>
 
@@ -33,6 +34,7 @@
 #include "types/containers/ColumnVector.hpp"
 #include "types/operations/Operation.pb.h"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
+#include "types/operations/binary_operations/BinaryOperationID.hpp"
 
 #include "glog/logging.h"
 
@@ -101,13 +103,15 @@ TypedValue ScalarBinaryExpression::getValueForJoinedTuples(
   }
 }
 
-ColumnVector* ScalarBinaryExpression::getAllValues(
+ColumnVectorPtr ScalarBinaryExpression::getAllValues(
     ValueAccessor *accessor,
-    const SubBlocksReference *sub_blocks_ref) const {
+    const SubBlocksReference *sub_blocks_ref,
+    ColumnVectorCache *cv_cache) const {
   if (fast_operator_.get() == nullptr) {
-    return ColumnVector::MakeVectorOfValue(getType(),
-                                           static_value_,
-                                           accessor->getNumTuplesVirtual());
+    return ColumnVectorPtr(
+        ColumnVector::MakeVectorOfValue(getType(),
+                                        static_value_,
+                                        accessor->getNumTuplesVirtual()));
   } else {
     // NOTE(chasseur): We don't check if BOTH operands have a static value,
     // because if they did then this expression would also have a static value
@@ -117,35 +121,39 @@ ColumnVector* ScalarBinaryExpression::getAllValues(
       const attribute_id right_operand_attr_id
           = right_operand_->getAttributeIdForValueAccessor();
       if (right_operand_attr_id != -1) {
-        return fast_operator_->applyToStaticValueAndValueAccessor(
-            left_operand_->getStaticValue(),
-            accessor,
-            right_operand_attr_id);
+        return ColumnVectorPtr(
+            fast_operator_->applyToStaticValueAndValueAccessor(
+                left_operand_->getStaticValue(),
+                accessor,
+                right_operand_attr_id));
       }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
-      std::unique_ptr<ColumnVector> right_result(
-          right_operand_->getAllValues(accessor, sub_blocks_ref));
-      return fast_operator_->applyToStaticValueAndColumnVector(
-          left_operand_->getStaticValue(),
-          *right_result);
+      ColumnVectorPtr right_result(
+          right_operand_->getAllValues(accessor, sub_blocks_ref, cv_cache));
+      return ColumnVectorPtr(
+          fast_operator_->applyToStaticValueAndColumnVector(
+              left_operand_->getStaticValue(),
+              *right_result));
     } else if (right_operand_->hasStaticValue()) {
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
       const attribute_id left_operand_attr_id
           = left_operand_->getAttributeIdForValueAccessor();
       if (left_operand_attr_id != -1) {
-        return fast_operator_->applyToValueAccessorAndStaticValue(
-            accessor,
-            left_operand_attr_id,
-            right_operand_->getStaticValue());
+        return ColumnVectorPtr(
+            fast_operator_->applyToValueAccessorAndStaticValue(
+                accessor,
+                left_operand_attr_id,
+                right_operand_->getStaticValue()));
       }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
-      std::unique_ptr<ColumnVector> left_result(
-          left_operand_->getAllValues(accessor, sub_blocks_ref));
-      return fast_operator_->applyToColumnVectorAndStaticValue(
-          *left_result,
-          right_operand_->getStaticValue());
+      ColumnVectorPtr left_result(
+          left_operand_->getAllValues(accessor, sub_blocks_ref, cv_cache));
+      return ColumnVectorPtr(
+          fast_operator_->applyToColumnVectorAndStaticValue(
+              *left_result,
+              right_operand_->getStaticValue()));
     } else {
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
       const attribute_id left_operand_attr_id
@@ -155,44 +163,53 @@ ColumnVector* ScalarBinaryExpression::getAllValues(
 
       if (left_operand_attr_id != -1) {
         if (right_operand_attr_id != -1) {
-          return fast_operator_->applyToSingleValueAccessor(accessor,
-                                                            left_operand_attr_id,
-                                                            right_operand_attr_id);
+          return ColumnVectorPtr(
+              fast_operator_->applyToSingleValueAccessor(
+                  accessor,
+                  left_operand_attr_id,
+                  right_operand_attr_id));
         } else {
-          std::unique_ptr<ColumnVector> right_result(
-              right_operand_->getAllValues(accessor, sub_blocks_ref));
-          return fast_operator_->applyToValueAccessorAndColumnVector(accessor,
-                                                                     left_operand_attr_id,
-                                                                     *right_result);
+          ColumnVectorPtr right_result(
+              right_operand_->getAllValues(accessor, sub_blocks_ref, cv_cache));
+          return ColumnVectorPtr(
+              fast_operator_->applyToValueAccessorAndColumnVector(
+                  accessor,
+                  left_operand_attr_id,
+                  *right_result));
         }
       } else if (right_operand_attr_id != -1) {
-        std::unique_ptr<ColumnVector> left_result(
-            left_operand_->getAllValues(accessor, sub_blocks_ref));
-        return fast_operator_->applyToColumnVectorAndValueAccessor(*left_result,
-                                                                   accessor,
-                                                                   right_operand_attr_id);
+        ColumnVectorPtr left_result(
+            left_operand_->getAllValues(accessor, sub_blocks_ref, cv_cache));
+        return ColumnVectorPtr(
+            fast_operator_->applyToColumnVectorAndValueAccessor(
+                *left_result,
+                accessor,
+                right_operand_attr_id));
       }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
-      std::unique_ptr<ColumnVector> left_result(
-          left_operand_->getAllValues(accessor, sub_blocks_ref));
-      std::unique_ptr<ColumnVector> right_result(
-          right_operand_->getAllValues(accessor, sub_blocks_ref));
-      return fast_operator_->applyToColumnVectors(*left_result, *right_result);
+      ColumnVectorPtr left_result(
+          left_operand_->getAllValues(accessor, sub_blocks_ref, cv_cache));
+      ColumnVectorPtr right_result(
+          right_operand_->getAllValues(accessor, sub_blocks_ref, cv_cache));
+      return ColumnVectorPtr(
+          fast_operator_->applyToColumnVectors(*left_result, *right_result));
     }
   }
 }
 
-ColumnVector* ScalarBinaryExpression::getAllValuesForJoin(
+ColumnVectorPtr ScalarBinaryExpression::getAllValuesForJoin(
     const relation_id left_relation_id,
     ValueAccessor *left_accessor,
     const relation_id right_relation_id,
     ValueAccessor *right_accessor,
-    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
+    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+    ColumnVectorCache *cv_cache) const {
   if (fast_operator_.get() == nullptr) {
-    return ColumnVector::MakeVectorOfValue(getType(),
-                                           static_value_,
-                                           joined_tuple_ids.size());
+    return ColumnVectorPtr(
+        ColumnVector::MakeVectorOfValue(getType(),
+                                        static_value_,
+                                        joined_tuple_ids.size()));
   } else {
     if (left_operand_->hasStaticValue()) {
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
@@ -207,24 +224,27 @@ ColumnVector* ScalarBinaryExpression::getAllValuesForJoin(
         const bool using_left_relation = (right_operand_relation_id == left_relation_id);
         ValueAccessor *right_operand_accessor = using_left_relation ? left_accessor
                                                                     : right_accessor;
-        return fast_operator_->applyToStaticValueAndValueAccessorForJoin(
-            left_operand_->getStaticValue(),
-            right_operand_accessor,
-            using_left_relation,
-            right_operand_attr_id,
-            joined_tuple_ids);
+        return ColumnVectorPtr(
+            fast_operator_->applyToStaticValueAndValueAccessorForJoin(
+                left_operand_->getStaticValue(),
+                right_operand_accessor,
+                using_left_relation,
+                right_operand_attr_id,
+                joined_tuple_ids));
       }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
 
-      std::unique_ptr<ColumnVector> right_result(
+      ColumnVectorPtr right_result(
           right_operand_->getAllValuesForJoin(left_relation_id,
                                               left_accessor,
                                               right_relation_id,
                                               right_accessor,
-                                              joined_tuple_ids));
-      return fast_operator_->applyToStaticValueAndColumnVector(
-          left_operand_->getStaticValue(),
-          *right_result);
+                                              joined_tuple_ids,
+                                              cv_cache));
+      return ColumnVectorPtr(
+          fast_operator_->applyToStaticValueAndColumnVector(
+              left_operand_->getStaticValue(),
+              *right_result));
     } else if (right_operand_->hasStaticValue()) {
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
       const attribute_id left_operand_attr_id
@@ -238,24 +258,27 @@ ColumnVector* ScalarBinaryExpression::getAllValuesForJoin(
         const bool using_left_relation = (left_operand_relation_id == left_relation_id);
         ValueAccessor *left_operand_accessor = using_left_relation ? left_accessor
                                                                    : right_accessor;
-        return fast_operator_->applyToValueAccessorAndStaticValueForJoin(
-            left_operand_accessor,
-            using_left_relation,
-            left_operand_attr_id,
-            right_operand_->getStaticValue(),
-            joined_tuple_ids);
+        return ColumnVectorPtr(
+            fast_operator_->applyToValueAccessorAndStaticValueForJoin(
+                left_operand_accessor,
+                using_left_relation,
+                left_operand_attr_id,
+                right_operand_->getStaticValue(),
+                joined_tuple_ids));
       }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
 
-      std::unique_ptr<ColumnVector> left_result(
+      ColumnVectorPtr left_result(
           left_operand_->getAllValuesForJoin(left_relation_id,
                                              left_accessor,
                                              right_relation_id,
                                              right_accessor,
-                                             joined_tuple_ids));
-      return fast_operator_->applyToColumnVectorAndStaticValue(
-          *left_result,
-          right_operand_->getStaticValue());
+                                             joined_tuple_ids,
+                                             cv_cache));
+      return ColumnVectorPtr(
+          fast_operator_->applyToColumnVectorAndStaticValue(
+              *left_result,
+              right_operand_->getStaticValue()));
     } else {
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
       const attribute_id left_operand_attr_id
@@ -284,28 +307,31 @@ ColumnVector* ScalarBinaryExpression::getAllValuesForJoin(
               = (right_operand_relation_id == left_relation_id);
           ValueAccessor *right_operand_accessor = using_left_relation_for_right_operand ? left_accessor
                                                                                         : right_accessor;
-          return fast_operator_->applyToValueAccessorsForJoin(left_operand_accessor,
-                                                              using_left_relation_for_left_operand,
-                                                              left_operand_attr_id,
-                                                              right_operand_accessor,
-                                                              using_left_relation_for_right_operand,
-                                                              right_operand_attr_id,
-                                                              joined_tuple_ids);
+          return ColumnVectorPtr(
+              fast_operator_->applyToValueAccessorsForJoin(left_operand_accessor,
+                                                           using_left_relation_for_left_operand,
+                                                           left_operand_attr_id,
+                                                           right_operand_accessor,
+                                                           using_left_relation_for_right_operand,
+                                                           right_operand_attr_id,
+                                                           joined_tuple_ids));
         }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN_WITH_BINARY_EXPRESSIONS
-        std::unique_ptr<ColumnVector> right_result(
+        ColumnVectorPtr right_result(
             right_operand_->getAllValuesForJoin(left_relation_id,
                                                 left_accessor,
                                                 right_relation_id,
                                                 right_accessor,
-                                                joined_tuple_ids));
+                                                joined_tuple_ids,
+                                                cv_cache));
 
-        return fast_operator_->applyToValueAccessorAndColumnVectorForJoin(
-            left_operand_accessor,
-            using_left_relation_for_left_operand,
-            left_operand_attr_id,
-            *right_result,
-            joined_tuple_ids);
+        return ColumnVectorPtr(
+            fast_operator_->applyToValueAccessorAndColumnVectorForJoin(
+                left_operand_accessor,
+                using_left_relation_for_left_operand,
+                left_operand_attr_id,
+                *right_result,
+                joined_tuple_ids));
       } else if (right_operand_attr_id != -1) {
         const relation_id right_operand_relation_id
             = right_operand_->getRelationIdForValueAccessor();
@@ -317,34 +343,39 @@ ColumnVector* ScalarBinaryExpression::getAllValuesForJoin(
         ValueAccessor *right_operand_accessor = using_left_relation_for_right_operand ? left_accessor
                                                                                       : right_accessor;
 
-        std::unique_ptr<ColumnVector> left_result(
+        ColumnVectorPtr left_result(
             left_operand_->getAllValuesForJoin(left_relation_id,
                                                left_accessor,
                                                right_relation_id,
                                                right_accessor,
-                                               joined_tuple_ids));
-        return fast_operator_->applyToColumnVectorAndValueAccessorForJoin(
-            *left_result,
-            right_operand_accessor,
-            using_left_relation_for_right_operand,
-            right_operand_attr_id,
-            joined_tuple_ids);
+                                               joined_tuple_ids,
+                                               cv_cache));
+        return ColumnVectorPtr(
+            fast_operator_->applyToColumnVectorAndValueAccessorForJoin(
+                *left_result,
+                right_operand_accessor,
+                using_left_relation_for_right_operand,
+                right_operand_attr_id,
+                joined_tuple_ids));
       }
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
 
-      std::unique_ptr<ColumnVector> left_result(
+      ColumnVectorPtr left_result(
           left_operand_->getAllValuesForJoin(left_relation_id,
                                              left_accessor,
                                              right_relation_id,
                                              right_accessor,
-                                             joined_tuple_ids));
-      std::unique_ptr<ColumnVector> right_result(
+                                             joined_tuple_ids,
+                                             cv_cache));
+      ColumnVectorPtr right_result(
           right_operand_->getAllValuesForJoin(left_relation_id,
                                               left_accessor,
                                               right_relation_id,
                                               right_accessor,
-                                              joined_tuple_ids));
-      return fast_operator_->applyToColumnVectors(*left_result, *right_result);
+                                              joined_tuple_ids,
+                                              cv_cache));
+      return ColumnVectorPtr(
+          fast_operator_->applyToColumnVectors(*left_result, *right_result));
     }
   }
 }
@@ -374,4 +405,38 @@ void ScalarBinaryExpression::initHelper(bool own_children) {
   }
 }
 
+void ScalarBinaryExpression::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<const Expression*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const Expression*>> *container_child_fields) const {
+  Scalar::getFieldStringItems(inline_field_names,
+                              inline_field_values,
+                              non_container_child_field_names,
+                              non_container_child_fields,
+                              container_child_field_names,
+                              container_child_fields);
+
+  if (fast_operator_ == nullptr) {
+    inline_field_names->emplace_back("static_value");
+    if (static_value_.isNull()) {
+      inline_field_values->emplace_back("NULL");
+    } else {
+      inline_field_values->emplace_back(type_.printValueToString(static_value_));
+    }
+  }
+
+  inline_field_names->emplace_back("operation");
+  inline_field_values->emplace_back(
+      kBinaryOperationNames[static_cast<std::underlying_type<BinaryOperationID>::type>(
+          operation_.getBinaryOperationID())]);
+
+  non_container_child_field_names->emplace_back("left_operand");
+  non_container_child_fields->emplace_back(left_operand_.get());
+  non_container_child_field_names->emplace_back("right_operand");
+  non_container_child_fields->emplace_back(right_operand_.get());
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/ScalarBinaryExpression.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarBinaryExpression.hpp b/expressions/scalar/ScalarBinaryExpression.hpp
index c84792a..4ac1f62 100644
--- a/expressions/scalar/ScalarBinaryExpression.hpp
+++ b/expressions/scalar/ScalarBinaryExpression.hpp
@@ -21,6 +21,7 @@
 #define QUICKSTEP_EXPRESSIONS_SCALAR_SCALAR_BINARY_EXPRESSION_HPP_
 
 #include <memory>
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -29,6 +30,7 @@
 #include "expressions/scalar/Scalar.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
 #include "utility/Macros.hpp"
 
@@ -36,7 +38,7 @@
 
 namespace quickstep {
 
-class ColumnVector;
+class ColumnVectorCache;
 class ValueAccessor;
 
 struct SubBlocksReference;
@@ -97,15 +99,26 @@ class ScalarBinaryExpression : public Scalar {
     return static_value_;
   }
 
-  ColumnVector* getAllValues(ValueAccessor *accessor,
-                             const SubBlocksReference *sub_blocks_ref) const override;
+  ColumnVectorPtr getAllValues(ValueAccessor *accessor,
+                               const SubBlocksReference *sub_blocks_ref,
+                               ColumnVectorCache *cv_cache) const override;
 
-  ColumnVector* getAllValuesForJoin(
+  ColumnVectorPtr getAllValuesForJoin(
       const relation_id left_relation_id,
       ValueAccessor *left_accessor,
       const relation_id right_relation_id,
       ValueAccessor *right_accessor,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override;
+      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+      ColumnVectorCache *cv_cache) const override;
+
+ 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<const Expression*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const Expression*>> *container_child_fields) const override;
 
  private:
   void initHelper(bool own_children);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/ScalarCaseExpression.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarCaseExpression.cpp b/expressions/scalar/ScalarCaseExpression.cpp
index c81f723..00a7710 100644
--- a/expressions/scalar/ScalarCaseExpression.cpp
+++ b/expressions/scalar/ScalarCaseExpression.cpp
@@ -21,6 +21,7 @@
 
 #include <cstddef>
 #include <memory>
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -193,18 +194,21 @@ TypedValue ScalarCaseExpression::getValueForJoinedTuples(
   }
 }
 
-ColumnVector* ScalarCaseExpression::getAllValues(
+ColumnVectorPtr ScalarCaseExpression::getAllValues(
     ValueAccessor *accessor,
-    const SubBlocksReference *sub_blocks_ref) const {
+    const SubBlocksReference *sub_blocks_ref,
+    ColumnVectorCache *cv_cache) const {
   return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
       accessor,
-      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      [&](auto *accessor) -> ColumnVectorPtr {  // NOLINT(build/c++11)
     if (has_static_value_) {
-      return ColumnVector::MakeVectorOfValue(type_,
-                                             static_value_,
-                                             accessor->getNumTuples());
+      return ColumnVectorPtr(
+          ColumnVector::MakeVectorOfValue(type_,
+                                          static_value_,
+                                          accessor->getNumTuples()));
     } else if (fixed_result_expression_ != nullptr) {
-      return fixed_result_expression_->getAllValues(accessor, sub_blocks_ref);
+      return fixed_result_expression_->getAllValues(
+          accessor, sub_blocks_ref, cv_cache);
     }
 
     const TupleIdSequence *accessor_sequence = accessor->getTupleIdSequence();
@@ -238,21 +242,23 @@ ColumnVector* ScalarCaseExpression::getAllValues(
     }
 
     // Generate a ColumnVector of all the values for each case.
-    std::vector<std::unique_ptr<ColumnVector>> case_results;
+    std::vector<ColumnVectorPtr> case_results;
     for (std::vector<std::unique_ptr<TupleIdSequence>>::size_type case_idx = 0;
          case_idx < case_matches.size();
          ++case_idx) {
       std::unique_ptr<ValueAccessor> case_accessor(
           accessor->createSharedTupleIdSequenceAdapter(*case_matches[case_idx]));
       case_results.emplace_back(
-          result_expressions_[case_idx]->getAllValues(case_accessor.get(), sub_blocks_ref));
+          result_expressions_[case_idx]->getAllValues(
+              case_accessor.get(), sub_blocks_ref, cv_cache));
     }
 
-    std::unique_ptr<ColumnVector> else_results;
+    ColumnVectorPtr else_results;
     if (!else_matches->empty()) {
       std::unique_ptr<ValueAccessor> else_accessor(
           accessor->createSharedTupleIdSequenceAdapter(*else_matches));
-      else_results.reset(else_result_expression_->getAllValues(else_accessor.get(), sub_blocks_ref));
+      else_results = else_result_expression_->getAllValues(
+          else_accessor.get(), sub_blocks_ref, cv_cache);
     }
 
     // Multiplex per-case results into a single ColumnVector with values in the
@@ -262,17 +268,18 @@ ColumnVector* ScalarCaseExpression::getAllValues(
         accessor_sequence,
         case_matches,
         *else_matches,
-        &case_results,
-        else_results.get());
+        case_results,
+        else_results);
   });
 }
 
-ColumnVector* ScalarCaseExpression::getAllValuesForJoin(
+ColumnVectorPtr ScalarCaseExpression::getAllValuesForJoin(
     const relation_id left_relation_id,
     ValueAccessor *left_accessor,
     const relation_id right_relation_id,
     ValueAccessor *right_accessor,
-    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
+    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+    ColumnVectorCache *cv_cache) const {
   // Slice 'joined_tuple_ids' apart by case.
   //
   // NOTE(chasseur): We use TupleIdSequence to keep track of the positions in
@@ -321,7 +328,7 @@ ColumnVector* ScalarCaseExpression::getAllValuesForJoin(
   }
 
   // Generate a ColumnVector of all the values for each case.
-  std::vector<std::unique_ptr<ColumnVector>> case_results;
+  std::vector<ColumnVectorPtr> case_results;
   for (std::vector<std::vector<std::pair<tuple_id, tuple_id>>>::size_type case_idx = 0;
        case_idx < case_matches.size();
        ++case_idx) {
@@ -330,22 +337,24 @@ ColumnVector* ScalarCaseExpression::getAllValuesForJoin(
         left_accessor,
         right_relation_id,
         right_accessor,
-        case_matches[case_idx]));
+        case_matches[case_idx],
+        cv_cache));
   }
 
-  std::unique_ptr<ColumnVector> else_results;
+  ColumnVectorPtr else_results;
   if (!else_positions.empty()) {
     std::vector<std::pair<tuple_id, tuple_id>> else_matches;
     for (tuple_id pos : else_positions) {
       else_matches.emplace_back(joined_tuple_ids[pos]);
     }
 
-    else_results.reset(else_result_expression_->getAllValuesForJoin(
+    else_results = else_result_expression_->getAllValuesForJoin(
         left_relation_id,
         left_accessor,
         right_relation_id,
         right_accessor,
-        else_matches));
+        else_matches,
+        cv_cache);
   }
 
   // Multiplex per-case results into a single ColumnVector with values in the
@@ -355,8 +364,8 @@ ColumnVector* ScalarCaseExpression::getAllValuesForJoin(
       nullptr,
       case_positions,
       else_positions,
-      &case_results,
-      else_results.get());
+      case_results,
+      else_results);
 }
 
 void ScalarCaseExpression::MultiplexNativeColumnVector(
@@ -420,15 +429,15 @@ void ScalarCaseExpression::MultiplexNativeColumnVector(
 void ScalarCaseExpression::MultiplexIndirectColumnVector(
     const TupleIdSequence *source_sequence,
     const TupleIdSequence &case_matches,
-    IndirectColumnVector *case_result,
+    const IndirectColumnVector &case_result,
     IndirectColumnVector *output) {
   if (source_sequence == nullptr) {
     TupleIdSequence::const_iterator output_pos_it = case_matches.begin();
     for (std::size_t input_pos = 0;
-         input_pos < case_result->size();
+         input_pos < case_result.size();
          ++input_pos, ++output_pos_it) {
       output->positionalWriteTypedValue(*output_pos_it,
-                                        case_result->moveTypedValue(input_pos));
+                                        case_result.getTypedValue(input_pos));
     }
   } else {
     std::size_t input_pos = 0;
@@ -438,20 +447,20 @@ void ScalarCaseExpression::MultiplexIndirectColumnVector(
          ++output_pos, ++source_sequence_it) {
       if (case_matches.get(*source_sequence_it)) {
         output->positionalWriteTypedValue(output_pos,
-                                          case_result->moveTypedValue(input_pos++));
+                                          case_result.getTypedValue(input_pos++));
       }
     }
   }
 }
 
-ColumnVector* ScalarCaseExpression::multiplexColumnVectors(
+ColumnVectorPtr ScalarCaseExpression::multiplexColumnVectors(
     const std::size_t output_size,
     const TupleIdSequence *source_sequence,
     const std::vector<std::unique_ptr<TupleIdSequence>> &case_matches,
     const TupleIdSequence &else_matches,
-    std::vector<std::unique_ptr<ColumnVector>> *case_results,
-    ColumnVector *else_result) const {
-  DCHECK_EQ(case_matches.size(), case_results->size());
+    const std::vector<ColumnVectorPtr> &case_results,
+    const ColumnVectorPtr &else_result) const {
+  DCHECK_EQ(case_matches.size(), case_results.size());
 
   if (NativeColumnVector::UsableForType(type_)) {
     std::unique_ptr<NativeColumnVector> native_result(
@@ -461,12 +470,12 @@ ColumnVector* ScalarCaseExpression::multiplexColumnVectors(
     for (std::vector<std::unique_ptr<TupleIdSequence>>::size_type case_idx = 0;
          case_idx < case_matches.size();
          ++case_idx) {
-      DCHECK((*case_results)[case_idx]->isNative());
+      DCHECK(case_results[case_idx]->isNative());
       if (!case_matches[case_idx]->empty()) {
         MultiplexNativeColumnVector(
             source_sequence,
             *case_matches[case_idx],
-            static_cast<const NativeColumnVector&>(*(*case_results)[case_idx]),
+            static_cast<const NativeColumnVector&>(*case_results[case_idx]),
             native_result.get());
       }
     }
@@ -480,7 +489,7 @@ ColumnVector* ScalarCaseExpression::multiplexColumnVectors(
                                   native_result.get());
     }
 
-    return native_result.release();
+    return ColumnVectorPtr(native_result.release());
   } else {
     std::unique_ptr<IndirectColumnVector> indirect_result(
         new IndirectColumnVector(type_, output_size));
@@ -489,12 +498,12 @@ ColumnVector* ScalarCaseExpression::multiplexColumnVectors(
     for (std::vector<std::unique_ptr<TupleIdSequence>>::size_type case_idx = 0;
          case_idx < case_matches.size();
          ++case_idx) {
-      DCHECK(!(*case_results)[case_idx]->isNative());
+      DCHECK(!case_results[case_idx]->isNative());
       if (!case_matches[case_idx]->empty()) {
         MultiplexIndirectColumnVector(
             source_sequence,
             *case_matches[case_idx],
-            static_cast<IndirectColumnVector*>((*case_results)[case_idx].get()),
+            static_cast<const IndirectColumnVector&>(*case_results[case_idx]),
             indirect_result.get());
       }
     }
@@ -504,11 +513,52 @@ ColumnVector* ScalarCaseExpression::multiplexColumnVectors(
       DCHECK(!else_matches.empty());
       MultiplexIndirectColumnVector(source_sequence,
                                     else_matches,
-                                    static_cast<IndirectColumnVector*>(else_result),
+                                    static_cast<const IndirectColumnVector&>(*else_result),
                                     indirect_result.get());
     }
 
-    return indirect_result.release();
+    return ColumnVectorPtr(indirect_result.release());
+  }
+}
+
+void ScalarCaseExpression::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<const Expression*> *non_container_child_fields,
+    std::vector<std::string> *container_child_field_names,
+    std::vector<std::vector<const Expression*>> *container_child_fields) const {
+  Scalar::getFieldStringItems(inline_field_names,
+                              inline_field_values,
+                              non_container_child_field_names,
+                              non_container_child_fields,
+                              container_child_field_names,
+                              container_child_fields);
+
+  if (has_static_value_) {
+    inline_field_names->emplace_back("static_value");
+    if (static_value_.isNull()) {
+      inline_field_values->emplace_back("NULL");
+    } else {
+      inline_field_values->emplace_back(type_.printValueToString(static_value_));
+    }
+  }
+
+  container_child_field_names->emplace_back("when_predicates");
+  container_child_fields->emplace_back();
+  for (const auto &predicate : when_predicates_) {
+    container_child_fields->back().emplace_back(predicate.get());
+  }
+
+  container_child_field_names->emplace_back("result_expressions");
+  container_child_fields->emplace_back();
+  for (const auto &expression : result_expressions_) {
+    container_child_fields->back().emplace_back(expression.get());
+  }
+
+  if (else_result_expression_ != nullptr) {
+    non_container_child_field_names->emplace_back("else_result_expression");
+    non_container_child_fields->emplace_back(else_result_expression_.get());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8169306c/expressions/scalar/ScalarCaseExpression.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarCaseExpression.hpp b/expressions/scalar/ScalarCaseExpression.hpp
index e6809fa..3d0ed71 100644
--- a/expressions/scalar/ScalarCaseExpression.hpp
+++ b/expressions/scalar/ScalarCaseExpression.hpp
@@ -22,6 +22,7 @@
 
 #include <cstddef>
 #include <memory>
+#include <string>
 #include <utility>
 #include <vector>
 
@@ -31,15 +32,14 @@
 #include "expressions/scalar/Scalar.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
-class ColumnVector;
-class IndirectColumnVector;
-class NativeColumnVector;
+class ColumnVectorCache;
 class TupleIdSequence;
 class Type;
 class ValueAccessor;
@@ -132,15 +132,26 @@ class ScalarCaseExpression : public Scalar {
     }
   }
 
-  ColumnVector* getAllValues(ValueAccessor *accessor,
-                             const SubBlocksReference *sub_blocks_ref) const override;
+  ColumnVectorPtr getAllValues(ValueAccessor *accessor,
+                               const SubBlocksReference *sub_blocks_ref,
+                               ColumnVectorCache *cv_cache) const override;
 
-  ColumnVector* getAllValuesForJoin(
+  ColumnVectorPtr getAllValuesForJoin(
       const relation_id left_relation_id,
       ValueAccessor *left_accessor,
       const relation_id right_relation_id,
       ValueAccessor *right_accessor,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override;
+      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids,
+      ColumnVectorCache *cv_cache) const override;
+
+ 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<const Expression*> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<const Expression*>> *container_child_fields) const override;
 
  private:
   // Merge the values in the NativeColumnVector 'case_result' into '*output' at
@@ -158,7 +169,7 @@ class ScalarCaseExpression : public Scalar {
   static void MultiplexIndirectColumnVector(
       const TupleIdSequence *source_sequence,
       const TupleIdSequence &case_matches,
-      IndirectColumnVector *case_result,
+      const IndirectColumnVector &case_result,
       IndirectColumnVector *output);
 
   // Create and return a new ColumnVector by multiplexing the ColumnVectors
@@ -171,13 +182,13 @@ class ScalarCaseExpression : public Scalar {
   // the explicit WHEN clauses. Similarly, '*case_results' are the values
   // generated for the tuples matching each WHEN clause, and '*else_results'
   // are the values generated for the ELSE tuples.
-  ColumnVector* multiplexColumnVectors(
+  ColumnVectorPtr multiplexColumnVectors(
       const std::size_t output_size,
       const TupleIdSequence *source_sequence,
       const std::vector<std::unique_ptr<TupleIdSequence>> &case_matches,
       const TupleIdSequence &else_matches,
-      std::vector<std::unique_ptr<ColumnVector>> *case_results,
-      ColumnVector *else_result) const;
+      const std::vector<ColumnVectorPtr> &case_results,
+      const ColumnVectorPtr &else_result) const;
 
   std::vector<std::unique_ptr<Predicate>> when_predicates_;
   std::vector<std::unique_ptr<Scalar>> result_expressions_;



[17/32] incubator-quickstep git commit: Partition-aware CopyFrom.

Posted by ji...@apache.org.
Partition-aware CopyFrom.


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

Branch: refs/heads/new-op
Commit: 0572f40075534db25a090ee28d31da8ab7a5ee76
Parents: 678e2c3
Author: Zuyu Zhang <zu...@apache.org>
Authored: Sat Apr 29 16:32:14 2017 -0700
Committer: Zuyu Zhang <zu...@apache.org>
Committed: Tue May 2 17:16:43 2017 -0700

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp | 19 +++++++++++++------
 1 file changed, 13 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/0572f400/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 9625a91..acc2bf1 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -1011,16 +1011,23 @@ void ExecutionGenerator::convertCopyFrom(
       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);
+  if (output_relation->hasPartitionScheme()) {
+    insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::PARTITION_AWARE);
+    insert_destination_proto->MutableExtension(S::PartitionAwareInsertDestination::partition_scheme)
+        ->MergeFrom(output_relation->getPartitionScheme()->getProto());
+  } else {
+    insert_destination_proto->set_insert_destination_type(S::InsertDestinationType::BLOCK_POOL);
+
+    const vector<block_id> blocks(output_relation->getBlocksSnapshot());
+    for (const block_id block : blocks) {
+      insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
+    }
+  }
+
   insert_destination_proto->set_relation_id(output_rel_id);
   insert_destination_proto->mutable_layout()->MergeFrom(
       output_relation->getDefaultStorageBlockLayout().getDescription());
 
-  const vector<block_id> blocks(physical_plan->catalog_relation()->getBlocksSnapshot());
-  for (const block_id block : blocks) {
-    insert_destination_proto->AddExtension(S::BlockPoolInsertDestination::blocks, block);
-  }
-
   const QueryPlan::DAGNodeIndex scan_operator_index =
       execution_plan_->addRelationalOperator(
           new TextScanOperator(