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/10/05 22:03:46 UTC

[51/51] [abbrv] incubator-quickstep git commit: More updates, refactor names

More updates, refactor names


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

Branch: refs/heads/new-op
Commit: 98d794480a321be59ed26bcc6774fb47630a8c76
Parents: c87bdae
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Oct 5 17:02:33 2017 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Oct 5 17:02:33 2017 -0500

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp          |   3 +-
 query_optimizer/expressions/CMakeLists.txt      |  15 +
 query_optimizer/expressions/Cast.cpp            | 107 ++++++
 query_optimizer/expressions/Cast.hpp            | 125 +++++++
 query_optimizer/resolver/CMakeLists.txt         |   3 +
 query_optimizer/resolver/Resolver.cpp           |  10 +
 types/AsciiStringSuperType.hpp                  |   7 +-
 types/CharType.cpp                              |  64 +++-
 types/CharType.hpp                              |  13 +
 types/GenericValue.hpp                          |   4 +
 types/MetaType-decl.hpp                         |   3 +-
 types/NumericTypeSafeCoercibility.hpp           |  10 +-
 types/TypeRegistrar.hpp                         |   6 +-
 types/TypeSynthesizer.hpp                       | 136 +++++++-
 types/operations/CMakeLists.txt                 |  23 +-
 types/operations/OperationFactory.cpp           |  12 +-
 types/operations/OperationUtil.hpp              | 334 ------------------
 .../binary_operations/AddBinaryOperation.hpp    |  94 ------
 .../ArithmeticBinaryFunctors.hpp                | 182 ++++++++++
 .../ArithmeticBinaryOperations.hpp              | 182 ----------
 .../AsciiStringBinaryFunctors.hpp               | 130 +++++++
 .../AsciiStringBinaryOperations.hpp             | 130 -------
 .../BinaryOperationWrapper.hpp                  |   8 +-
 .../operations/binary_operations/CMakeLists.txt |  30 +-
 .../binary_operations/CMathBinaryFunctors.hpp   |  78 +++++
 .../binary_operations/CMathBinaryOperations.hpp |  78 -----
 .../MultiplyBinaryOperation.hpp                 | 102 ------
 .../ArithmeticUnaryFunctors.hpp                 |  80 +++++
 .../ArithmeticUnaryOperations.hpp               |  81 -----
 .../AsciiStringUnaryFunctors.hpp                | 122 +++++++
 .../AsciiStringUnaryOperations.hpp              | 122 -------
 .../operations/unary_operations/CMakeLists.txt  |  40 ++-
 .../unary_operations/CMathUnaryFunctors.hpp     | 116 +++++++
 .../unary_operations/CMathUnaryOperations.hpp   | 116 -------
 .../unary_operations/CastFunctorOverloads.hpp   | 183 ++++++++++
 .../unary_operations/CastOperation.cpp          | 314 +++++------------
 .../unary_operations/CastOperation.hpp          |  60 +---
 .../unary_operations/UnaryOperationWrapper.hpp  |   6 +-
 types/operations/utility/CMakeLists.txt         |  37 ++
 types/operations/utility/CastUtil.cpp           |   0
 types/operations/utility/CastUtil.hpp           |   0
 .../utility/OperationSynthesizeUtil.hpp         | 335 +++++++++++++++++++
 utility/meta/Common.hpp                         |  15 +-
 utility/meta/Dispatchers.hpp                    |   2 +-
 utility/meta/MetaprogrammingModule.hpp          |  24 ++
 utility/meta/TransitiveClosure.hpp              |   2 +-
 utility/meta/TypeList.hpp                       |  63 ++--
 utility/meta/TypeListMetaFunctions.hpp          |  80 ++++-
 48 files changed, 2045 insertions(+), 1642 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 372d576..d4544a0 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -1466,7 +1466,8 @@ void ExecutionGenerator::convertInsertTuple(
 
   S::Tuple *tuple_proto = query_context_proto_->add_tuples();
   for (const E::ScalarLiteralPtr &literal : physical_plan->column_values()) {
-    tuple_proto->add_attribute_values()->CopyFrom(literal->value().getProto());
+    tuple_proto->add_attribute_values()->CopyFrom(
+        literal->value().toTypedValue().getProto());
   }
 
   // FIXME(qzeng): A better way is using a traits struct to look up whether a storage

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/query_optimizer/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/CMakeLists.txt b/query_optimizer/expressions/CMakeLists.txt
index cf2727f..b51a0a8 100644
--- a/query_optimizer/expressions/CMakeLists.txt
+++ b/query_optimizer/expressions/CMakeLists.txt
@@ -20,6 +20,7 @@ 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)
@@ -102,6 +103,20 @@ target_link_libraries(quickstep_queryoptimizer_expressions_BinaryExpression
                       quickstep_types_operations_binaryoperations_BinaryOperation
                       quickstep_utility_HashPair
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_expressions_Cast
+                      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_CastOperation
+                      quickstep_utility_HashPair
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_CommonSubexpression
                       glog
                       quickstep_expressions_scalar_ScalarSharedExpression

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/query_optimizer/expressions/Cast.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Cast.cpp b/query_optimizer/expressions/Cast.cpp
new file mode 100644
index 0000000..6b2015f
--- /dev/null
+++ b/query_optimizer/expressions/Cast.cpp
@@ -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.
+ **/
+
+#include "query_optimizer/expressions/Cast.hpp"
+
+#include <cstddef>
+#include <memory>
+#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/MetaType.hpp"
+#include "types/Type.hpp"
+#include "types/operations/OperationSignature.hpp"
+#include "types/operations/OperationFactory.hpp"
+#include "types/operations/unary_operations/UnaryOperation.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 {
+  const OperationSignaturePtr op_signature =
+      OperationSignature::Create(
+          "cast", {operand_->getValueType().getTypeID(), kMetaType}, 1);
+  const UnaryOperationPtr cast_operation =
+      OperationFactory::Instance().getUnaryOperation(op_signature);
+
+  std::vector<TypedValue> meta_type_value =
+      { GenericValue::CreateWithLiteral(
+            MetaType::InstanceNonNullable(), &target_type_).toTypedValue() };
+  DCHECK(cast_operation->canApplyTo(operand_->getValueType(), meta_type_value));
+
+  return new ::quickstep::ScalarUnaryExpression(
+      op_signature, cast_operation, operand_->concretize(substitution_map),
+      std::make_shared<const std::vector<TypedValue>>(std::move(meta_type_value)));
+}
+
+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/98d79448/query_optimizer/expressions/Cast.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Cast.hpp b/query_optimizer/expressions/Cast.hpp
new file mode 100644
index 0000000..11be775
--- /dev/null
+++ b/query_optimizer/expressions/Cast.hpp
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#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/98d79448/query_optimizer/resolver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/CMakeLists.txt b/query_optimizer/resolver/CMakeLists.txt
index 8a1116a..4dd13f2 100644
--- a/query_optimizer/resolver/CMakeLists.txt
+++ b/query_optimizer/resolver/CMakeLists.txt
@@ -47,6 +47,7 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_parser_ParseBasicExpressions
                       quickstep_parser_ParseBlockProperties
                       quickstep_parser_ParseCaseExpressions
+                      quickstep_parser_ParseDataType
                       quickstep_parser_ParseExpression
                       quickstep_parser_ParseGeneratorTableReference
                       quickstep_parser_ParseGroupBy
@@ -74,6 +75,7 @@ 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
@@ -123,6 +125,7 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_types_ArrayType
                       quickstep_types_GenericValue
                       quickstep_types_IntType
+                      quickstep_types_LongType
                       quickstep_types_MetaType
                       quickstep_types_NullType
                       quickstep_types_Type

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index cf3ca6e..40b460d 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -71,6 +71,7 @@
 #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"
@@ -2551,6 +2552,15 @@ E::ScalarPtr Resolver::resolveExpression(
           expression_resolution_info,
           true /* has_single_column */);
     }
+    case ParseExpression::kTypeCast: {
+      const ParseTypeCast &parse_type_cast =
+          static_cast<const ParseTypeCast&>(parse_expression);
+      return E::Cast::Create(
+          resolveExpression(parse_type_cast.operand(),
+                            nullptr /* type_hint */,
+                            expression_resolution_info),
+          resolveDataType(parse_type_cast.target_type()));
+    }
     default:
       LOG(FATAL) << "Unknown scalar type: "
                  << parse_expression.getExpressionType();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/AsciiStringSuperType.hpp
----------------------------------------------------------------------
diff --git a/types/AsciiStringSuperType.hpp b/types/AsciiStringSuperType.hpp
index b6abcdf..7de550f 100644
--- a/types/AsciiStringSuperType.hpp
+++ b/types/AsciiStringSuperType.hpp
@@ -52,7 +52,7 @@ class AsciiStringSuperType : public TypeSynthesizer<type_id> {
    * @return The maximum length of a string of this type.
    **/
   inline std::size_t getStringLength() const {
-    return length_;
+    return TypeSynthesizer<type_id>::length_;
   }
 
  protected:
@@ -61,12 +61,9 @@ class AsciiStringSuperType : public TypeSynthesizer<type_id> {
                        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) {
+            nullable, minimum_byte_length, maximum_byte_length, string_length) {
   }
 
-  const std::size_t length_;
-
  private:
   DISALLOW_COPY_AND_ASSIGN(AsciiStringSuperType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/CharType.cpp
----------------------------------------------------------------------
diff --git a/types/CharType.cpp b/types/CharType.cpp
index eb3f64a..319ae33 100644
--- a/types/CharType.cpp
+++ b/types/CharType.cpp
@@ -19,6 +19,7 @@
 
 #include "types/CharType.hpp"
 
+#include <algorithm>
 #include <cstddef>
 #include <cstdio>
 #include <cstdlib>
@@ -33,12 +34,9 @@
 #include "types/port/strnlen.hpp"
 #include "utility/PtrMap.hpp"
 
-#include "glog/logging.h"
+#include "third_party/src/farmhash/farmhash.h"
 
-using std::pair;
-using std::size_t;
-using std::strcmp;
-using std::string;
+#include "glog/logging.h"
 
 namespace quickstep {
 
@@ -55,8 +53,8 @@ bool CharType::isSafelyCoercibleFrom(const Type &original_type) const {
   }
 }
 
-string CharType::getName() const {
-  string name("Char(");
+std::string CharType::getName() const {
+  std::string name("Char(");
   name.append(std::to_string(length_));
   name.push_back(')');
   if (nullable_) {
@@ -65,10 +63,58 @@ string CharType::getName() const {
   return name;
 }
 
+std::size_t CharType::hashValue(const UntypedLiteral *value) const {
+  const char *cstr = static_cast<const char*>(castValueToLiteral(value));
+  const std::size_t len = strnlen(cstr, length_);
+  return util::Hash(cstr, len);
+}
+
+bool CharType::checkValuesEqual(const UntypedLiteral *lhs,
+                                const UntypedLiteral *rhs,
+                                const Type &rhs_type) const {
+  return std::strncmp(static_cast<const char*>(castValueToLiteral(lhs)),
+                      static_cast<const char*>(castValueToLiteral(rhs)),
+                      length_);
+}
+
+UntypedLiteral* CharType::cloneValue(const UntypedLiteral *value) const {
+  DCHECK(value != nullptr);
+
+  const char *cstr = static_cast<const char*>(castValueToLiteral(value));
+  const std::size_t len = strnlen(cstr, length_);
+  char *value_copy = static_cast<char*>(std::malloc(length_));
+  std::memcpy(value_copy, cstr, len);
+  if (len < length_) {
+    value_copy[len] = 0;
+  }
+  return new cpptype(value_copy);
+}
+
+
+TypedValue CharType::marshallValue(const UntypedLiteral *value) const {
+  DCHECK(value != nullptr);
+
+  const char *cstr = static_cast<const char*>(castValueToLiteral(value));
+  const std::size_t len = std::min(strnlen(cstr, length_) + 1, length_);
+  return TypedValue(kChar, cstr, len).ensureNotReference();
+}
+
+UntypedLiteral* CharType::unmarshallValue(const void *data,
+                                          const std::size_t length) const {
+  const char *cstr = static_cast<const char*>(data);
+  const std::size_t len = std::min(strnlen(cstr, length), length_);
+  char *value = static_cast<char*>(std::malloc(length_));
+  std::memcpy(value, cstr, len);
+  if (len < length_) {
+    value[len] = 0;
+  }
+  return new cpptype(value);
+}
+
 std::string CharType::printValueToString(const UntypedLiteral *value) const {
   DCHECK(value != nullptr);
 
-  const char *cstr = static_cast<const char*>(castValueToLiteral(value).getOutOfLineData());
+  const char *cstr = static_cast<const char*>(castValueToLiteral(value));
   return std::string(cstr, strnlen(cstr, length_));
 }
 
@@ -84,7 +130,7 @@ void CharType::printValueToFile(const UntypedLiteral *value,
                "%*.*s",
                padding,
                static_cast<int>(length_),
-               castValueToLiteral(value).getOutOfLineData());
+               castValueToLiteral(value));
 }
 
 bool CharType::parseTypedValueFromString(const std::string &value_string,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/CharType.hpp
----------------------------------------------------------------------
diff --git a/types/CharType.hpp b/types/CharType.hpp
index 81a32ff..3355243 100644
--- a/types/CharType.hpp
+++ b/types/CharType.hpp
@@ -53,6 +53,19 @@ class CharType : public AsciiStringSuperType<kChar> {
     return length_;
   }
 
+  std::size_t hashValue(const UntypedLiteral *value) const override;
+
+  bool checkValuesEqual(const UntypedLiteral *lhs,
+                        const UntypedLiteral *rhs,
+                        const Type &rhs_type) const override;
+
+  UntypedLiteral* cloneValue(const UntypedLiteral *value) const override;
+
+  TypedValue marshallValue(const UntypedLiteral *value) const override;
+
+  UntypedLiteral* unmarshallValue(const void *data,
+                                  const std::size_t length) const override;
+
   std::string printValueToString(const UntypedLiteral *value) const override;
 
   void printValueToFile(const UntypedLiteral *value,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/GenericValue.hpp
----------------------------------------------------------------------
diff --git a/types/GenericValue.hpp b/types/GenericValue.hpp
index 3e8045b..d289a1b 100644
--- a/types/GenericValue.hpp
+++ b/types/GenericValue.hpp
@@ -110,6 +110,10 @@ class GenericValue {
     return type_;
   }
 
+  inline TypeID getTypeID() const {
+    return type_.getTypeID();
+  }
+
   inline const UntypedLiteral* getValue() const {
     return value_;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/MetaType-decl.hpp
----------------------------------------------------------------------
diff --git a/types/MetaType-decl.hpp b/types/MetaType-decl.hpp
index 6942338..80c5956 100644
--- a/types/MetaType-decl.hpp
+++ b/types/MetaType-decl.hpp
@@ -26,14 +26,13 @@
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypeSynthesizer.hpp"
+#include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
 #include "glog/logging.h"
 
 namespace quickstep {
 
-class TypedValue;
-
 /** \addtogroup Types
  *  @{
  */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/NumericTypeSafeCoercibility.hpp
----------------------------------------------------------------------
diff --git a/types/NumericTypeSafeCoercibility.hpp b/types/NumericTypeSafeCoercibility.hpp
index 914927c..b8ff876 100644
--- a/types/NumericTypeSafeCoercibility.hpp
+++ b/types/NumericTypeSafeCoercibility.hpp
@@ -21,15 +21,10 @@
 #define QUICKSTEP_TYPES_NUMERIC_TYPE_SAFE_COERCIBILITY_HPP_
 
 #include "utility/meta/TMP.hpp"
+#include "types/TypeRegistrar.hpp"
 
 namespace quickstep {
 
-class BoolType;
-class DoubleType;
-class FloatType;
-class IntType;
-class LongType;
-
 /** \addtogroup Types
  *  @{
  */
@@ -42,7 +37,8 @@ using NumericTypeSafeCoersionPartialOrder = meta::TypeList<
     IsSafelyCoercible<IntType, FloatType>,
     IsSafelyCoercible<IntType, LongType>,
     IsSafelyCoercible<FloatType, DoubleType>,
-    IsSafelyCoercible<LongType, DoubleType>>;
+    IsSafelyCoercible<LongType, DoubleType>
+>;
 
 using NumericTypeSafeCoersionClosure =
     meta::TransitiveClosure<NumericTypeSafeCoersionPartialOrder>;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/TypeRegistrar.hpp
----------------------------------------------------------------------
diff --git a/types/TypeRegistrar.hpp b/types/TypeRegistrar.hpp
index 3a25226..bb6c40d 100644
--- a/types/TypeRegistrar.hpp
+++ b/types/TypeRegistrar.hpp
@@ -48,6 +48,8 @@ using UntypedLiteral = void;
 
 using ArrayLiteral = std::vector<UntypedLiteral*>;
 using MetaTypeLiteral = const Type*;
+using ParInlinePodLiteral = const void*;
+using ParOutOfLinePodLiteral = TypedValue;
 
 template <TypeID type_id>
 struct TypeIDTrait;
@@ -83,9 +85,9 @@ REGISTER_TYPE(DatetimeIntervalType, kDatetimeInterval,
 REGISTER_TYPE(YearMonthIntervalType, kYearMonthInterval,
               SuperTypeID::kOther, kCxxInlinePod, YearMonthIntervalLit);
 REGISTER_TYPE(CharType, kChar,
-              SuperTypeID::kAsciiString, kParInlinePod, TypedValue);
+              SuperTypeID::kAsciiString, kParInlinePod, ParInlinePodLiteral);
 REGISTER_TYPE(VarCharType, kVarChar,
-              SuperTypeID::kAsciiString, kParOutOfLinePod, TypedValue);
+              SuperTypeID::kAsciiString, kParOutOfLinePod, ParOutOfLinePodLiteral);
 REGISTER_TYPE(TextType, kText,
               SuperTypeID::kOther, kCxxGeneric, std::string);
 REGISTER_TYPE(ArrayType, kArray,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/TypeSynthesizer.hpp
----------------------------------------------------------------------
diff --git a/types/TypeSynthesizer.hpp b/types/TypeSynthesizer.hpp
index 0b5842f..d69b1b4 100644
--- a/types/TypeSynthesizer.hpp
+++ b/types/TypeSynthesizer.hpp
@@ -175,20 +175,148 @@ class TypeSynthesizePolicy<
 
 
 ////////////////////////////////////////////////////////////////////////////////
-///////////////////////  ParInlinePod & ParOutOfLinePod  ///////////////////////
+/////////////////////////////////  ParInlinePod  ///////////////////////////////
 ////////////////////////////////////////////////////////////////////////////////
 template <TypeID type_id>
 class TypeSynthesizePolicy<
     type_id,
-    std::enable_if_t<TypeIDTrait<type_id>::kMemoryLayout == kParInlinePod ||
-                     TypeIDTrait<type_id>::kMemoryLayout == kParOutOfLinePod>> : public Type {
+    std::enable_if_t<TypeIDTrait<type_id>::kMemoryLayout == kParInlinePod>> : public Type {
+ private:
+  using Trait = TypeIDTrait<type_id>;
+  using TypeClass = typename Trait::TypeClass;
+  using cpptype = typename Trait::cpptype;
+
+  static_assert(std::is_same<cpptype, const void*>::value,
+                "Unexpected cpptype for ParInlinePod.");
+
+ 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);
+    }
+  }
+
+  std::size_t length() const {
+    return length_;
+  }
+
+  std::size_t getHash() const override {
+    return CombineHashes(static_cast<std::size_t>(type_id), length_);
+  }
+
+  std::size_t hashValue(const UntypedLiteral *value) const override {
+    // TODO(refactor-type): Implementation.
+    return TypedValue(type_id_,
+                      *static_cast<const cpptype*>(value),
+                      maximum_byte_length_).getHash();
+  }
+
+  bool checkValuesEqual(const UntypedLiteral *lhs,
+                        const UntypedLiteral *rhs,
+                        const Type &rhs_type) const override {
+    if (!equals(rhs_type)) {
+      return false;
+    }
+    return !std::memcmp(*static_cast<const cpptype*>(lhs),
+                        *static_cast<const cpptype*>(rhs),
+                        maximum_byte_length_);
+  }
+
+  UntypedLiteral* cloneValue(const UntypedLiteral *value) const override {
+    DCHECK(value != nullptr);
+    void *value_copy = std::malloc(maximum_byte_length_);
+    std::memcpy(value_copy,
+                *static_cast<const cpptype*>(value),
+                maximum_byte_length_);
+    return new cpptype(value_copy);
+  }
+
+  void destroyValue(UntypedLiteral *value) const override {
+    DCHECK(value != nullptr);
+    cpptype *value_ptr = static_cast<cpptype*>(value);
+    std::free(const_cast<void*>(*value_ptr));
+    delete value_ptr;
+  }
+
+  TypedValue marshallValue(const UntypedLiteral *value) const override {
+    return TypedValue(type_id_, value, maximum_byte_length_).ensureNotReference();
+  }
+
+  UntypedLiteral* unmarshallValue(const void *data,
+                                  const std::size_t length) const override {
+    DCHECK_EQ(maximum_byte_length_, length);
+    void *value = std::malloc(maximum_byte_length_);
+    std::memcpy(value, data, length);
+    return new cpptype(value);
+  }
+
+ protected:
+  TypeSynthesizePolicy(const bool nullable,
+                       const std::size_t minimum_byte_length,
+                       const std::size_t maximum_byte_length,
+                       const std::size_t length)
+      : Type(Trait::kStaticSuperTypeID, type_id, nullable,
+             minimum_byte_length, maximum_byte_length),
+        length_(length) {}
+
+  const std::size_t length_;
+
+  inline const Type& getInstance(const bool nullable) const {
+    return nullable ? InstanceNullable(length_) : InstanceNonNullable(length_);
+  }
+
+  inline void mergeIntoProto(serialization::Type *proto) const {
+    proto->set_length(length_);
+  }
+
+  inline UntypedLiteral* unmarshallTypedValueInl(const TypedValue &value) const {
+    return unmarshallValue(value.getDataPtr(), value.getDataSize());
+  }
+
+  template <typename Functor>
+  inline auto invokeOnUnmarshalledTypedValue(const TypedValue &value,
+                                             const Functor &functor) const {
+    return functor(&value);
+  }
+
+ private:
+  template <bool nullable>
+  inline static const TypeClass& InstanceInternal(const std::size_t length) {
+    static std::unordered_map<size_t, std::unique_ptr<TypeClass>> instance_map;
+    auto imit = instance_map.find(length);
+    if (imit == instance_map.end()) {
+      std::unique_ptr<TypeClass> instance(new TypeClass(nullable, length));
+      imit = instance_map.emplace(length, std::move(instance)).first;
+    }
+    return *(imit->second);
+  }
+};
+
+
+////////////////////////////////////////////////////////////////////////////////
+////////////////////////////////  ParOutOfLinePod  /////////////////////////////
+////////////////////////////////////////////////////////////////////////////////
+template <TypeID type_id>
+class TypeSynthesizePolicy<
+    type_id,
+    std::enable_if_t<TypeIDTrait<type_id>::kMemoryLayout == kParOutOfLinePod>> : public Type {
  private:
   using Trait = TypeIDTrait<type_id>;
   using TypeClass = typename Trait::TypeClass;
   using cpptype = typename Trait::cpptype;
 
   static_assert(std::is_same<cpptype, TypedValue>::value,
-                "Unexpected cpptype for paramerized PODs.");
+                "Unexpected cpptype for ParOutOfLinePod.");
 
  public:
   static const TypeClass& InstanceNonNullable(const std::size_t length) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/CMakeLists.txt b/types/operations/CMakeLists.txt
index 948d013..caf5f72 100644
--- a/types/operations/CMakeLists.txt
+++ b/types/operations/CMakeLists.txt
@@ -18,6 +18,7 @@
 add_subdirectory(binary_operations)
 add_subdirectory(comparisons)
 add_subdirectory(unary_operations)
+add_subdirectory(utility)
 
 QS_PROTOBUF_GENERATE_CPP(types_operations_Operation_proto_srcs
                          types_operations_Operation_proto_hdrs
@@ -26,7 +27,6 @@ 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})
 
@@ -42,14 +42,14 @@ target_link_libraries(quickstep_types_operations_OperationFactory
                       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_ArithmeticBinaryFunctors
+                      quickstep_types_operations_binaryoperations_AsciiStringBinaryFunctors
                       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_binaryoperations_CMathBinaryFunctors
+                      quickstep_types_operations_unaryoperations_ArithmeticUnaryFunctors
+                      quickstep_types_operations_unaryoperations_AsciiStringUnaryFunctors
+                      quickstep_types_operations_unaryoperations_CMathUnaryFunctors
                       quickstep_types_operations_unaryoperations_CastOperation
                       quickstep_types_operations_unaryoperations_DateExtractOperation
                       quickstep_types_operations_unaryoperations_SubstringOperation
@@ -58,11 +58,6 @@ target_link_libraries(quickstep_types_operations_OperationFactory
                       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
@@ -79,9 +74,9 @@ 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
-                      quickstep_types_operations_unaryoperations)
+                      quickstep_types_operations_unaryoperations
+                      quickstep_types_operations_utility)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/operations/OperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationFactory.cpp b/types/operations/OperationFactory.cpp
index 253a75d..74cc3c1 100644
--- a/types/operations/OperationFactory.cpp
+++ b/types/operations/OperationFactory.cpp
@@ -31,13 +31,13 @@
 #include "types/TypeUtil.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/ArithmeticBinaryFunctors.hpp"
+#include "types/operations/binary_operations/AsciiStringBinaryFunctors.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/binary_operations/CMathBinaryFunctors.hpp"
+#include "types/operations/unary_operations/ArithmeticUnaryFunctors.hpp"
+#include "types/operations/unary_operations/AsciiStringUnaryFunctors.hpp"
+#include "types/operations/unary_operations/CMathUnaryFunctors.hpp"
 #include "types/operations/unary_operations/CastOperation.hpp"
 #include "types/operations/unary_operations/DateExtractOperation.hpp"
 #include "types/operations/unary_operations/SubstringOperation.hpp"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/operations/OperationUtil.hpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationUtil.hpp b/types/operations/OperationUtil.hpp
deleted file mode 100644
index af23b1c..0000000
--- a/types/operations/OperationUtil.hpp
+++ /dev/null
@@ -1,334 +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_OPERATION_SYNTHESIZE_UTIL_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_OPERATION_SYNTHESIZE_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::kMemoryLayout == kCxxInlinePod>> {
-  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::kMemoryLayout == kParInlinePod>> {
-  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::kMemoryLayout == kParOutOfLinePod>> {
-  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_SYNTHESIZE_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/operations/binary_operations/AddBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/AddBinaryOperation.hpp b/types/operations/binary_operations/AddBinaryOperation.hpp
deleted file mode 100644
index 2309563..0000000
--- a/types/operations/binary_operations/AddBinaryOperation.hpp
+++ /dev/null
@@ -1,94 +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_ADD_BINARY_OPERATION_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ADD_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 addition.
- **/
-class AddBinaryOperation : 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 AddBinaryOperation& Instance() {
-    static AddBinaryOperation instance;
-    return instance;
-  }
-
-  bool isCommutative() const override {
-    return true;
-  }
-
-  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:
-  AddBinaryOperation()
-      : ArithmeticBinaryOperation(BinaryOperationID::kAdd) {
-  }
-
-  DISALLOW_COPY_AND_ASSIGN(AddBinaryOperation);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ADD_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/operations/binary_operations/ArithmeticBinaryFunctors.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryFunctors.hpp b/types/operations/binary_operations/ArithmeticBinaryFunctors.hpp
new file mode 100644
index 0000000..402bd0a
--- /dev/null
+++ b/types/operations/binary_operations/ArithmeticBinaryFunctors.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_FUNCTORS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_FUNCTORS_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 FunctorOverload,
+          typename FunctorName>
+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 FunctorName::ToString();
+  }
+  const FunctorOverload<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_FUNCTORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/operations/binary_operations/ArithmeticBinaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryOperations.hpp b/types/operations/binary_operations/ArithmeticBinaryOperations.hpp
deleted file mode 100644
index fa4d926..0000000
--- a/types/operations/binary_operations/ArithmeticBinaryOperations.hpp
+++ /dev/null
@@ -1,182 +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_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_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/operations/binary_operations/AsciiStringBinaryFunctors.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/AsciiStringBinaryFunctors.hpp b/types/operations/binary_operations/AsciiStringBinaryFunctors.hpp
new file mode 100644
index 0000000..5fd54d6
--- /dev/null
+++ b/types/operations/binary_operations/AsciiStringBinaryFunctors.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_FUNCTORS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ASCII_STRING_BINARY_FUNCTORS_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/unary_operations/UnaryOperationWrapper.hpp"
+#include "types/operations/utility/OperationSynthesizeUtil.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_FUNCTORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/98d79448/types/operations/binary_operations/AsciiStringBinaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/AsciiStringBinaryOperations.hpp b/types/operations/binary_operations/AsciiStringBinaryOperations.hpp
deleted file mode 100644
index 7181bc6..0000000
--- a/types/operations/binary_operations/AsciiStringBinaryOperations.hpp
+++ /dev/null
@@ -1,130 +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_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/98d79448/types/operations/binary_operations/BinaryOperationWrapper.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationWrapper.hpp b/types/operations/binary_operations/BinaryOperationWrapper.hpp
index f5edf52..3a336ee 100644
--- a/types/operations/binary_operations/BinaryOperationWrapper.hpp
+++ b/types/operations/binary_operations/BinaryOperationWrapper.hpp
@@ -35,8 +35,8 @@
 #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 "types/operations/utility/OperationSynthesizeUtil.hpp"
 #include "utility/Macros.hpp"
 #include "utility/meta/Common.hpp"
 
@@ -237,9 +237,9 @@ class UncheckedBinaryOperatorWrapperCodegen : public UncheckedBinaryOperator {
   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>;
+  using LeftGen = OperationCodegen<FuncSpec, LeftType>;
+  using RightGen = OperationCodegen<FuncSpec, RightType>;
+  using ResultGen = OperationCodegen<FuncSpec, ResultType>;
 
   template <bool left_nullable, bool right_nullable>
   struct Implementation;