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/10 18:25:32 UTC

[02/38] incubator-quickstep git commit: Updates to casts

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4bf0857c/types/operations/OperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationFactory.cpp b/types/operations/OperationFactory.cpp
index 74cc3c1..df536ed 100644
--- a/types/operations/OperationFactory.cpp
+++ b/types/operations/OperationFactory.cpp
@@ -33,7 +33,7 @@
 #include "types/operations/OperationSignature.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/BinaryOperationSynthesizer.hpp"
 #include "types/operations/binary_operations/CMathBinaryFunctors.hpp"
 #include "types/operations/unary_operations/ArithmeticUnaryFunctors.hpp"
 #include "types/operations/unary_operations/AsciiStringUnaryFunctors.hpp"
@@ -41,9 +41,11 @@
 #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 "types/operations/unary_operations/UnaryOperationSynthesizer.hpp"
 #include "utility/StringUtil.hpp"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 
 namespace {
@@ -52,13 +54,13 @@ 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>>() };
+    return { std::make_shared<const UnaryOperationSynthesizer<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>>() };
+    return { std::make_shared<const BinaryOperationSynthesizer<FunctorT>>() };
   }
 
   template <typename FunctorT>
@@ -94,6 +96,98 @@ OperationFactory::OperationFactory() {
   registerFunctorPack<CMathBinaryFunctorPack>();
 }
 
+bool OperationFactory::HasOperation(const std::string &operation_name,
+                                    const std::size_t arity) {
+  const auto &primary_index = Instance().primary_index_;
+  const auto indices_it =
+      primary_index.find(std::make_pair(operation_name, arity));
+  return indices_it != primary_index.end();
+}
+
+bool OperationFactory::HasOperation(const OperationSignaturePtr &op_signature) {
+  const auto &operations = Instance().operations_;
+  return operations.find(op_signature) != operations.end();
+}
+
+bool OperationFactory::CanApplyUnaryOperation(
+    const std::string &operation_name,
+    const Type &type,
+    const std::vector<GenericValue> &static_arguments) {
+  std::vector<TypeID> argument_type_ids = {type.getTypeID()};
+  std::vector<TypedValue> static_tv_arguments;
+  for (const auto &value : static_arguments) {
+    argument_type_ids.emplace_back(value.getTypeID());
+    // TODO(refactor-type): Remove this.
+    static_tv_arguments.emplace_back(value.toTypedValue());
+  }
+  const OperationSignaturePtr op_signature =
+      OperationSignature::Create(
+          operation_name, argument_type_ids, static_arguments.size());
+  if (!HasOperation(op_signature)) {
+    return false;
+  }
+  return GetUnaryOperation(op_signature)->canApplyTo(type, static_tv_arguments);
+}
+
+bool OperationFactory::CanApplyBinaryOperation(
+    const std::string &operation_name,
+    const Type &left, const Type &right,
+    const std::vector<GenericValue> &static_arguments) {
+  std::vector<TypeID> argument_type_ids = {left.getTypeID(), right.getTypeID()};
+  std::vector<TypedValue> static_tv_arguments;
+  for (const auto &value : static_arguments) {
+    argument_type_ids.emplace_back(value.getTypeID());
+    // TODO(refactor-type): Remove this.
+    static_tv_arguments.emplace_back(value.toTypedValue());
+  }
+  // TODO(refactor-type): Handle this.
+  DCHECK_EQ(0u, static_arguments.size());
+  const OperationSignaturePtr op_signature =
+      OperationSignature::Create(
+          operation_name, argument_type_ids, static_arguments.size());
+  if (!HasOperation(op_signature)) {
+    return false;
+  }
+  return GetBinaryOperation(op_signature)->canApplyTo(left, right, static_tv_arguments);
+}
+
+OperationPtr OperationFactory::GetOperation(
+    const OperationSignaturePtr &op_signature) {
+  DCHECK(HasOperation(op_signature));
+  return Instance().operations_.at(op_signature);
+}
+
+
+UnaryOperationPtr OperationFactory::GetUnaryOperation(
+    const OperationSignaturePtr &op_signature) {
+  const OperationPtr operation = GetOperation(op_signature);
+  DCHECK(operation->getOperationSuperTypeID() == Operation::kUnaryOperation);
+  return std::static_pointer_cast<const UnaryOperation>(operation);
+}
+
+BinaryOperationPtr OperationFactory::GetBinaryOperation(
+    const OperationSignaturePtr &op_signature) {
+  const OperationPtr operation = GetOperation(op_signature);
+  DCHECK(operation->getOperationSuperTypeID() == Operation::kBinaryOperation);
+  return std::static_pointer_cast<const BinaryOperation>(operation);
+}
+
+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<GenericValue>> &static_arguments,
+    std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
+    std::shared_ptr<const std::vector<GenericValue>> *coerced_static_arguments,
+    std::string *message) {
+  return Instance().resolveOperation(operation_name,
+                                     argument_types,
+                                     static_arguments,
+                                     coerced_argument_types,
+                                     coerced_static_arguments,
+                                     message);
+}
+
+
 OperationSignaturePtr OperationFactory::resolveOperation(
     const std::string &operation_name,
     const std::shared_ptr<const std::vector<const Type*>> &argument_types,
@@ -171,7 +265,7 @@ OperationFactory::ResolveStatus OperationFactory::resolveOperationWithFullTypeMa
 
   if (it != secondary_index.end() && *it->first.first == argument_type_ids) {
     const OperationSignaturePtr op_signature = it->second;
-    const OperationPtr operation = getOperation(op_signature);
+    const OperationPtr operation = operations_.at(op_signature);
 
     *coerced_static_arguments =
         std::make_shared<const std::vector<GenericValue>>(
@@ -264,7 +358,7 @@ OperationFactory::ResolveStatus OperationFactory::resolveOperationWithPartialTyp
         coerced_arg_types.emplace_back(&value.getType());
       }
 
-      const OperationPtr operation = getOperation(it->second);
+      const OperationPtr operation = operations_.at(it->second);
       if (canApplyOperationTo(operation,
                               coerced_arg_types,
                               coerced_static_args,
@@ -360,4 +454,58 @@ void OperationFactory::registerOperationInternal(const OperationPtr &operation)
 }
 
 
+// ----------------------------------------------------------------------------
+// Implemenation of utility short-cuts.
+
+bool OperationFactory::CanApplyCastOperation(const Type &source_type,
+                                             const Type &target_type) {
+  const GenericValue target_meta_type_value =
+      GenericValue::CreateWithLiteral(MetaType::InstanceNonNullable(),
+                                      &target_type);
+  return CanApplyUnaryOperation("cast", source_type, {target_meta_type_value});
+}
+
+UnaryOperationPtr OperationFactory::GetCastOperation(const TypeID source_id) {
+  const OperationSignaturePtr op_signature =
+      OperationSignature::Create("cast", {source_id, kMetaType}, 1);
+  DCHECK(HasOperation(op_signature));
+  return GetUnaryOperation(op_signature);
+}
+
+bool OperationFactory::CanApplyAddOperation(const Type &left, const Type &right) {
+  return CanApplyBinaryOperation("+", left, right);
+}
+
+BinaryOperationPtr OperationFactory::GetAddOperation(const TypeID left_id,
+                                                     const TypeID right_id) {
+  return GetBinaryOperation(OperationSignature::Create("+", {left_id, right_id}, 0));
+}
+
+bool OperationFactory::CanApplySubtractOperation(const Type &left, const Type &right) {
+  return CanApplyBinaryOperation("-", left, right);
+}
+
+BinaryOperationPtr OperationFactory::GetSubtractOperation(const TypeID left_id,
+                                                          const TypeID right_id) {
+  return GetBinaryOperation(OperationSignature::Create("-", {left_id, right_id}, 0));
+}
+
+bool OperationFactory::CanApplyMultiplyOperation(const Type &left, const Type &right) {
+  return CanApplyBinaryOperation("*", left, right);
+}
+
+BinaryOperationPtr OperationFactory::GetMultiplyOperation(const TypeID left_id,
+                                                          const TypeID right_id) {
+  return GetBinaryOperation(OperationSignature::Create("*", {left_id, right_id}, 0));
+}
+
+bool OperationFactory::CanApplyDivideOperation(const Type &left, const Type &right) {
+  return CanApplyBinaryOperation("/", left, right);
+}
+
+BinaryOperationPtr OperationFactory::GetDivideOperation(const TypeID left_id,
+                                                        const TypeID right_id) {
+  return GetBinaryOperation(OperationSignature::Create("/", {left_id, right_id}, 0));
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4bf0857c/types/operations/OperationFactory.hpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationFactory.hpp b/types/operations/OperationFactory.hpp
index b3fd9b4..7fccbc5 100644
--- a/types/operations/OperationFactory.hpp
+++ b/types/operations/OperationFactory.hpp
@@ -47,71 +47,85 @@ class Type;
 
 class OperationFactory {
  public:
-  static const OperationFactory& Instance();
+  static bool HasOperation(const std::string &operation_name,
+                           const std::size_t arity);
+
+  static bool HasOperation(const OperationSignaturePtr &op_signature);
 
-  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(
+  static bool CanApplyUnaryOperation(
       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 Type &type,
+      const std::vector<GenericValue> &static_arguments = {});
+
+  static bool CanApplyBinaryOperation(
       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));
-  }
+      const Type &left, const Type &right,
+      const std::vector<GenericValue> &static_arguments = {});
 
-  OperationSignaturePtr resolveOperation(
+  /**
+   * @brief Get the Operation for a specified operation signature.
+   */
+  static OperationPtr GetOperation(const OperationSignaturePtr &op_signature);
+
+
+  /**
+   * @brief Get the UnaryOperation for a specified operation signature.
+   */
+  static UnaryOperationPtr GetUnaryOperation(const OperationSignaturePtr &op_signature);
+
+  /**
+   * @brief Get the BinaryOperation for a specified operation signature.
+   */
+  static BinaryOperationPtr GetBinaryOperation(const OperationSignaturePtr &op_signature);
+
+  /**
+   * @brief Resolve an operation from its name and arguments.
+   */
+  static 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<GenericValue>> &static_arguments,
       std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
       std::shared_ptr<const std::vector<GenericValue>> *coerced_static_arguments,
-      std::string *message) const;
+      std::string *message);
+
+
+  // ---------------------------------------------------------------------------
+  // Utility short-cuts.
+
+
+  static bool CanApplyCastOperation(const Type &source_type, const Type &target_type);
+
+  static UnaryOperationPtr GetCastOperation(const TypeID source_id);
+
+
+  static bool CanApplyAddOperation(const Type &left, const Type &right);
+
+  static BinaryOperationPtr GetAddOperation(const TypeID left_id,
+                                            const TypeID right_id);
+
+
+  static bool CanApplySubtractOperation(const Type &left, const Type &right);
+
+  static BinaryOperationPtr GetSubtractOperation(const TypeID left_id,
+                                                 const TypeID right_id);
+
+
+  static bool CanApplyMultiplyOperation(const Type &left,  const Type &right);
+
+  static BinaryOperationPtr GetMultiplyOperation(const TypeID left_id,
+                                                 const TypeID right_id);
+
+  static bool CanApplyDivideOperation(const Type &left, const Type &right);
+
+  static BinaryOperationPtr GetDivideOperation(const TypeID left_id,
+                                               const TypeID right_id);
 
  private:
   OperationFactory();
 
+  static const OperationFactory& Instance();
+
   template <typename OperationT>
   void registerOperation();
 
@@ -151,6 +165,14 @@ class OperationFactory {
     kNotFound
   };
 
+  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<GenericValue>> &static_arguments,
+      std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
+      std::shared_ptr<const std::vector<GenericValue>> *coerced_static_arguments,
+      std::string *message) const;
+
   ResolveStatus resolveOperationWithFullTypeMatch(
       const PartialSignatureIndex &secondary_index,
       const std::vector<TypeID> &argument_type_ids,
@@ -170,16 +192,6 @@ class OperationFactory {
       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<GenericValue> &static_arguments,
-//      std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
-//      std::shared_ptr<const std::vector<GenericValue>> *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<GenericValue> &static_arguments,
@@ -196,6 +208,8 @@ class OperationFactory {
   DISALLOW_COPY_AND_ASSIGN(OperationFactory);
 };
 
+
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4bf0857c/types/operations/binary_operations/ArithmeticBinaryFunctors.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryFunctors.hpp b/types/operations/binary_operations/ArithmeticBinaryFunctors.hpp
index 402bd0a..bab71ce 100644
--- a/types/operations/binary_operations/ArithmeticBinaryFunctors.hpp
+++ b/types/operations/binary_operations/ArithmeticBinaryFunctors.hpp
@@ -36,7 +36,7 @@
 #include "types/TypedValue.hpp"
 #include "types/YearMonthIntervalType.hpp"
 #include "types/operations/binary_operations/ArithmeticBinaryFunctorOverloads.hpp"
-#include "types/operations/binary_operations/BinaryOperationWrapper.hpp"
+#include "types/operations/binary_operations/BinaryOperationSynthesizer.hpp"
 #include "utility/meta/Common.hpp"
 
 namespace quickstep {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4bf0857c/types/operations/binary_operations/AsciiStringBinaryFunctors.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/AsciiStringBinaryFunctors.hpp b/types/operations/binary_operations/AsciiStringBinaryFunctors.hpp
index 5fd54d6..4d704a1 100644
--- a/types/operations/binary_operations/AsciiStringBinaryFunctors.hpp
+++ b/types/operations/binary_operations/AsciiStringBinaryFunctors.hpp
@@ -26,11 +26,12 @@
 
 #include "types/CharType.hpp"
 #include "types/IntType.hpp"
+#include "types/TextType.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/unary_operations/UnaryOperationSynthesizer.hpp"
 #include "types/operations/utility/OperationSynthesizeUtil.hpp"
 #include "types/port/strnlen.hpp"
 
@@ -41,8 +42,8 @@ namespace quickstep {
  */
 
 template <typename LeftT, typename RightT, typename ResultT>
-struct AsciiStringConcatFunctor : public BinaryFunctor<LeftT, RightT, ResultT> {
-  explicit AsciiStringConcatFunctor(const LeftT &left, const RightT &right)
+struct RawStringConcatFunctor : public BinaryFunctor<LeftT, RightT, ResultT> {
+  explicit RawStringConcatFunctor(const LeftT &left, const RightT &right)
       : left_max_(left.getStringLength()),
         right_max_(right.getStringLength()),
         result_max_(left_max_ + right_max_) {}
@@ -114,13 +115,22 @@ struct AsciiStringConcatFunctor : public BinaryFunctor<LeftT, RightT, ResultT> {
   const std::size_t result_max_;
 };
 
+struct CxxStringConcatFunctor : public BinaryFunctor<TextType, TextType, TextType> {
+  inline std::string apply(const std::string &left, const std::string &right) const {
+    return left + right;
+  }
+  inline static std::string GetName() {
+    return "+";
+  }
+};
 
 using AsciiStringBinaryFunctorPack = FunctorPack<
 // concat
-    AsciiStringConcatFunctor<CharType, CharType, CharType>,
-    AsciiStringConcatFunctor<CharType, VarCharType, VarCharType>,
-    AsciiStringConcatFunctor<VarCharType, CharType, VarCharType>,
-    AsciiStringConcatFunctor<VarCharType, VarCharType, VarCharType>
+    RawStringConcatFunctor<CharType, CharType, CharType>,
+    RawStringConcatFunctor<CharType, VarCharType, VarCharType>,
+    RawStringConcatFunctor<VarCharType, CharType, VarCharType>,
+    RawStringConcatFunctor<VarCharType, VarCharType, VarCharType>,
+    CxxStringConcatFunctor
 >;
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4bf0857c/types/operations/binary_operations/BinaryOperationSynthesizer.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationSynthesizer.hpp b/types/operations/binary_operations/BinaryOperationSynthesizer.hpp
new file mode 100644
index 0000000..f30a3b0
--- /dev/null
+++ b/types/operations/binary_operations/BinaryOperationSynthesizer.hpp
@@ -0,0 +1,660 @@
+/**
+ * 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/binary_operations/BinaryOperation.hpp"
+#include "types/operations/utility/OperationSynthesizeUtil.hpp"
+#include "utility/Macros.hpp"
+#include "utility/meta/Common.hpp"
+
+#include "glog/logging.h"
+
+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 UncheckedBinaryOperatorSynthesizer : public UncheckedBinaryOperator {
+ public:
+  template <typename ...FunctorArgs>
+  UncheckedBinaryOperatorSynthesizer(const Type &left_type,
+                                     const Type &right_type,
+                                     const Type &result_type,
+                                     FunctorArgs &&...args)
+      : functor_(std::forward<FunctorArgs>(args)...),
+        impl_(functor_, left_type, right_type, result_type) {
+    DCHECK(left_type.getTypeID() == LeftType::kStaticTypeID);
+    DCHECK(right_type.getTypeID() == RightType::kStaticTypeID);
+    DCHECK(result_type.getTypeID() == ResultType::kStaticTypeID);
+  }
+
+  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(left.getImplementation() == LeftCVT::kImplementation);
+    using LeftAccessorT = ColumnVectorAccessor<LeftCVT>;
+    LeftAccessorT left_accessor(static_cast<const LeftCVT&>(left));
+
+    using RightCVT = typename RightGen::ColumnVectorType;
+    DCHECK(right.getImplementation() == RightCVT::kImplementation);
+    using RightAccessorT = ColumnVectorAccessor<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(left.getImplementation() == LeftCVT::kImplementation);
+
+    using LeftAccessorT = ColumnVectorAccessor<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(right.getImplementation() == RightCVT::kImplementation);
+
+    using RightAccessorT = ColumnVectorAccessor<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(left.getImplementation() == LeftCVT::kImplementation);
+    using LeftAccessorT = ColumnVectorAccessor<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(right.getImplementation() == RightCVT::kImplementation);
+    using RightAccessorT = ColumnVectorAccessor<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::kMemoryLayout == kCxxInlinePod &&
+        std::is_copy_assignable<typename LeftType::cpptype>::value;
+
+    using RightCVT = typename RightGen::ColumnVectorType;
+    DCHECK(column_vector.getImplementation() == RightCVT::kImplementation);
+    using RightAccessorT = ColumnVectorAccessor<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::kMemoryLayout == kCxxInlinePod &&
+        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 = OperationCodegen<FuncSpec, LeftType>;
+  using RightGen = OperationCodegen<FuncSpec, RightType>;
+  using ResultGen = OperationCodegen<FuncSpec, ResultType>;
+
+  template <bool left_nullable, bool right_nullable>
+  struct Implementation;
+
+  const FunctorT functor_;
+  const Implementation<false, false> impl_;
+
+  DISALLOW_COPY_AND_ASSIGN(UncheckedBinaryOperatorSynthesizer);
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <bool left_nullable, bool right_nullable>
+struct UncheckedBinaryOperatorSynthesizer<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(static_cast<const LeftType&>(left_type_in)),
+        right_type(static_cast<const RightType&>(right_type_in)),
+        result_type(static_cast<const ResultType&>(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, left_type),
+        RightGen::ToNativeValueConst(right, right_type),
+        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, right_type);
+    std::unique_ptr<typename AccessorGen::NativeType> value_cache;
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename AccessorGen::NativeTypeConstPtr arg_value =
+          AccessorGen::template GetValuePtr<accessor_nullable, AccessorT>(
+              accessor, attr_id, left_type, &value_cache);
+      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, left_type);
+    std::unique_ptr<typename AccessorGen::NativeType> value_cache;
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename AccessorGen::NativeTypeConstPtr arg_value =
+          AccessorGen::template GetValuePtr<accessor_nullable, AccessorT>(
+              accessor, attr_id, right_type, &value_cache);
+      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());
+
+    std::unique_ptr<typename LeftGen::NativeType> left_value_cache;
+    std::unique_ptr<typename RightGen::NativeType> right_value_cache;
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename LeftGen::NativeTypeConstPtr left_value =
+          LeftGen::template GetValuePtr<left_nullable, AccessorT>(
+              accessor, left_id, left_type, &left_value_cache);
+      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, right_type, &right_value_cache);
+      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);
+
+    std::unique_ptr<typename LeftGen::NativeType> left_value_cache;
+    std::unique_ptr<typename RightGen::NativeType> right_value_cache;
+    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, left_type, &left_value_cache);
+      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, right_type, &right_value_cache);
+      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>();
+
+    std::unique_ptr<typename RightGen::NativeType> value_cache;
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename RightGen::NativeTypeConstPtr right_value =
+          RightGen::template GetValuePtr<
+              right_nullable, AccessorT>(accessor, attr_id, right_type, &value_cache);
+      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 UncheckedBinaryOperatorSynthesizer"
+               << "::accumulateValueAccessor() because ResultType and LeftType "
+               << "are not same or not native types.";
+  }
+
+  const FunctorT &functor;
+  const LeftType &left_type;
+  const RightType &right_type;
+  const ResultType &result_type;
+};
+
+
+template <typename FunctorT>
+class BinaryOperationSynthesizer : public BinaryOperation {
+ public:
+  BinaryOperationSynthesizer()
+      : 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<HasGetType<FunctorT>::value>(
+        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;
+
+  QUICKSTEP_TRAIT_HAS_STATIC_METHOD(HasGetType, GetType);
+
+  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 UncheckedBinaryOperatorSynthesizer<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 UncheckedBinaryOperatorSynthesizer<FunctorT>(
+        left, right, *getResultType(left, right, static_arguments),
+        static_cast<const LeftType&>(left),
+        static_cast<const RightType&>(right));
+  }
+
+  template <bool user_defined_get_type>
+  inline const Type* getResultTypeImpl(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<!user_defined_get_type &&
+                       ResultType::kMemoryLayout == kCxxInlinePod> * = 0) const {
+    return &TypeFactory::GetType(ResultType::kStaticTypeID,
+                                 left.isNullable() || right.isNullable());
+  }
+
+  template <bool user_defined_get_type>
+  inline const Type* getResultTypeImpl(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<!user_defined_get_type &&
+                       ResultType::kMemoryLayout == kCxxGeneric> * = 0) const {
+    return &TypeFactory::GetType(ResultType::kStaticTypeID,
+                                 std::vector<GenericValue>(),
+                                 left.isNullable() || right.isNullable());
+  }
+
+  template <bool user_defined_get_type>
+  inline const Type* getResultTypeImpl(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<user_defined_get_type ||
+                       ResultType::kMemoryLayout == kParInlinePod ||
+                       ResultType::kMemoryLayout == kParOutOfLinePod> * = 0) const {
+    // TODO(refactor-type): Specialize with regard to static arguments.
+    return FunctorT::GetResultType(left, right);
+  }
+
+  const std::string operation_name_;
+
+  DISALLOW_COPY_AND_ASSIGN(BinaryOperationSynthesizer);
+};
+
+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 BinaryOperationSynthesizer<
+            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/4bf0857c/types/operations/binary_operations/BinaryOperationWrapper.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationWrapper.hpp b/types/operations/binary_operations/BinaryOperationWrapper.hpp
deleted file mode 100644
index 3a336ee..0000000
--- a/types/operations/binary_operations/BinaryOperationWrapper.hpp
+++ /dev/null
@@ -1,629 +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_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/binary_operations/BinaryOperation.hpp"
-#include "types/operations/utility/OperationSynthesizeUtil.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::kMemoryLayout == kCxxInlinePod &&
-        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::kMemoryLayout == kCxxInlinePod &&
-        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 = OperationCodegen<FuncSpec, LeftType>;
-  using RightGen = OperationCodegen<FuncSpec, RightType>;
-  using ResultGen = OperationCodegen<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::kIsParPod>(
-        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/4bf0857c/types/operations/binary_operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/CMakeLists.txt b/types/operations/binary_operations/CMakeLists.txt
index c6fd4e2..dbd6b10 100644
--- a/types/operations/binary_operations/CMakeLists.txt
+++ b/types/operations/binary_operations/CMakeLists.txt
@@ -28,9 +28,9 @@ add_library(quickstep_types_operations_binaryoperations_AsciiStringBinaryFunctor
 add_library(quickstep_types_operations_binaryoperations_BinaryOperation
             BinaryOperation.cpp
             BinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+add_library(quickstep_types_operations_binaryoperations_BinaryOperationSynthesizer
             ../../../empty_src.cpp
-            BinaryOperationWrapper.hpp)
+            BinaryOperationSynthesizer.hpp)
 add_library(quickstep_types_operations_binaryoperations_CMathBinaryFunctors
             ../../../empty_src.cpp
             CMathBinaryFunctors.hpp)
@@ -53,17 +53,18 @@ target_link_libraries(quickstep_types_operations_binaryoperations_ArithmeticBina
                       quickstep_types_TypedValue
                       quickstep_types_YearMonthIntervalType
                       quickstep_types_operations_binaryoperations_ArithmeticBinaryFunctorOverloads
-                      quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+                      quickstep_types_operations_binaryoperations_BinaryOperationSynthesizer
                       quickstep_utility_meta_Common)
 target_link_libraries(quickstep_types_operations_binaryoperations_AsciiStringBinaryFunctors
                       glog
                       quickstep_types_CharType
                       quickstep_types_IntType
+                      quickstep_types_TextType
                       quickstep_types_Type
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_VarCharType
-                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper
+                      quickstep_types_operations_unaryoperations_UnaryOperationSynthesizer
                       quickstep_types_operations_utility_OperationSynthesizeUtil
                       quickstep_types_port_strnlen)
 target_link_libraries(quickstep_types_operations_binaryoperations_BinaryOperation
@@ -73,7 +74,7 @@ target_link_libraries(quickstep_types_operations_binaryoperations_BinaryOperatio
                       quickstep_types_operations_Operation
                       quickstep_types_operations_OperationSignature
                       quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+target_link_libraries(quickstep_types_operations_binaryoperations_BinaryOperationSynthesizer
                       glog
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_ValueAccessor
@@ -93,7 +94,7 @@ target_link_libraries(quickstep_types_operations_binaryoperations_CMathBinaryFun
                       quickstep_types_FloatType
                       quickstep_types_IntType
                       quickstep_types_LongType
-                      quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+                      quickstep_types_operations_binaryoperations_BinaryOperationSynthesizer
                       quickstep_types_operations_utility_OperationSynthesizeUtil
                       quickstep_utility_meta_Common)
 
@@ -105,7 +106,7 @@ target_link_libraries(quickstep_types_operations_binaryoperations
                       quickstep_types_operations_binaryoperations_ArithmeticBinaryFunctorOverloads
                       quickstep_types_operations_binaryoperations_AsciiStringBinaryFunctors
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+                      quickstep_types_operations_binaryoperations_BinaryOperationSynthesizer
                       quickstep_types_operations_binaryoperations_CMathBinaryFunctors)
 
 # Tests:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4bf0857c/types/operations/binary_operations/CMathBinaryFunctors.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/CMathBinaryFunctors.hpp b/types/operations/binary_operations/CMathBinaryFunctors.hpp
index c6fff3b..00c8979 100644
--- a/types/operations/binary_operations/CMathBinaryFunctors.hpp
+++ b/types/operations/binary_operations/CMathBinaryFunctors.hpp
@@ -27,7 +27,7 @@
 #include "types/FloatType.hpp"
 #include "types/IntType.hpp"
 #include "types/LongType.hpp"
-#include "types/operations/binary_operations/BinaryOperationWrapper.hpp"
+#include "types/operations/binary_operations/BinaryOperationSynthesizer.hpp"
 #include "types/operations/utility/OperationSynthesizeUtil.hpp"
 #include "utility/meta/Common.hpp"
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4bf0857c/types/operations/comparisons/BasicComparison.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/BasicComparison.hpp b/types/operations/comparisons/BasicComparison.hpp
index 1d4bbdc..125942c 100644
--- a/types/operations/comparisons/BasicComparison.hpp
+++ b/types/operations/comparisons/BasicComparison.hpp
@@ -284,19 +284,18 @@ template <template <typename LeftCppType, bool left_type_nullable,
                     bool right_nullable, bool right_null_terminated, bool right_longer> class StringComparator>
 UncheckedComparator* BasicComparison::makeUncheckedComparatorForTypesHelper(const Type &left,
                                                                             const Type &right) const {
-//  if (left.getSuperTypeID() == Type::kNumeric && right.getSuperTypeID() == Type::kNumeric) {
-//    return makeNumericComparatorOuterHelper<LiteralComparator>(left, right);
-//  } else if ((left.getTypeID() == kDate && right.getTypeID() == kDate)                         ||
-//             (left.getTypeID() == kDatetime && right.getTypeID() == kDatetime)                 ||
-//             (left.getTypeID() == kDatetimeInterval && right.getTypeID() == kDatetimeInterval) ||
-//             (left.getTypeID() == kYearMonthInterval && right.getTypeID() == kYearMonthInterval)) {
-//    return makeDateComparatorOuterHelper<LiteralComparator>(left, right);
-//  } else if (left.getSuperTypeID() == Type::kAsciiString && right.getSuperTypeID() == Type::kAsciiString) {
-//    return makeStringComparatorOuterHelper<StringComparator>(left, right);
-//  } else {
+  if (left.getSuperTypeID() == SuperTypeID::kNumeric && right.getSuperTypeID() == SuperTypeID::kNumeric) {
+    return makeNumericComparatorOuterHelper<LiteralComparator>(left, right);
+  } else if ((left.getTypeID() == kDate && right.getTypeID() == kDate)                         ||
+             (left.getTypeID() == kDatetime && right.getTypeID() == kDatetime)                 ||
+             (left.getTypeID() == kDatetimeInterval && right.getTypeID() == kDatetimeInterval) ||
+             (left.getTypeID() == kYearMonthInterval && right.getTypeID() == kYearMonthInterval)) {
+    return makeDateComparatorOuterHelper<LiteralComparator>(left, right);
+  } else if (left.getSuperTypeID() == SuperTypeID::kAsciiString && right.getSuperTypeID() == SuperTypeID::kAsciiString) {
+    return makeStringComparatorOuterHelper<StringComparator>(left, right);
+  } else {
     throw OperationInapplicableToType(getName(), 2, kTypeNames[left.getTypeID()], kTypeNames[right.getTypeID()]);
-//  }
-  // TODO(refactor-type): Switch back.
+  }
 }
 
 template <template <typename LeftCppType, bool left_type_nullable,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4bf0857c/types/operations/comparisons/LiteralComparators-inl.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/LiteralComparators-inl.hpp b/types/operations/comparisons/LiteralComparators-inl.hpp
index ad54d4e..b8c4c07 100644
--- a/types/operations/comparisons/LiteralComparators-inl.hpp
+++ b/types/operations/comparisons/LiteralComparators-inl.hpp
@@ -62,8 +62,8 @@ TupleIdSequence* LiteralUncheckedComparator<ComparisonFunctor,
   static constexpr bool short_circuit = false;
 #endif
   // All literal types are usable with NativeColumnVector.
-  DCHECK(left.isNative());
-  DCHECK(right.isNative());
+  DCHECK(left.getImplementation() == ColumnVector::kNative);
+  DCHECK(right.getImplementation() == ColumnVector::kNative);
 
   const NativeColumnVector &left_native = static_cast<const NativeColumnVector&>(left);
   const NativeColumnVector &right_native = static_cast<const NativeColumnVector&>(right);
@@ -155,7 +155,7 @@ TupleIdSequence* LiteralUncheckedComparator<ComparisonFunctor,
   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());
+  DCHECK(column_vector.getImplementation() == ColumnVector::kNative);
   const NativeColumnVector &native_column_vector
       = static_cast<const NativeColumnVector&>(column_vector);
 
@@ -402,7 +402,7 @@ TupleIdSequence* LiteralUncheckedComparator<ComparisonFunctor,
         const attribute_id value_accessor_attr_id,
         const TupleIdSequence *filter,
         const TupleIdSequence *existence_bitmap) const {
-  DCHECK(column_vector.isNative());
+  DCHECK(column_vector.getImplementation() == ColumnVector::kNative);
   const NativeColumnVector &native_column_vector
       = static_cast<const NativeColumnVector&>(column_vector);
 
@@ -642,7 +642,7 @@ TypedValue LiteralUncheckedComparator<ComparisonFunctor,
   const void *current_literal = current.isNull() ? nullptr : current.getDataPtr();
 
   // All literal types are usable with NativeColumnVector.
-  DCHECK(column_vector.isNative());
+  DCHECK(column_vector.getImplementation() == ColumnVector::kNative);
 
   const NativeColumnVector &native_vector = static_cast<const NativeColumnVector&>(column_vector);
   for (std::size_t pos = 0; pos < native_vector.size(); ++pos) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4bf0857c/types/operations/unary_operations/ArithmeticUnaryFunctors.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/ArithmeticUnaryFunctors.hpp b/types/operations/unary_operations/ArithmeticUnaryFunctors.hpp
index ad45535..0eaddef 100644
--- a/types/operations/unary_operations/ArithmeticUnaryFunctors.hpp
+++ b/types/operations/unary_operations/ArithmeticUnaryFunctors.hpp
@@ -28,7 +28,7 @@
 #include "types/IntType.hpp"
 #include "types/LongType.hpp"
 #include "types/YearMonthIntervalType.hpp"
-#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
+#include "types/operations/unary_operations/UnaryOperationSynthesizer.hpp"
 #include "types/operations/utility/OperationSynthesizeUtil.hpp"
 
 namespace quickstep {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4bf0857c/types/operations/unary_operations/AsciiStringUnaryFunctors.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/AsciiStringUnaryFunctors.hpp b/types/operations/unary_operations/AsciiStringUnaryFunctors.hpp
index 34a78ab..a232dd8 100644
--- a/types/operations/unary_operations/AsciiStringUnaryFunctors.hpp
+++ b/types/operations/unary_operations/AsciiStringUnaryFunctors.hpp
@@ -30,7 +30,7 @@
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/VarCharType.hpp"
-#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
+#include "types/operations/unary_operations/UnaryOperationSynthesizer.hpp"
 #include "types/operations/utility/OperationSynthesizeUtil.hpp"
 #include "types/port/strnlen.hpp"
 #include "utility/meta/Common.hpp"