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/03/09 21:29:13 UTC

[3/8] incubator-quickstep git commit: Initial commit

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/CMakeLists.txt b/types/operations/CMakeLists.txt
index c5dad0f..6d38d80 100644
--- a/types/operations/CMakeLists.txt
+++ b/types/operations/CMakeLists.txt
@@ -25,19 +25,28 @@ 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_OperationSignature OperationSignature.cpp OperationSignature.hpp)
 add_library(quickstep_types_operations_Operation_proto ${types_operations_Operation_proto_srcs})
 
 # Link dependencies:
 target_link_libraries(quickstep_types_operations_Operation
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_types_operations_OperationSignature
+                      quickstep_types_TypeID
+                      quickstep_types_Type_proto
+                      quickstep_types_operations_Operation_proto
+                      quickstep_utility_HashPair
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_types_operations_Operation_proto
                       quickstep_types_Type_proto
+                      quickstep_types_TypedValue_proto
                       ${PROTOBUF_LIBRARY})
 
 # Module all-in-one library:
 add_library(quickstep_types_operations ../../empty_src.cpp)
 target_link_libraries(quickstep_types_operations
                       quickstep_types_operations_Operation
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_binaryoperations
                       quickstep_types_operations_comparisons

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/Operation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/Operation.hpp b/types/operations/Operation.hpp
index 51178b5..ad7c868 100644
--- a/types/operations/Operation.hpp
+++ b/types/operations/Operation.hpp
@@ -20,6 +20,9 @@
 #ifndef QUICKSTEP_TYPES_OPERATIONS_OPERATION_HPP_
 #define QUICKSTEP_TYPES_OPERATIONS_OPERATION_HPP_
 
+#include <string>
+#include <vector>
+
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -72,7 +75,7 @@ class Operation {
    * @return The human-readable name of this Operation.
    **/
   inline const char* getName() const {
-    return name_;
+    return "NoName";
   }
 
   /**
@@ -81,7 +84,7 @@ class Operation {
    * @return The short name of this Operation.
    **/
   inline const char* getShortName() const {
-    return short_name_;
+    return "NoShortName";
   }
 
   /**
@@ -98,19 +101,12 @@ class Operation {
   }
 
  protected:
-  Operation(const OperationSuperTypeID super_type_id,
-            const char *name,
-            const char *short_name)
-      : super_type_id_(super_type_id),
-        name_(name),
-        short_name_(short_name) {
+  Operation(const OperationSuperTypeID super_type_id)
+      : super_type_id_(super_type_id) {
   }
 
  private:
   const OperationSuperTypeID super_type_id_;
-  const char *name_;
-  const char *short_name_;
-
 
   DISALLOW_COPY_AND_ASSIGN(Operation);
 };

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/OperationSignature.cpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationSignature.cpp b/types/operations/OperationSignature.cpp
new file mode 100644
index 0000000..6b6c4a6
--- /dev/null
+++ b/types/operations/OperationSignature.cpp
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "types/operations/OperationSignature.hpp"
+
+#include <cstdint>
+#include <string>
+#include <type_traits>
+#include <vector>
+
+#include "types/TypeID.hpp"
+#include "types/Type.pb.h"
+#include "types/operations/Operation.pb.h"
+
+namespace quickstep {
+
+serialization::OperationSignature OperationSignature::getProto() const {
+  serialization::OperationSignature op_signature;
+
+  op_signature.set_operation_name(operation_name_);
+  for (const TypeID tid : argument_type_ids_) {
+    op_signature.add_argument_type_ids()->CopyFrom(TypeIDFactory::GetProto(tid));
+  }
+  op_signature.set_num_static_arguments(
+      static_cast<std::uint32_t>(num_static_arguments_));
+
+  return op_signature;
+}
+
+OperationSignaturePtr OperationSignature::ReconstructFromProto(
+    const serialization::OperationSignature &proto) {
+  std::vector<TypeID> argument_type_ids;
+  for (int i = 0; i < proto.argument_type_ids_size(); ++i) {
+    argument_type_ids.emplace_back(
+        TypeIDFactory::ReconstructFromProto(proto.argument_type_ids(i)));
+  }
+
+  return Create(proto.operation_name(),
+                argument_type_ids,
+                proto.num_static_arguments());
+}
+
+std::string OperationSignature::toString() const {
+  const std::size_t num_regular_arguments =
+      argument_type_ids_.size() - num_static_arguments_;
+
+  std::string str;
+  str.append(operation_name_);
+  str.push_back('(');
+  for (std::size_t i = 0; i < num_regular_arguments; ++i) {
+    if (i != 0) {
+      str.append(", ");
+    }
+    str.append(
+        kTypeNames[static_cast<std::underlying_type_t<TypeID>>(
+            argument_type_ids_[i])]);
+  }
+  if (num_static_arguments_ > 0) {
+    str.append(", static(");
+    for (std::size_t i = 0; i < num_static_arguments_; ++i) {
+      if (i != 0) {
+        str.append(", ");
+      }
+      str.append(
+          kTypeNames[static_cast<std::underlying_type_t<TypeID>>(
+              argument_type_ids_[i + num_regular_arguments])]);
+    }
+    str.push_back(')');
+  }
+  str.push_back(')');
+
+  return str;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/OperationSignature.hpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationSignature.hpp b/types/operations/OperationSignature.hpp
new file mode 100644
index 0000000..7020e50
--- /dev/null
+++ b/types/operations/OperationSignature.hpp
@@ -0,0 +1,192 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_OPERATIONS_OPERATION_SIGNATURE_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_OPERATION_SIGNATURE_HPP_
+
+#include <memory>
+#include <string>
+#include <type_traits>
+#include <vector>
+
+#include "types/TypeID.hpp"
+#include "types/operations/Operation.pb.h"
+
+#include "utility/HashPair.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+class OperationSignature;
+typedef std::shared_ptr<const OperationSignature> OperationSignaturePtr;
+
+class OperationSignature {
+ public:
+  serialization::OperationSignature getProto() const;
+
+  static OperationSignaturePtr ReconstructFromProto(
+      const serialization::OperationSignature &proto);
+
+  inline const std::string& getName() const {
+    return operation_name_;
+  }
+
+  inline std::size_t getArity() const {
+    return argument_type_ids_.size();
+  }
+
+  inline std::size_t getNonStaticArity() const {
+    return argument_type_ids_.size() - num_static_arguments_;
+  }
+
+  inline const std::vector<TypeID>& getArgumentTypeIDs() const {
+    return argument_type_ids_;
+  }
+
+  inline std::size_t getNumStaticArguments() const {
+    return num_static_arguments_;
+  }
+
+  inline bool operator==(const OperationSignature &r) const {
+    return operation_name_ == r.operation_name_
+        && argument_type_ids_ == r.argument_type_ids_
+        && num_static_arguments_ == r.num_static_arguments_;
+  }
+
+  inline bool operator<(const OperationSignature &r) const {
+    int cmp_code = operation_name_.compare(r.operation_name_);
+    if (cmp_code != 0) {
+      return cmp_code < 0;
+    }
+    cmp_code = static_cast<int>(getArity() - r.getArity());
+    if (cmp_code != 0) {
+      return cmp_code < 0;
+    }
+    cmp_code = static_cast<int>(num_static_arguments_ - r.num_static_arguments_);
+    if (cmp_code != 0) {
+      return cmp_code < 0;
+    }
+    for (std::size_t i = 0; i < getArity(); ++i) {
+      const auto l_tid =
+          static_cast<std::underlying_type_t<TypeID>>(argument_type_ids_.at(i));
+      const auto r_tid =
+          static_cast<std::underlying_type_t<TypeID>>(r.argument_type_ids_.at(i));
+      if (l_tid != r_tid) {
+        return l_tid < r_tid;
+      }
+    }
+    return false;
+  }
+
+  std::string toString() const;
+
+  static OperationSignaturePtr Create(
+      const std::string &operation_name,
+      const std::vector<TypeID> &argument_type_ids,
+      const std::size_t num_static_arguments) {
+    return OperationSignaturePtr(
+        new OperationSignature(operation_name,
+                               argument_type_ids,
+                               num_static_arguments));
+  }
+
+  static OperationSignaturePtr Create(
+      const std::string &operation_name,
+      const std::vector<TypeID> &regular_argument_type_ids,
+      const std::vector<TypeID> &static_argument_type_ids) {
+    std::vector<TypeID> argument_type_ids = regular_argument_type_ids;
+    argument_type_ids.insert(argument_type_ids.end(),
+                             static_argument_type_ids.begin(),
+                             static_argument_type_ids.end());
+    return OperationSignaturePtr(
+        new OperationSignature(operation_name,
+                               argument_type_ids,
+                               static_argument_type_ids.size()));
+  }
+
+ private:
+  OperationSignature(const std::string &operation_name,
+                     const std::vector<TypeID> &argument_type_ids,
+                     const std::size_t num_static_arguments)
+      : operation_name_(operation_name),
+        argument_type_ids_(argument_type_ids),
+        num_static_arguments_(num_static_arguments) {
+    DCHECK_GE(argument_type_ids_.size(), num_static_arguments_);
+  }
+
+  const std::string operation_name_;
+  const std::vector<TypeID> argument_type_ids_;
+  const std::size_t num_static_arguments_;
+
+  friend struct OperationSignatureHash;
+  friend struct OperationSignatureNumStaticArgumentsGreater;
+
+  DISALLOW_COPY_AND_ASSIGN(OperationSignature);
+};
+
+/**
+ * @brief Implements the equal function for operation signatures.
+ */
+struct OperationSignatureEqual {
+  inline bool operator()(const OperationSignaturePtr &lhs,
+                         const OperationSignaturePtr &rhs) const {
+    return *lhs == *rhs;
+  }
+};
+
+/**
+ * @brief Implements the hash function for operation signatures.
+ */
+struct OperationSignatureHash {
+  inline std::size_t operator()(const OperationSignaturePtr &op_sig) const {
+    std::size_t hash = std::hash<std::string>()(op_sig->operation_name_);
+    for (const TypeID tid : op_sig->argument_type_ids_) {
+      hash = CombineHashes(hash, static_cast<std::size_t>(tid));
+    }
+    hash = CombineHashes(hash, op_sig->num_static_arguments_);
+    return hash;
+  }
+};
+
+/**
+ * @brief Implements the greater function for sorting operation signature
+ *        in descending order of num_static_arguments_.
+ */
+struct OperationSignatureNumStaticArgumentsGreater {
+  inline bool operator()(const OperationSignaturePtr &lhs,
+                         const OperationSignaturePtr &rhs) const {
+    if (lhs->num_static_arguments_ != rhs->num_static_arguments_) {
+      return lhs->num_static_arguments_ > rhs->num_static_arguments_;
+    } else {
+      return *lhs < *rhs;
+    }
+  }
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_OPERATION_SIGNATURE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryOperators.hpp b/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
index 7224a0c..a9ccd61 100644
--- a/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
+++ b/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
@@ -144,8 +144,10 @@ struct DivideFunctor<float, std::int64_t> {
   }
 };
 
-template <typename LeftArgument, typename RightArgument> struct IntegerModuloFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &right) const -> decltype(left % right) {
+template <typename LeftArgument, typename RightArgument>
+struct IntegerModuloFunctor {
+  inline auto operator() (const LeftArgument &left,
+                          const RightArgument &right) const -> decltype(left % right) {
     return left % right;
   }
 };
@@ -155,9 +157,10 @@ template <typename LeftArgument, typename RightArgument> struct IntegerModuloFun
 // (2) (float, float) -> float
 // (3) (long double, long double) -> long double
 // (3) (Arithmetic, Arithmetic) -> double
-template <typename LeftArgument, typename RightArgument> struct FloatModuloFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &right) const
-      -> decltype(std::fmod(left, right)) {
+template <typename LeftArgument, typename RightArgument>
+struct FloatModuloFunctor {
+  inline auto operator() (const LeftArgument &left,
+                          const RightArgument &right) const -> decltype(std::fmod(left, right)) {
     return std::fmod(left, right);
   }
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/binary_operations/BinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperation.hpp b/types/operations/binary_operations/BinaryOperation.hpp
index 585a1c6..b3115d6 100644
--- a/types/operations/binary_operations/BinaryOperation.hpp
+++ b/types/operations/binary_operations/BinaryOperation.hpp
@@ -510,11 +510,7 @@ class BinaryOperation : public Operation {
 
  protected:
   explicit BinaryOperation(const BinaryOperationID operation_id)
-      : Operation(Operation::kBinaryOperation,
-                  kBinaryOperationNames[
-                      static_cast<typename std::underlying_type<BinaryOperationID>::type>(operation_id)],
-                  kBinaryOperationShortNames[
-                      static_cast<typename std::underlying_type<BinaryOperationID>::type>(operation_id)]),
+      : Operation(Operation::kBinaryOperation),
         operation_id_(operation_id) {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/comparisons/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/CMakeLists.txt b/types/operations/comparisons/CMakeLists.txt
index 321c0f6..dcf8908 100644
--- a/types/operations/comparisons/CMakeLists.txt
+++ b/types/operations/comparisons/CMakeLists.txt
@@ -105,11 +105,13 @@ target_link_libraries(quickstep_types_operations_comparisons_ComparisonID
 target_link_libraries(quickstep_types_operations_comparisons_ComparisonUtil
                       glog
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_types_CharType
                       quickstep_types_DatetimeLit
                       quickstep_types_IntervalLit
                       quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
+                      quickstep_types_VarCharType
                       quickstep_types_containers_Tuple
                       quickstep_types_operations_comparisons_AsciiStringComparators
                       quickstep_types_operations_comparisons_Comparison

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

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/ArithmeticUnaryOperations.cpp b/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
deleted file mode 100644
index c10d5cf..0000000
--- a/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
+++ /dev/null
@@ -1,145 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#include "types/operations/unary_operations/ArithmeticUnaryOperations.hpp"
-
-#include <string>
-
-#include "types/DatetimeIntervalType.hpp"
-#include "types/DoubleType.hpp"
-#include "types/FloatType.hpp"
-#include "types/IntType.hpp"
-#include "types/LongType.hpp"
-#include "types/Type.hpp"
-#include "types/TypeErrors.hpp"
-#include "types/TypeID.hpp"
-#include "types/TypedValue.hpp"
-#include "types/YearMonthIntervalType.hpp"
-#include "types/operations/unary_operations/ArithmeticUnaryOperators.hpp"
-#include "utility/EqualsAnyConstant.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-bool ArithmeticUnaryOperation::canApplyToType(const Type &type) const {
-  return QUICKSTEP_EQUALS_ANY_CONSTANT(
-      type.getTypeID(),
-      kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-}
-
-const Type* ArithmeticUnaryOperation::resultTypeForArgumentType(const Type &type) const {
-  if (canApplyToType(type)) {
-    return &type;
-  } else {
-    return nullptr;
-  }
-}
-
-const Type* ArithmeticUnaryOperation::pushDownTypeHint(const Type *type_hint) const {
-  if (type_hint == nullptr) {
-    return nullptr;
-  }
-
-  if (canApplyToType(*type_hint)) {
-    return type_hint;
-  } else {
-    return nullptr;
-  }
-}
-
-bool NegateUnaryOperation::resultTypeIsPlausible(const Type &result_type) const {
-  return QUICKSTEP_EQUALS_ANY_CONSTANT(
-      result_type.getTypeID(),
-      kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-}
-
-TypedValue NegateUnaryOperation::applyToChecked(const TypedValue &argument,
-                                                const Type &argument_type) const {
-  DCHECK_EQ(argument.getTypeID(), argument_type.getTypeID());
-
-  if (argument.isNull()) {
-    return argument;
-  }
-
-  switch (argument.getTypeID()) {
-    case kInt:
-      return TypedValue(-argument.getLiteral<typename IntType::cpptype>());
-    case kLong:
-      return TypedValue(-argument.getLiteral<typename LongType::cpptype>());
-    case kFloat:
-      return TypedValue(-argument.getLiteral<typename FloatType::cpptype>());
-    case kDouble:
-      return TypedValue(-argument.getLiteral<typename DoubleType::cpptype>());
-    case kDatetimeInterval:
-      return TypedValue(-argument.getLiteral<typename DatetimeIntervalType::cpptype>());
-    case kYearMonthInterval:
-      return TypedValue(-argument.getLiteral<typename YearMonthIntervalType::cpptype>());
-    default: {
-      LOG(FATAL) << "Can not apply UnaryOperation " << getName()
-                 << " to argument of type " << argument_type.getName();
-    }
-  }
-}
-
-UncheckedUnaryOperator* NegateUnaryOperation::makeUncheckedUnaryOperatorForType(const Type &type) const {
-  switch (type.getTypeID()) {
-    case kInt:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<IntType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<IntType, false>();
-      }
-    case kLong:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<LongType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<LongType, false>();
-      }
-    case kFloat:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<FloatType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<FloatType, false>();
-      }
-    case kDouble:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<DoubleType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<DoubleType, false>();
-      }
-    case kDatetimeInterval:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<DatetimeIntervalType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<DatetimeIntervalType, false>();
-      }
-    case kYearMonthInterval:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<YearMonthIntervalType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<YearMonthIntervalType, false>();
-      }
-    default:
-      throw OperationInapplicableToType(getName(), 1, type.getName().c_str());
-  }
-}
-
-}  // namespace quickstep

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/ArithmeticUnaryOperators.hpp b/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
deleted file mode 100644
index bf3f7b6..0000000
--- a/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
+++ /dev/null
@@ -1,169 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#ifndef QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATORS_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATORS_HPP_
-
-#include <cstddef>
-#include <utility>
-#include <vector>
-
-#include "catalog/CatalogTypedefs.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-#include "storage/StorageBlockInfo.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
-#include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief The UncheckedUnaryOperator for negation.
- **/
-template <class ResultType, bool argument_nullable>
-class NegateUncheckedUnaryOperator : public UncheckedUnaryOperator {
- public:
-  NegateUncheckedUnaryOperator() : UncheckedUnaryOperator() {
-  }
-
-  inline TypedValue applyToTypedValue(const TypedValue &argument) const override {
-    return applyToTypedValueInl(argument);
-  }
-
-  inline TypedValue applyToTypedValueInl(const TypedValue &argument) const {
-    if (argument_nullable && argument.isNull()) {
-      return argument;
-    }
-    return TypedValue(-argument.getLiteral<typename ResultType::cpptype>());
-  }
-
-  inline TypedValue applyToDataPtr(const void *argument) const override {
-    return applyToDataPtrInl(argument);
-  }
-
-  inline TypedValue applyToDataPtrInl(const void *argument) const {
-    if (argument_nullable && (argument == nullptr)) {
-      return TypedValue(ResultType::kStaticTypeID);
-    }
-    return TypedValue(-*static_cast<const typename ResultType::cpptype*>(argument));
-  }
-
-  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override {
-    DCHECK(NativeColumnVector::UsableForType(ResultType::Instance(argument_nullable)));
-    // All arithmetic types (numbers, datetime, and intervals) are usable with
-    // NativeColumnVector, so 'argument' should always be native.
-    DCHECK(argument.isNative());
-    const NativeColumnVector &native_argument = static_cast<const NativeColumnVector&>(argument);
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(argument_nullable),
-        native_argument.size());
-    for (std::size_t pos = 0;
-         pos < native_argument.size();
-         ++pos) {
-      const typename ResultType::cpptype *scalar_arg
-          = static_cast<const typename ResultType::cpptype*>(
-              native_argument.getUntypedValue<argument_nullable>(pos));
-      if (argument_nullable && (scalar_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-            = -(*scalar_arg);
-      }
-    }
-    return result;
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
-                                     const attribute_id argument_attr_id) const override {
-    DCHECK(NativeColumnVector::UsableForType(ResultType::Instance(argument_nullable)));
-    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        accessor,
-        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-      NativeColumnVector *result = new NativeColumnVector(
-          ResultType::Instance(argument_nullable),
-          accessor->getNumTuples());
-      accessor->beginIteration();
-      while (accessor->next()) {
-        const typename ResultType::cpptype *scalar_arg
-            = static_cast<const typename ResultType::cpptype*>(
-                accessor->template getUntypedValue<argument_nullable>(argument_attr_id));
-        if (argument_nullable && (scalar_arg == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-              = -(*scalar_arg);
-        }
-      }
-      return result;
-    });
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  ColumnVector* applyToValueAccessorForJoin(
-      ValueAccessor *accessor,
-      const bool use_left_relation,
-      const attribute_id argument_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    DCHECK(NativeColumnVector::UsableForType(ResultType::Instance(argument_nullable)));
-    NativeColumnVector *result = new NativeColumnVector(ResultType::Instance(argument_nullable),
-                                                        joined_tuple_ids.size());
-    InvokeOnValueAccessorNotAdapter(
-        accessor,
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-      for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-        const typename ResultType::cpptype *scalar_arg
-            = static_cast<const typename ResultType::cpptype*>(
-                accessor->template getUntypedValueAtAbsolutePosition<argument_nullable>(
-                    argument_attr_id,
-                    use_left_relation ? joined_pair.first : joined_pair.second));
-        if (argument_nullable && (scalar_arg == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-              = -(*scalar_arg);
-        }
-      }
-    });
-    return result;
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(NegateUncheckedUnaryOperator);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATORS_HPP_

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/CMathUnaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CMathUnaryOperations.hpp b/types/operations/unary_operations/CMathUnaryOperations.hpp
new file mode 100644
index 0000000..7d1625b
--- /dev/null
+++ b/types/operations/unary_operations/CMathUnaryOperations.hpp
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_CMATH_UNARY_OPERATIONS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_CMATH_UNARY_OPERATIONS_HPP_
+
+#include <cmath>
+#include <string>
+
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
+#include "utility/TemplateUtil.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename ArgumentT, typename ResultT,
+          typename ResultT::cpptype f(typename ArgumentT::cpptype),
+          typename FunctorNameT>
+struct CMathUnaryFunctorWrapper {
+  struct Implemenation : public UnaryFunctor<ArgumentT, ResultT> {
+    inline typename ResultT::cpptype apply(
+        const typename ArgumentT::cpptype &argument) const {
+      return f(argument);
+    }
+    inline std::string getName() const {
+      return FunctorNameT::ToString();
+    }
+  };
+
+  typedef Implemenation type;
+};
+
+template <typename ArgumentT, typename ReturnT,
+          typename ReturnT::cpptype f(typename ArgumentT::cpptype),
+          typename FunctorNameT>
+using CMathUnaryFunctor =
+    typename CMathUnaryFunctorWrapper<ArgumentT, ReturnT, f, FunctorNameT>::type;
+
+using CMathUnaryFunctorPack = UnaryFunctorPack<
+// abs
+    CMathUnaryFunctor<IntType, IntType,
+                      std::abs, StringLiteral<'a','b','s'>>,
+    CMathUnaryFunctor<LongType, LongType,
+                      std::abs, StringLiteral<'a','b','s'>>,
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::fabs, StringLiteral<'a','b','s'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::fabs, StringLiteral<'a','b','s'>>,
+// sqrt
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::sqrt, StringLiteral<'s','q','r','t'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::sqrt, StringLiteral<'s','q','r','t'>>,
+// exp
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::exp, StringLiteral<'e','x','p'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::exp, StringLiteral<'e','x','p'>>,
+// log
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::log, StringLiteral<'l','o','g'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::log, StringLiteral<'l','o','g'>>,
+// ceil
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::ceil, StringLiteral<'c','e','i','l'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::ceil, StringLiteral<'c','e','i','l'>>,
+// floor
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::floor, StringLiteral<'f','l','o','o','r'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::floor, StringLiteral<'f','l','o','o','r'>>,
+// round
+    CMathUnaryFunctor<FloatType, LongType,
+                      std::llround, StringLiteral<'r','o','u','n','d'>>,
+    CMathUnaryFunctor<DoubleType, LongType,
+                      std::llround, StringLiteral<'r','o','u','n','d'>>
+>;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_CMATH_UNARY_OPERATIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/CastOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CastOperation.cpp b/types/operations/unary_operations/CastOperation.cpp
new file mode 100644
index 0000000..ef5b8f8
--- /dev/null
+++ b/types/operations/unary_operations/CastOperation.cpp
@@ -0,0 +1,282 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "types/operations/unary_operations/CastOperation.hpp"
+
+#include <algorithm>
+#include <map>
+#include <cstdint>
+#include <cstdlib>
+#include <string>
+#include <vector>
+
+#include "types/CharType.hpp"
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeUtil.hpp"
+#include "types/TypedValue.hpp"
+#include "types/VarCharType.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
+#include "types/port/strnlen.hpp"
+#include "utility/EqualsAnyConstant.hpp"
+
+namespace quickstep {
+
+namespace {
+
+template <typename ArgumentT, typename ResultT>
+struct NumericCastToNumericFunctor
+    : public UnaryFunctor<ArgumentT, ResultT> {
+  inline typename ResultT::cpptype apply(
+      const typename ArgumentT::cpptype &argument) const {
+    return static_cast<typename ResultT::cpptype>(argument);
+  }
+};
+
+template <typename ArgumentT, typename ResultT>
+class NumericCastToAsciiStringFunctor : public UnaryFunctor<ArgumentT, ResultT> {
+ public:
+  explicit NumericCastToAsciiStringFunctor(const std::size_t max_length)
+      : max_length_(max_length) {}
+
+  inline void apply(const typename ArgumentT::cpptype &argument, void *result) const {
+    std::string str = std::to_string(argument);
+    const std::size_t str_len = str.length();
+
+    if (str_len < max_length_) {
+      std::memcpy(result, str.c_str(), str_len);
+      static_cast<char *>(result)[str_len] = 0;
+    } else {
+      std::memcpy(result, str.c_str(), max_length_);
+    }
+  }
+
+  inline TypedValue apply(const typename ArgumentT::cpptype &argument) const {
+    std::string str = std::to_string(argument);
+    const std::size_t len = std::min(str.length(), max_length_);
+    const std::size_t buf_len = len + 1;
+
+    char *buf = static_cast<char *>(std::malloc(buf_len));
+    std::memcpy(buf, str.c_str(), len);
+    buf[len] = 0;
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, buf_len);
+  }
+
+ private:
+  const std::size_t max_length_;
+};
+
+
+template <typename ResultCppType>
+ResultCppType CastStringToNumericImpl(const char *str);
+
+template <>
+int CastStringToNumericImpl(const char *str) {
+  return std::atoi(str);
+}
+template <>
+float CastStringToNumericImpl(const char *str) {
+  return static_cast<float>(std::atof(str));
+}
+template <>
+std::int64_t CastStringToNumericImpl(const char *str) {
+  return std::atoll(str);
+}
+template <>
+double CastStringToNumericImpl(const char *str) {
+  return std::atof(str);
+}
+
+template <typename ArgumentT, typename ResultT,
+          typename ResultT::cpptype f(const char*)>
+struct AsciiStringCastToNumericFunctor
+    : public UnaryFunctor<ArgumentT, ResultT> {
+  explicit AsciiStringCastToNumericFunctor(const std::size_t max_length)
+      : max_length_(max_length) {}
+
+  inline typename ResultT::cpptype apply(const TypedValue &argument) const {
+    return f(static_cast<const char*>(argument.getDataPtr()));
+  }
+
+  inline typename ResultT::cpptype apply(const void *argument) const {
+    const char *str = static_cast<const char*>(argument);
+    const std::string value(str, strnlen(str, max_length_));
+    return f(value.c_str());
+  }
+
+ private:
+  const std::size_t max_length_;
+};
+
+template <typename ArgumentT, typename ResultT>
+struct AsciiStringCastToAsciiStringFunctor
+    : public UnaryFunctor<ArgumentT, ResultT> {
+  explicit AsciiStringCastToAsciiStringFunctor(const std::size_t max_string_length)
+      : max_string_length_(max_string_length) {}
+
+  inline void apply(const void *argument, void *result) const {
+    std::memcpy(result, argument, max_string_length_);
+  }
+
+  inline void apply(const TypedValue &argument, void *result) const {
+    std::memcpy(result,
+                argument.getOutOfLineData(),
+                std::min(argument.getDataSize(), max_string_length_));
+  }
+
+  inline TypedValue apply(const void *argument) const {
+    const std::size_t len =
+        strnlen(static_cast<const char*>(argument), max_string_length_);
+
+    char *buf = static_cast<char *>(std::malloc(len+1));
+    std::memcpy(buf, argument, len);
+    buf[len] = 0;
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, len+1);
+  }
+
+  inline TypedValue apply(const TypedValue &argument) const {
+    const std::size_t len =
+        std::min(argument.getDataSize() - 1, max_string_length_);
+
+    char *buf = static_cast<char *>(std::malloc(len+1));
+    std::memcpy(buf, argument.getDataPtr(), len);
+    buf[len] = 0;
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, len+1);
+  }
+
+ private:
+  const std::size_t max_string_length_;
+};
+
+}  // namespace
+
+const re2::RE2 CastOperation::kTypePattern("([a-z]+)(\\(([0-9]+)\\))?");
+
+const std::map<std::string, TypeID> CastOperation::kNameToTypeIDMap = {
+    { "int",     kInt },
+    { "long",    kLong },
+    { "float",   kFloat },
+    { "double",  kDouble },
+    { "char",    kChar },
+    { "varchar", kVarChar }
+};
+
+UncheckedUnaryOperator* CastOperation::makeUncheckedUnaryOperator(
+    const Type &type,
+    const std::vector<TypedValue> &static_arguments) const {
+  const Type *result_type = getResultType(type, static_arguments);
+  DCHECK(result_type != nullptr);
+
+  const TypeID argument_type_id = type.getTypeID();
+  const TypeID result_type_id = result_type->getTypeID();
+
+  if (QUICKSTEP_EQUALS_ANY_CONSTANT(argument_type_id, kInt, kLong, kFloat, kDouble)) {
+    return InvokeOnTypeID<TypeIDSelectorEqualsAny<kInt, kLong, kFloat, kDouble>>(
+        argument_type_id,
+        [&](auto arg_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+      using ArgumentT = typename TypeGenerator<decltype(arg_tid)::value>::type;
+
+      switch (result_type_id) {
+        case kInt:  // Fall through
+        case kLong:
+        case kFloat:
+        case kDouble: {
+          return InvokeOnTypeID<TypeIDSelectorEqualsAny<kInt, kLong, kFloat, kDouble>>(
+              result_type_id,
+              [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+            using ResultT = typename TypeGenerator<decltype(result_tid)::value>::type;
+
+            return new UncheckedUnaryOperatorWrapperCodegen<
+                NumericCastToNumericFunctor<ArgumentT, ResultT>>(type, *result_type);
+          });
+        }
+        case kChar:  // Fall through
+        case kVarChar: {
+          return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
+              result_type_id,
+              [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+            using ResultT = typename TypeGenerator<decltype(result_tid)::value>::type;
+
+            return new UncheckedUnaryOperatorWrapperCodegen<
+                 NumericCastToAsciiStringFunctor<ArgumentT, ResultT>>(
+                     type, *result_type,
+                     static_cast<const ResultT*>(result_type)->getStringLength());
+          });
+        }
+        default:
+          LOG(FATAL) << "Unexpected result type " << result_type->getName()
+                     << " in CastOperation::makeUncheckedUnaryOperator "
+                     << "for argument type " << type.getName();
+      }
+    });
+  } else if (QUICKSTEP_EQUALS_ANY_CONSTANT(argument_type_id, kChar, kVarChar)) {
+    return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
+        argument_type_id,
+        [&](auto arg_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+      using ArgumentT = typename TypeGenerator<decltype(arg_tid)::value>::type;
+
+      switch (result_type_id) {
+        case kInt:  // Fall through
+        case kLong:
+        case kFloat:
+        case kDouble: {
+          return InvokeOnTypeID<TypeIDSelectorEqualsAny<kInt, kLong, kFloat, kDouble>>(
+              result_type_id,
+              [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+            using ResultT = typename TypeGenerator<decltype(result_tid)::value>::type;
+
+            return new UncheckedUnaryOperatorWrapperCodegen<
+                AsciiStringCastToNumericFunctor<
+                    ArgumentT, ResultT,
+                    CastStringToNumericImpl<typename ResultT::cpptype>>>(
+                        type, *result_type,
+                        static_cast<const ArgumentT&>(type).getStringLength());
+          });
+        }
+        case kChar:  // Fall through
+        case kVarChar: {
+          return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
+              result_type_id,
+              [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+            using ResultT = typename TypeGenerator<decltype(result_tid)::value>::type;
+
+            return new UncheckedUnaryOperatorWrapperCodegen<
+                 AsciiStringCastToAsciiStringFunctor<ArgumentT, ResultT>>(
+                     type, *result_type,
+                     std::min(static_cast<const ArgumentT&>(type).getStringLength(),
+                              static_cast<const ResultT*>(result_type)->getStringLength()));
+          });
+        }
+        default:
+          LOG(FATAL) << "Unexpected result type " << result_type->getName()
+                     << " in CastOperation::makeUncheckedUnaryOperator "
+                     << "for argument type " << type.getName();
+      }
+    });
+  }
+
+  LOG(FATAL) << "Unexpected argument type in "
+             << "CastOperation::makeUncheckedUnaryOperator: "
+             << result_type->getName();
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/CastOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CastOperation.hpp b/types/operations/unary_operations/CastOperation.hpp
new file mode 100644
index 0000000..23dbec2
--- /dev/null
+++ b/types/operations/unary_operations/CastOperation.hpp
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_
+
+#include <cstddef>
+#include <map>
+#include <string>
+#include <utility>
+
+#include "types/IntType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/operations/unary_operations/UnaryOperation.hpp"
+#include "utility/Macros.hpp"
+#include "utility/StringUtil.hpp"
+
+#include "glog/logging.h"
+
+#include "re2/stringpiece.h"
+#include "re2/re2.h"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+/**
+ * @brief UnaryOperation for CAST.
+ */
+class CastOperation : public UnaryOperation {
+ public:
+  CastOperation() {}
+
+  std::string getName() const override {
+    return "Cast";
+  }
+
+  std::string getShortName() const override {
+    return "Cast";
+  }
+
+  std::vector<OperationSignaturePtr> getSignatures() const override {
+    const std::vector<TypeID> source_type_ids =
+        { kInt, kLong, kFloat, kDouble, kChar, kVarChar };
+    const std::vector<TypeID> target_type_carrier = { kVarChar };
+
+    std::vector<OperationSignaturePtr> signatures;
+    for (const TypeID source_type_id : source_type_ids) {
+      signatures.emplace_back(
+          OperationSignature::Create(getName(), {source_type_id}, target_type_carrier));
+    }
+    return signatures;
+  }
+
+  bool canApplyTo(const Type &type,
+                  const std::vector<TypedValue> &static_arguments,
+                  std::string *message) const override {
+    DCHECK_EQ(1u, static_arguments.size());
+    if (parseType(static_arguments.front()) == nullptr) {
+      *message = "Invalid target type for CAST";
+      return false;
+    }
+    return true;
+  }
+
+  const Type* getResultType(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK_EQ(1u, static_arguments.size());
+    const Type *target_type = parseType(static_arguments.front());
+
+    DCHECK(target_type != nullptr);
+    if (type.isNullable()) {
+      return &target_type->getNullableVersion();
+    } else {
+      return target_type;
+    }
+  }
+
+  UncheckedUnaryOperator* makeUncheckedUnaryOperator(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override;
+
+ private:
+  static const Type* parseType(const TypedValue &type_arg) {
+    DCHECK(type_arg.getTypeID() == kVarChar);
+    const std::string type_str =
+        ToLower(std::string(static_cast<const char*>(type_arg.getOutOfLineData())));
+
+    const re2::StringPiece type_piece(type_str);
+    std::string type_name;
+    std::string length_str;
+    if (!re2::RE2::FullMatch(type_piece,
+                             kTypePattern,
+                             &type_name,
+                             static_cast<void *>(nullptr),
+                             &length_str)) {
+      return nullptr;
+    }
+
+    auto it = kNameToTypeIDMap.find(type_name);
+    if (it == kNameToTypeIDMap.end()) {
+      return nullptr;
+    }
+
+    if (length_str.empty()) {
+      return &TypeFactory::GetType(it->second);
+    } else {
+      TypedValue length_value;
+      if (IntType::InstanceNonNullable().parseValueFromString(length_str, &length_value)) {
+        return &TypeFactory::GetType(
+            it->second, static_cast<std::size_t>(length_value.getLiteral<int>()));
+      }
+    }
+    return nullptr;
+  }
+
+  static const re2::RE2 kTypePattern;
+  static const std::map<std::string, TypeID> kNameToTypeIDMap;
+
+  DISALLOW_COPY_AND_ASSIGN(CastOperation);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_