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/29 00:08:38 UTC

[34/40] incubator-quickstep git commit: Initial commit

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/959cd0d4/types/TypeUtil.hpp
----------------------------------------------------------------------
diff --git a/types/TypeUtil.hpp b/types/TypeUtil.hpp
new file mode 100644
index 0000000..6c58ef1
--- /dev/null
+++ b/types/TypeUtil.hpp
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_TYPE_UTIL_HPP_
+#define QUICKSTEP_TYPES_TYPE_UTIL_HPP_
+
+#include <type_traits>
+
+#include "types/CharType.hpp"
+#include "types/DateType.hpp"
+#include "types/DatetimeIntervalType.hpp"
+#include "types/DatetimeType.hpp"
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/NullType.hpp"
+#include "types/Type.hpp"
+#include "types/Type.pb.h"
+#include "types/TypeID.hpp"
+#include "types/VarCharType.hpp"
+#include "types/YearMonthIntervalType.hpp"
+#include "utility/Macros.hpp"
+#include "utility/TemplateUtil.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+template <TypeID type_id>
+struct TypeGenerator {};
+
+#define REGISTER_TYPE(T) \
+  template <> struct TypeGenerator<T::kStaticTypeID> { typedef T type; };
+
+REGISTER_TYPE(IntType);
+REGISTER_TYPE(LongType);
+REGISTER_TYPE(FloatType);
+REGISTER_TYPE(DoubleType);
+REGISTER_TYPE(DateType);
+REGISTER_TYPE(DatetimeType);
+REGISTER_TYPE(DatetimeIntervalType);
+REGISTER_TYPE(YearMonthIntervalType);
+REGISTER_TYPE(CharType);
+REGISTER_TYPE(VarCharType);
+REGISTER_TYPE(NullType);
+
+#undef REGISTER_TYPE
+
+namespace type_util_internal {
+
+template <bool require_parameterized>
+struct TypeIDSelectorParameterizedHelper {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+    inline static auto Invoke(const FunctorT &functor)
+        -> decltype(functor(TypeIDConstant())) {
+      DLOG(FATAL) << "Unexpected TypeID: "
+                  << kTypeNames[static_cast<int>(TypeIDConstant::value)];
+    }
+  };
+};
+
+template <bool require_non_parameterized>
+template <typename TypeIDConstant, typename FunctorT>
+struct TypeIDSelectorParameterizedHelper<require_non_parameterized>::Implementation<
+    TypeIDConstant, FunctorT,
+    std::enable_if_t<TypeGenerator<TypeIDConstant::value>::type::kParameterized
+                         ^ require_non_parameterized>> {
+  inline static auto Invoke(const FunctorT &functor) {
+    return functor(TypeIDConstant());
+  }
+};
+
+}  // namespace type_util_internal
+
+struct TypeIDSelectorAll {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+    inline static auto Invoke(const FunctorT &functor) {
+      return functor(TypeIDConstant());
+    }
+  };
+};
+
+using TypeIDSelectorNonParameterized =
+    type_util_internal::TypeIDSelectorParameterizedHelper<true>;
+
+using TypeIDSelectorParameterized =
+    type_util_internal::TypeIDSelectorParameterizedHelper<false>;
+
+template <TypeID ...candidates>
+struct TypeIDSelectorEqualsAny {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+    inline static auto Invoke(const FunctorT &functor)
+        -> decltype(functor(TypeIDConstant())) {
+      DLOG(FATAL) << "Unexpected TypeID: "
+                  << kTypeNames[static_cast<int>(TypeIDConstant::value)];
+    }
+  };
+};
+
+template <TypeID ...candidates>
+template <typename TypeIDConstant, typename FunctorT>
+struct TypeIDSelectorEqualsAny<candidates...>::Implementation<
+    TypeIDConstant, FunctorT,
+    std::enable_if_t<
+        EqualsAny<TypeIDConstant,
+                  std::integral_constant<TypeID, candidates>...>::value>> {
+  inline static auto Invoke(const FunctorT &functor) {
+      return functor(TypeIDConstant());
+  }
+};
+
+template <typename Selector = TypeIDSelectorAll, typename FunctorT>
+inline auto InvokeOnTypeID(const TypeID type_id,
+                           const FunctorT &functor) {
+#define REGISTER_TYPE_ID(type_id) \
+  case type_id: \
+    return Selector::template Implementation< \
+        std::integral_constant<TypeID, type_id>, FunctorT>::Invoke(functor)
+
+  switch (type_id) {
+    REGISTER_TYPE_ID(kInt);
+    REGISTER_TYPE_ID(kLong);
+    REGISTER_TYPE_ID(kFloat);
+    REGISTER_TYPE_ID(kDouble);
+    REGISTER_TYPE_ID(kDate);
+    REGISTER_TYPE_ID(kDatetime);
+    REGISTER_TYPE_ID(kDatetimeInterval);
+    REGISTER_TYPE_ID(kYearMonthInterval);
+    REGISTER_TYPE_ID(kChar);
+    REGISTER_TYPE_ID(kVarChar);
+    REGISTER_TYPE_ID(kNullType);
+    default:
+      FATAL_ERROR("Unrecognized TypeID in InvokeOnTypeID()");
+  }
+
+#undef REGISTER_TYPE_ID
+}
+
+class TypeUtil {
+ public:
+  static bool IsParameterized(const TypeID type_id) {
+    return InvokeOnTypeID(
+        type_id,
+        [&](auto tid) -> bool {  // NOLINT(build/c++11)
+      return TypeGenerator<decltype(tid)::value>::type::kParameterized;
+    });
+  }
+
+ private:
+  TypeUtil() {}
+
+  DISALLOW_COPY_AND_ASSIGN(TypeUtil);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_TYPE_UTIL_HPP_

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/959cd0d4/types/TypedValue.hpp
----------------------------------------------------------------------
diff --git a/types/TypedValue.hpp b/types/TypedValue.hpp
index 0ba3d53..18ed4af 100644
--- a/types/TypedValue.hpp
+++ b/types/TypedValue.hpp
@@ -264,9 +264,9 @@ class TypedValue {
    *        TypedValue will take ownership of this memory.
    * @param value_size The number of bytes of data at value_ptr.
    **/
-  static TypedValue CreateWithOwnedData(const TypeID type_id,
-                                        void *value_ptr,
-                                        const std::size_t value_size) {
+  inline static TypedValue CreateWithOwnedData(const TypeID type_id,
+                                               void *value_ptr,
+                                               const std::size_t value_size) {
     TypedValue val(type_id, value_ptr, value_size);
     val.value_info_ |= kOwnershipMask;
     return val;
@@ -467,7 +467,8 @@ class TypedValue {
    * @return The underlying literal value represented by this TypedValue.
    **/
   template <typename LiteralType>
-  LiteralType getLiteral() const;
+  LiteralType getLiteral() const {
+  }
 
   /**
    * @brief Get the out-of-line data which this TypedValue points to.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/959cd0d4/types/TypedValue.proto
----------------------------------------------------------------------
diff --git a/types/TypedValue.proto b/types/TypedValue.proto
index 7f3ab7a..db44dec 100644
--- a/types/TypedValue.proto
+++ b/types/TypedValue.proto
@@ -22,7 +22,7 @@ package quickstep.serialization;
 import "types/Type.proto";
 
 message TypedValue {
-  required Type.TypeID type_id = 1;
+  required TypeID type_id = 1;
 
   // NOTE(zuyu): For a NULL value, none of the optional fields are filled in.
   optional int32 int_value = 2;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/959cd0d4/types/VarCharType.cpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.cpp b/types/VarCharType.cpp
index 02845b1..05162cf 100644
--- a/types/VarCharType.cpp
+++ b/types/VarCharType.cpp
@@ -66,8 +66,7 @@ const VarCharType& VarCharType::InstanceFromProto(const serialization::Type &pro
 
 serialization::Type VarCharType::getProto() const {
   serialization::Type proto;
-  proto.set_type_id(serialization::Type::VAR_CHAR);
-
+  proto.mutable_type_id()->CopyFrom(TypeIDFactory::GetProto(type_id_));
   proto.set_nullable(nullable_);
 
   proto.SetExtension(serialization::VarCharType::length, length_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/959cd0d4/types/VarCharType.hpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.hpp b/types/VarCharType.hpp
index bb50e92..af3ab64 100644
--- a/types/VarCharType.hpp
+++ b/types/VarCharType.hpp
@@ -43,7 +43,7 @@ namespace quickstep {
  *       character. This means that the VARCHAR(X) type requires from 1 to X+1
  *       bytes of storage, depending on string length.
  **/
-class VarCharType : public AsciiStringSuperType {
+class VarCharType : public AsciiStringSuperType<kVarChar, kOutOfLine> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this Type
@@ -137,7 +137,7 @@ class VarCharType : public AsciiStringSuperType {
 
  private:
   VarCharType(const std::size_t length, const bool nullable)
-      : AsciiStringSuperType(kVarChar, nullable, 1, length + 1, length) {
+      : AsciiStringSuperType<kVarChar, kOutOfLine>(nullable, 1, length + 1, length) {
   }
 
   template <bool nullable_internal>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/959cd0d4/types/YearMonthIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/YearMonthIntervalType.hpp b/types/YearMonthIntervalType.hpp
index a2ba175..f768046 100644
--- a/types/YearMonthIntervalType.hpp
+++ b/types/YearMonthIntervalType.hpp
@@ -39,12 +39,9 @@ namespace quickstep {
 /**
  * @brief A type representing the year-month interval.
  **/
-class YearMonthIntervalType : public Type {
+class YearMonthIntervalType :
+    public TypeConcept<kYearMonthInterval, false, kNativeEmbedded, YearMonthIntervalLit> {
  public:
-  typedef YearMonthIntervalLit cpptype;
-
-  static const TypeID kStaticTypeID = kYearMonthInterval;
-
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
    *        Type.
@@ -115,7 +112,9 @@ class YearMonthIntervalType : public Type {
 
  private:
   explicit YearMonthIntervalType(const bool nullable)
-      : Type(Type::kOther, kYearMonthInterval, nullable, sizeof(YearMonthIntervalLit), sizeof(YearMonthIntervalLit)) {
+      : TypeConcept<kYearMonthInterval, false, kNativeEmbedded, YearMonthIntervalLit>(
+            Type::kOther, kStaticTypeID, nullable,
+            sizeof(YearMonthIntervalLit), sizeof(YearMonthIntervalLit)) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(YearMonthIntervalType);

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/959cd0d4/types/containers/ColumnVector.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.hpp b/types/containers/ColumnVector.hpp
index fc65656..5d11098 100644
--- a/types/containers/ColumnVector.hpp
+++ b/types/containers/ColumnVector.hpp
@@ -119,6 +119,8 @@ class ColumnVector {
  **/
 class NativeColumnVector : public ColumnVector {
  public:
+  static constexpr bool kNative = true;
+
   /**
    * @brief Constructor for a NativeColumnVector which owns its own array of
    *        values.
@@ -130,8 +132,8 @@ class NativeColumnVector : public ColumnVector {
   NativeColumnVector(const Type &type, const std::size_t reserved_length)
       : ColumnVector(type),
         type_length_(type.maximumByteLength()),
-        values_(std::malloc(type.maximumByteLength() * reserved_length)),
         reserved_length_(reserved_length),
+        values_(std::malloc(type.maximumByteLength() * reserved_length)),
         actual_length_(0u),
         null_bitmap_(type.isNullable() ? new BitVector<false>(reserved_length) : nullptr) {
     DCHECK(UsableForType(type_));
@@ -385,8 +387,9 @@ class NativeColumnVector : public ColumnVector {
 
  private:
   const std::size_t type_length_;
-  void *values_;
   const std::size_t reserved_length_;
+
+  void *values_;
   std::size_t actual_length_;
   std::unique_ptr<BitVector<false>> null_bitmap_;
 
@@ -399,6 +402,8 @@ class NativeColumnVector : public ColumnVector {
  **/
 class IndirectColumnVector : public ColumnVector {
  public:
+  static constexpr bool kNative = false;
+
   /**
    * @brief Constructor.
    *
@@ -493,11 +498,21 @@ class IndirectColumnVector : public ColumnVector {
    * @param value A value to append to this NativeColumnVector.
    **/
   inline void appendTypedValue(TypedValue &&value) {
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+    DCHECK(value.isPlausibleInstanceOf(type_.getSignature())) << type_.getName();
     DCHECK_LT(values_.size(), reserved_length_);
     values_.emplace_back(std::move(value));
   }
 
+  inline void appendNullValue() {
+    DCHECK(type_.isNullable());
+    DCHECK_LT(values_.size(), reserved_length_);
+    values_.emplace_back(type_.makeNullValue());
+  }
+
+  inline void fillWithNulls() {
+    fillWithValue(type_.makeNullValue());
+  }
+
   /**
    * @brief Fill this entire ColumnVector with copies of value.
    *
@@ -559,6 +574,7 @@ class IndirectColumnVector : public ColumnVector {
  private:
   const bool type_is_nullable_;
   const std::size_t reserved_length_;
+
   std::vector<TypedValue> values_;
 
   DISALLOW_COPY_AND_ASSIGN(IndirectColumnVector);

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/959cd0d4/types/operations/OperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationFactory.cpp b/types/operations/OperationFactory.cpp
new file mode 100644
index 0000000..5d4295a
--- /dev/null
+++ b/types/operations/OperationFactory.cpp
@@ -0,0 +1,360 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#include "types/operations/OperationFactory.hpp"
+
+#include <list>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypeUtil.hpp"
+#include "types/TypedValue.hpp"
+#include "types/operations/Operation.hpp"
+#include "types/operations/OperationSignature.hpp"
+#include "types/operations/binary_operations/ArithmeticBinaryOperations.hpp"
+#include "types/operations/binary_operations/AsciiStringBinaryOperations.hpp"
+#include "types/operations/binary_operations/BinaryOperationWrapper.hpp"
+#include "types/operations/binary_operations/CMathBinaryOperations.hpp"
+#include "types/operations/unary_operations/ArithmeticUnaryOperations.hpp"
+#include "types/operations/unary_operations/AsciiStringUnaryOperations.hpp"
+#include "types/operations/unary_operations/CMathUnaryOperations.hpp"
+#include "types/operations/unary_operations/CastOperation.hpp"
+#include "types/operations/unary_operations/DateExtractOperation.hpp"
+#include "types/operations/unary_operations/SubstringOperation.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
+#include "utility/StringUtil.hpp"
+
+namespace quickstep {
+
+namespace {
+
+struct FunctorPackDispatcher {
+  template <typename FunctorT>
+  inline static std::list<OperationPtr> Generate(
+      std::enable_if_t<FunctorT::kOperationSuperTypeID == Operation::kUnaryOperation>* = 0) {
+    return { std::make_shared<const UnaryOperationWrapper<FunctorT>>() };
+  }
+
+  template <typename FunctorT>
+  inline static std::list<OperationPtr> Generate(
+      std::enable_if_t<FunctorT::kOperationSuperTypeID == Operation::kBinaryOperation>* = 0) {
+    return { std::make_shared<const BinaryOperationWrapper<FunctorT>>() };
+  }
+
+  template <typename FunctorT>
+  inline static std::list<OperationPtr> Generate(
+      decltype(FunctorT::template GenerateOperations<FunctorPackDispatcher>())* = 0) {
+    return FunctorT::template GenerateOperations<FunctorPackDispatcher>();
+  }
+};
+
+}  // namespace
+
+OperationFactory::OperationFactory() {
+  registerOperation<CastOperation>();
+  registerOperation<DateExtractOperation>();
+  registerOperation<SubstringOperation>();
+
+  registerFunctorPack<ArithmeticUnaryFunctorPack>();
+  registerFunctorPack<AsciiStringUnaryFunctorPack>();
+  registerFunctorPack<CMathUnaryFunctorPack>();
+
+  registerFunctorPack<ArithmeticBinaryFunctorPack>();
+  registerFunctorPack<AsciiStringBinaryFunctorPack>();
+  registerFunctorPack<CMathBinaryFunctorPack>();
+}
+
+OperationSignaturePtr OperationFactory::resolveOperation(
+    const std::string &operation_name,
+    const std::shared_ptr<const std::vector<const Type*>> &argument_types,
+    const std::shared_ptr<const std::vector<TypedValue>> &static_arguments,
+    std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
+    std::shared_ptr<const std::vector<TypedValue>> *coerced_static_arguments,
+    std::string *message) const {
+  const std::string lower_case_name = ToLower(operation_name);
+  const std::size_t arity = argument_types->size();
+  const auto &indices_it =
+      primary_index_.find(std::make_pair(lower_case_name, arity));
+
+  if (indices_it == primary_index_.end()) {
+    *message = "Unrecognized function " + operation_name
+                   + " with " + std::to_string(arity) + " arguments";
+  }
+
+  ResolveStatus status;
+  OperationSignaturePtr op_signature = nullptr;
+  const auto &secondary_indices = indices_it->second;
+
+  std::vector<TypeID> argument_type_ids;
+  for (const auto *type : *argument_types) {
+    argument_type_ids.emplace_back(type->getTypeID());
+  }
+
+  // First, try full exact matching.
+  status = resolveOperationWithFullTypeMatch(secondary_indices.full_match_index,
+                                             argument_type_ids,
+                                             *argument_types,
+                                             *static_arguments,
+                                             coerced_static_arguments,
+                                             &op_signature,
+                                             message);
+  if (status == ResolveStatus::kSuccess) {
+    DCHECK(op_signature != nullptr);
+    *coerced_argument_types = argument_types;
+    return op_signature;
+  } else if (status == ResolveStatus::kError) {
+    return nullptr;
+  }
+
+  // Otherwise, try partial (non-static arguments) exact matching.
+  status = resolveOperationWithPartialTypeMatch(secondary_indices.non_static_match_index,
+                                                argument_type_ids,
+                                                *argument_types,
+                                                *static_arguments,
+                                                coerced_argument_types,
+                                                coerced_static_arguments,
+                                                &op_signature,
+                                                message);
+  if (status == ResolveStatus::kSuccess) {
+    DCHECK(op_signature != nullptr);
+    return op_signature;
+  } else if (status == ResolveStatus::kError) {
+    return nullptr;
+  }
+
+  // TODO
+  *message = "Cannot resolve function " + operation_name;
+  return nullptr;
+}
+
+OperationFactory::ResolveStatus OperationFactory::resolveOperationWithFullTypeMatch(
+    const FullSignatureIndex &full_match_index,
+    const std::vector<TypeID> &argument_type_ids,
+    const std::vector<const Type*> &argument_types,
+    const std::vector<TypedValue> &static_arguments,
+    std::shared_ptr<const std::vector<TypedValue>> *partial_static_arguments,
+    OperationSignaturePtr *resolved_op_signature,
+    std::string *message) const {
+  const std::size_t max_num_static_arguments = static_arguments.size();
+  auto it = full_match_index.lower_bound(
+      std::make_pair(&argument_type_ids, max_num_static_arguments));
+
+  if (it != full_match_index.end() && *it->first.first == argument_type_ids) {
+    const OperationSignaturePtr op_signature = it->second;
+    const OperationPtr operation = getOperation(op_signature);
+
+    *partial_static_arguments =
+        std::make_shared<const std::vector<TypedValue>>(
+            static_arguments.begin()
+                + (max_num_static_arguments - op_signature->getNumStaticArguments()),
+            static_arguments.end());
+
+    if (canApplyOperationTo(operation,
+                            argument_types,
+                            **partial_static_arguments,
+                            message)) {
+      *resolved_op_signature = op_signature;
+      return ResolveStatus::kSuccess;
+    } else {
+      return ResolveStatus::kError;
+    }
+  }
+
+  return ResolveStatus::kNotFound;
+}
+
+OperationFactory::ResolveStatus OperationFactory::resolveOperationWithPartialTypeMatch(
+    const PartialSignatureIndex &non_static_match_index,
+    const std::vector<TypeID> &argument_type_ids,
+    const std::vector<const Type*> &argument_types,
+    const std::vector<TypedValue> &static_arguments,
+    std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
+    std::shared_ptr<const std::vector<TypedValue>> *coerced_static_arguments,
+    OperationSignaturePtr *resolved_op_signature,
+    std::string *message) const {
+  const std::size_t arity = argument_types.size();
+  const std::size_t max_num_static_arguments = static_arguments.size();
+  const std::size_t first_static_argument_position = arity - max_num_static_arguments;
+
+  auto it = non_static_match_index.lower_bound(
+      std::make_pair(&argument_type_ids, max_num_static_arguments));
+  while (it != non_static_match_index.end()) {
+    const std::vector<TypeID> &expected_type_ids = *it->first.first;
+    DCHECK_GE(expected_type_ids.size(), it->first.second);
+    const std::size_t num_non_static_arguments =
+        expected_type_ids.size() - it->first.second;
+
+    if (!std::equal(expected_type_ids.begin(),
+                    expected_type_ids.begin() + num_non_static_arguments,
+                    argument_type_ids.begin())) {
+      break;
+    }
+
+    // Coerce static arguments
+    std::vector<const Type*> coerced_static_arg_types;
+    std::vector<TypedValue> coerced_static_args;
+
+    bool is_coercible = true;
+    for (std::size_t i = num_non_static_arguments; i < arity; ++i) {
+      const Type &arg_type = *argument_types.at(i);
+      const TypedValue &arg_value =
+          static_arguments.at(i - first_static_argument_position);
+      const TypeID &expected_type_id = expected_type_ids.at(i);
+
+      if (arg_type.getTypeID() == expected_type_id) {
+        coerced_static_arg_types.emplace_back(&arg_type);
+        coerced_static_args.emplace_back(arg_value);
+      } else {
+        const Type *expected_type = nullptr;
+        if (TypeUtil::IsParameterized(expected_type_id)) {
+          // TODO: refactor type system to make this coercion extensible.
+          if (expected_type_id == kChar && arg_type.getTypeID() == kVarChar) {
+            expected_type = &TypeFactory::GetType(
+                expected_type_id, arg_type.maximumByteLength() - 1);
+          } else if (expected_type_id == kVarChar && arg_type.getTypeID() == kChar) {
+            expected_type = &TypeFactory::GetType(
+                expected_type_id, arg_type.maximumByteLength() + 1);
+          }
+        } else {
+          expected_type = &TypeFactory::GetType(expected_type_id);
+        }
+
+        if (expected_type != nullptr && expected_type->isSafelyCoercibleFrom(arg_type)) {
+          coerced_static_arg_types.emplace_back(expected_type);
+          coerced_static_args.emplace_back(
+              expected_type->coerceValue(arg_value, arg_type));
+        } else {
+          is_coercible = false;
+          break;
+        }
+      }
+    }
+
+    if (is_coercible) {
+      std::vector<const Type*> coerced_arg_types(
+          argument_types.begin(),
+          argument_types.begin() + num_non_static_arguments);
+      for (const Type *type : coerced_static_arg_types) {
+        coerced_arg_types.emplace_back(type);
+      }
+
+      const OperationPtr operation = getOperation(it->second);
+      if (canApplyOperationTo(operation,
+                              coerced_arg_types,
+                              coerced_static_args,
+                              message)) {
+        *coerced_argument_types =
+            std::make_shared<const std::vector<const Type*>>(std::move(coerced_arg_types));
+        *coerced_static_arguments =
+            std::make_shared<const std::vector<TypedValue>>(std::move(coerced_static_args));
+        *resolved_op_signature = it->second;
+        return ResolveStatus::kSuccess;
+      }
+    }
+
+    ++it;
+  }
+
+  return ResolveStatus::kNotFound;
+}
+
+bool OperationFactory::canApplyOperationTo(
+    const OperationPtr operation,
+    const std::vector<const Type*> &argument_types,
+    const std::vector<TypedValue> &static_arguments,
+    std::string *message) const {
+  switch (operation->getOperationSuperTypeID()) {
+    case Operation::kUnaryOperation: {
+      const UnaryOperationPtr unary_operation =
+          std::static_pointer_cast<const UnaryOperation>(operation);
+      return unary_operation->canApplyTo(*argument_types[0],
+                                         static_arguments,
+                                         message);
+    }
+    case Operation::kBinaryOperation: {
+      const BinaryOperationPtr binary_operation =
+          std::static_pointer_cast<const BinaryOperation>(operation);
+      return binary_operation->canApplyTo(*argument_types[0],
+                                          *argument_types[1],
+                                          static_arguments,
+                                          message);
+    }
+    default: {
+      const auto operation_id =
+         static_cast<std::underlying_type_t<Operation::OperationSuperTypeID>>(
+             operation->getOperationSuperTypeID());
+      LOG(FATAL) << "Unknown opeation super type id: " << operation_id;
+    }
+  }
+}
+
+
+const OperationFactory& OperationFactory::Instance() {
+  static OperationFactory instance;
+  return instance;
+}
+
+template <typename OperationT>
+void OperationFactory::registerOperation() {
+  registerOperationInternal(std::make_shared<const OperationT>());
+}
+
+template <typename FunctorPackT>
+void OperationFactory::registerFunctorPack() {
+  for (const OperationPtr &operation :
+           FunctorPackT::template GenerateOperations<FunctorPackDispatcher>()) {
+    registerOperationInternal(operation);
+  }
+}
+
+void OperationFactory::registerOperationInternal(const OperationPtr &operation) {
+  DCHECK(operation->getOperationSuperTypeID() == Operation::kUnaryOperation ||
+         operation->getOperationSuperTypeID() == Operation::kBinaryOperation);
+
+  for (const OperationSignaturePtr op_sig_orig : operation->getSignatures()) {
+    DCHECK(operation->getOperationSuperTypeID() != Operation::kUnaryOperation ||
+           op_sig_orig->getNonStaticArity() == 1u);
+    DCHECK(operation->getOperationSuperTypeID() != Operation::kBinaryOperation ||
+           op_sig_orig->getNonStaticArity() == 2u);
+
+    const OperationSignaturePtr op_sig =
+        OperationSignature::Create(ToLower(op_sig_orig->getName()),
+                                   op_sig_orig->getArgumentTypeIDs(),
+                                   op_sig_orig->getNumStaticArguments());
+
+    // TODO: print error message for collision
+    operations_.emplace(op_sig, operation);
+
+    OperationSecondaryIndices &indices =
+        primary_index_[std::make_pair(op_sig->getName(), op_sig->getArity())];
+    const SignatureReference sig_ref =
+        std::make_pair(&op_sig->getArgumentTypeIDs(),
+                       op_sig->getNumStaticArguments());
+    indices.full_match_index.emplace(sig_ref, op_sig);
+    indices.non_static_match_index.emplace(sig_ref, op_sig);
+    indices.signatures.emplace(op_sig);
+  }
+}
+
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/959cd0d4/types/operations/OperationFactory.hpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationFactory.hpp b/types/operations/OperationFactory.hpp
new file mode 100644
index 0000000..6ecc84f
--- /dev/null
+++ b/types/operations/OperationFactory.hpp
@@ -0,0 +1,231 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_OPERATIONS_OPERATION_FACTORY_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_OPERATION_FACTORY_HPP_
+
+#include <memory>
+#include <set>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/operations/Operation.hpp"
+#include "types/operations/OperationSignature.hpp"
+#include "types/operations/binary_operations/BinaryOperation.hpp"
+#include "types/operations/unary_operations/UnaryOperation.hpp"
+#include "utility/HashPair.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class Type;
+
+/** \addtogroup Types
+ *  @{
+ */
+
+class OperationFactory {
+ public:
+  static const OperationFactory& Instance();
+
+  inline bool hasOperation(const std::string &operation_name,
+                           const std::size_t arity) const {
+    const auto indices_it =
+        primary_index_.find(std::make_pair(operation_name, arity));
+    return indices_it != primary_index_.end();
+  }
+
+  inline OperationPtr getOperation(const OperationSignaturePtr &op_signature) const {
+    DCHECK(operations_.find(op_signature) != operations_.end());
+    return operations_.at(op_signature);
+  }
+
+  inline OperationPtr getOperation(const std::string &operation_name,
+                                   const std::vector<TypeID> &argument_type_ids,
+                                   const std::size_t num_static_arguments = 0) const {
+    return getOperation(
+        OperationSignature::Create(
+            operation_name, argument_type_ids, num_static_arguments));
+  }
+
+  inline UnaryOperationPtr getUnaryOperation(
+      const OperationSignaturePtr &op_signature) const {
+    const OperationPtr operation = getOperation(op_signature);
+    DCHECK(operation->getOperationSuperTypeID() == Operation::kUnaryOperation);
+    return std::static_pointer_cast<const UnaryOperation>(operation);
+  }
+
+  inline UnaryOperationPtr getUnaryOperation(
+      const std::string &operation_name,
+      const std::vector<TypeID> &argument_type_ids,
+      const std::size_t num_static_arguments = 0) const {
+    return getUnaryOperation(
+        OperationSignature::Create(
+            operation_name, argument_type_ids, num_static_arguments));
+  }
+
+  inline BinaryOperationPtr getBinaryOperation(
+      const OperationSignaturePtr &op_signature) const {
+    const OperationPtr operation = getOperation(op_signature);
+    DCHECK(operation->getOperationSuperTypeID() == Operation::kBinaryOperation);
+    return std::static_pointer_cast<const BinaryOperation>(operation);
+  }
+
+  inline BinaryOperationPtr getBinaryOperation(
+      const std::string &operation_name,
+      const std::vector<TypeID> &argument_type_ids,
+      const std::size_t num_static_arguments = 0) const {
+    return getBinaryOperation(
+        OperationSignature::Create(
+            operation_name, argument_type_ids, num_static_arguments));
+  }
+
+  OperationSignaturePtr resolveOperation(
+      const std::string &operation_name,
+      const std::shared_ptr<const std::vector<const Type*>> &argument_types,
+      const std::shared_ptr<const std::vector<TypedValue>> &static_arguments,
+      std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
+      std::shared_ptr<const std::vector<TypedValue>> *coerced_static_arguments,
+      std::string *message) const;
+
+ private:
+  OperationFactory();
+
+  template <typename OperationT>
+  void registerOperation();
+
+  template <typename FunctorPackT>
+  void registerFunctorPack();
+
+  void registerOperationInternal(const OperationPtr &operation);
+
+  using SignatureReference = std::pair<const std::vector<TypeID>*, std::size_t>;
+
+  struct FullSignatureLess {
+    inline bool operator()(const SignatureReference &lhs,
+                           const SignatureReference &rhs) const {
+      int cmp_code = static_cast<int>(lhs.first->size())
+                         - static_cast<int>(lhs.first->size());
+      if (cmp_code != 0) {
+        return cmp_code < 0;
+      }
+      for (std::size_t i = 0; i < lhs.first->size(); ++i) {
+        cmp_code = static_cast<int>(lhs.first->at(i))
+                       - static_cast<int>(rhs.first->at(i));
+        if (cmp_code != 0) {
+          return cmp_code < 0;
+        }
+      }
+      return lhs.second > rhs.second;
+    }
+  };
+  using FullSignatureIndex = std::map<SignatureReference,
+                                      OperationSignaturePtr,
+                                      FullSignatureLess>;
+
+  struct PartialSignatureLess {
+    inline bool operator()(const SignatureReference &lhs,
+                           const SignatureReference &rhs) const {
+      const std::size_t l_arity = lhs.first->size() - lhs.second;
+      const std::size_t r_arity = rhs.first->size() - rhs.second;
+      int cmp_code = static_cast<int>(l_arity) - static_cast<int>(r_arity);
+      if (cmp_code != 0) {
+        return cmp_code < 0;
+      }
+      for (std::size_t i = 0; i < l_arity; ++i) {
+        cmp_code = static_cast<int>(lhs.first->at(i))
+                       - static_cast<int>(rhs.first->at(i));
+        if (cmp_code != 0) {
+          return cmp_code < 0;
+        }
+      }
+      return lhs.second > rhs.second;
+    }
+  };
+  using PartialSignatureIndex = std::map<SignatureReference,
+                                         OperationSignaturePtr,
+                                         PartialSignatureLess>;
+
+  struct OperationSecondaryIndices {
+    std::set<OperationSignaturePtr> signatures;
+    FullSignatureIndex full_match_index;
+    PartialSignatureIndex non_static_match_index;
+  };
+
+  enum class ResolveStatus {
+    kSuccess = 0,
+    kError,
+    kNotFound
+  };
+
+  ResolveStatus resolveOperationWithFullTypeMatch(
+      const FullSignatureIndex &full_match_index,
+      const std::vector<TypeID> &argument_type_ids,
+      const std::vector<const Type*> &argument_types,
+      const std::vector<TypedValue> &static_arguments,
+      std::shared_ptr<const std::vector<TypedValue>> *trimmed_static_arguments,
+      OperationSignaturePtr *resolved_op_signature,
+      std::string *message) const;
+
+  ResolveStatus resolveOperationWithPartialTypeMatch(
+      const PartialSignatureIndex &non_static_match_index,
+      const std::vector<TypeID> &argument_type_ids,
+      const std::vector<const Type*> &argument_types,
+      const std::vector<TypedValue> &static_arguments,
+      std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
+      std::shared_ptr<const std::vector<TypedValue>> *coerced_static_arguments,
+      OperationSignaturePtr *resolved_op_signature,
+      std::string *message) const;
+
+//  ResolveStatus resolveOperationGeneric(
+//      const std::set<OperationSignaturePtr> signatures,
+//      const std::vector<TypeID> &argument_type_ids,
+//      const std::vector<const Type*> &argument_types,
+//      const std::vector<TypedValue> &static_arguments,
+//      std::shared_ptr<const std::vector<const Type*>> *coerced_argument_types,
+//      std::shared_ptr<const std::vector<TypedValue>> *coerced_static_arguments,
+//      OperationSignaturePtr *op_signature,
+//      std::string *message) const;
+
+  bool canApplyOperationTo(const OperationPtr operation,
+                           const std::vector<const Type*> &argument_types,
+                           const std::vector<TypedValue> &static_arguments,
+                           std::string *message) const;
+
+  std::unordered_map<OperationSignaturePtr,
+                     OperationPtr,
+                     OperationSignatureHash,
+                     OperationSignatureEqual> operations_;
+
+  std::unordered_map<std::pair<std::string, std::size_t>,
+                     OperationSecondaryIndices> primary_index_;
+
+  DISALLOW_COPY_AND_ASSIGN(OperationFactory);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_OPERATION_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/959cd0d4/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/959cd0d4/types/operations/OperationSignature.hpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationSignature.hpp b/types/operations/OperationSignature.hpp
new file mode 100644
index 0000000..b326aef
--- /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/959cd0d4/types/operations/OperationUtil.hpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationUtil.hpp b/types/operations/OperationUtil.hpp
new file mode 100644
index 0000000..076dc0c
--- /dev/null
+++ b/types/operations/OperationUtil.hpp
@@ -0,0 +1,334 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_OPERATIONS_OPERATION_UTIL_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_OPERATION_UTIL_HPP_
+
+#include <cstddef>
+#include <list>
+#include <string>
+#include <type_traits>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "types/Type.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename FunctorT, typename ...SpecArgs>
+struct FunctorSpecializer {
+  template <bool specialize = (sizeof...(SpecArgs) != 0),
+            typename EnableT = void>
+  struct Implementation;
+
+  typedef Implementation<> type;
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <bool specialize>
+struct FunctorSpecializer<FunctorT, SpecArgs...>
+    ::Implementation<specialize, std::enable_if_t<specialize>> {
+  template <typename ...FuncArgs>
+  inline static auto Invoke(const FunctorT &functor, FuncArgs &&...args) {
+    return functor.template apply<SpecArgs...>(std::forward<FuncArgs>(args)...);
+  }
+  typedef FunctorT FunctorType;
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <bool specialize>
+struct FunctorSpecializer<FunctorT, SpecArgs...>
+    ::Implementation<specialize, std::enable_if_t<!specialize>> {
+  template <typename ...FuncArgs>
+  inline static auto Invoke(const FunctorT &functor, FuncArgs &&...args) {
+    return functor.apply(std::forward<FuncArgs>(args)...);
+  }
+  typedef FunctorT FunctorType;
+};
+
+template <typename ColumnVectorT>
+struct ColumnVectorValueAccessor {
+  explicit ColumnVectorValueAccessor(const ColumnVectorT &column_vector_in)
+      : column_vector(column_vector_in),
+        length(column_vector.size()) {}
+
+  inline void beginIteration() {
+    pos = static_cast<std::size_t>(-1);
+  }
+
+  inline bool next() {
+    return (++pos) < length;
+  }
+
+  inline std::size_t getNumTuples() const {
+    return length;
+  }
+
+  template <bool nullable>
+  inline const void* getUntypedValue(const attribute_id) const {
+    return column_vector.template getUntypedValue<nullable>(pos);
+  }
+
+  inline TypedValue getTypedValue(const attribute_id) const {
+    return column_vector.getTypedValue(pos);
+  }
+
+  const ColumnVectorT &column_vector;
+  const std::size_t length;
+  std::size_t pos;
+};
+
+template <typename FuncSpec, typename T, typename EnableT = void>
+struct Codegen;
+
+template <typename FuncSpec, typename T>
+struct Codegen<FuncSpec, T, std::enable_if_t<T::kLayout == kNativeEmbedded>> {
+  using ColumnVectorType = NativeColumnVector;
+  using FunctorSpecializer = FuncSpec;
+
+  using NativeType = typename T::cpptype;
+  using NativeTypeConst = const typename T::cpptype;
+  using NativeTypeConstRef = const NativeType&;
+  using NativeTypeConstPtr = const NativeType*;
+
+  template <typename ArgumentGen>
+  inline static TypedValue ApplyUnaryTypedValue(
+      typename ArgumentGen::NativeTypeConstRef argument,
+      const Type &result_type,
+      const typename FuncSpec::FunctorType &functor) {
+    return TypedValue(FuncSpec::Invoke(functor, argument));
+  }
+
+  template <typename ArgumentGen>
+  inline static void ApplyUnaryColumnVector(
+      const typename ArgumentGen::NativeTypeConstRef argument,
+      const typename FuncSpec::FunctorType &functor,
+      ColumnVectorType *cv) {
+    *static_cast<NativeType *>(cv->getPtrForDirectWrite()) =
+        FuncSpec::Invoke(functor, argument);
+  }
+
+  template <typename LeftGen, typename RightGen>
+  inline static TypedValue ApplyBinaryTypedValue(
+      typename LeftGen::NativeTypeConstRef left,
+      typename RightGen::NativeTypeConstRef right,
+      const Type &result_type,
+      const typename FuncSpec::FunctorType &functor) {
+    return TypedValue(FuncSpec::Invoke(functor, left, right));
+  }
+
+  template <typename LeftGen, typename RightGen>
+  inline static void ApplyBinaryColumnVector(
+      const typename LeftGen::NativeTypeConstRef left,
+      const typename RightGen::NativeTypeConstRef right,
+      const typename FuncSpec::FunctorType &functor,
+      ColumnVectorType *cv) {
+    *static_cast<NativeType *>(cv->getPtrForDirectWrite()) =
+        FuncSpec::Invoke(functor, left, right);
+  }
+
+  template <bool nullable, typename AccessorT>
+  inline static NativeTypeConstPtr GetValuePtr(
+      const AccessorT *accessor,
+      const attribute_id attr_id) {
+    return static_cast<NativeTypeConstPtr>(
+        accessor->template getUntypedValue<nullable>(attr_id));
+  }
+
+  inline static bool IsNull(const NativeType *value) {
+    return value == nullptr;
+  }
+
+  // Dereference: NativeTypeConstPtr& -> const NativeType&
+  inline static const NativeType& Dereference(const NativeType *value) {
+    return *value;
+  }
+
+  inline static const NativeType ToNativeValueConst(const TypedValue &value) {
+    return value.getLiteral<NativeType>();
+  }
+};
+
+template <typename FuncSpec, typename T>
+struct Codegen<FuncSpec, T, std::enable_if_t<T::kLayout == kNonNativeInline>> {
+  using ColumnVectorType = NativeColumnVector;
+  using FunctorSpecializer = FuncSpec;
+
+  using NativeType = void*;
+  using NativeTypeConst = const void*;
+  using NativeTypeConstRef = const void*;
+  using NativeTypeConstPtr = const void*;
+
+  template <typename ArgumentGen>
+  inline static TypedValue ApplyUnaryTypedValue(
+      typename ArgumentGen::NativeTypeConstRef argument,
+      const Type &result_type,
+      const typename FuncSpec::FunctorType &functor) {
+    void *result = std::malloc(result_type.maximumByteLength());
+    FuncSpec::Invoke(functor, argument, result);
+    return TypedValue::CreateWithOwnedData(T::kStaticTypeID,
+                                           result,
+                                           result_type.maximumByteLength());
+  }
+
+  template <typename ArgumentGen>
+  inline static void ApplyUnaryColumnVector(
+      const typename ArgumentGen::NativeTypeConstRef argument,
+      const typename FuncSpec::FunctorType &functor,
+      ColumnVectorType *cv) {
+    FuncSpec::Invoke(functor, argument, cv->getPtrForDirectWrite());
+  }
+
+  template <typename LeftGen, typename RightGen>
+  inline static TypedValue ApplyBinaryTypedValue(
+      typename LeftGen::NativeTypeConstRef left,
+      typename RightGen::NativeTypeConstRef right,
+      const Type &result_type,
+      const typename FuncSpec::FunctorType &functor) {
+    void *result = std::malloc(result_type.maximumByteLength());
+    FuncSpec::Invoke(functor, left, right, result);
+    return TypedValue::CreateWithOwnedData(T::kStaticTypeID,
+                                           result,
+                                           result_type.maximumByteLength());
+  }
+
+  template <typename LeftGen, typename RightGen>
+  inline static void ApplyBinaryColumnVector(
+      const typename LeftGen::NativeTypeConstRef left,
+      const typename RightGen::NativeTypeConstRef right,
+      const typename FuncSpec::FunctorType &functor,
+      ColumnVectorType *cv) {
+    FuncSpec::Invoke(functor, left, right, cv->getPtrForDirectWrite());
+  }
+
+  template <bool nullable, typename AccessorT>
+  inline static NativeTypeConstPtr GetValuePtr(
+      const AccessorT *accessor,
+      const attribute_id attr_id) {
+    return accessor->template getUntypedValue<nullable>(attr_id);
+  }
+
+  inline static bool IsNull(const void *value) {
+    return value == nullptr;
+  }
+
+  // Dereference: NativeTypeConstPtr& -> const NativeType&
+  inline static const void* Dereference(const void *value) {
+    return value;
+  }
+
+  inline static const void* ToNativeValueConst(const TypedValue &value) {
+    return value.getDataPtr();
+  }
+};
+
+template <typename FuncSpec, typename T>
+struct Codegen<FuncSpec, T, std::enable_if_t<T::kLayout == kOutOfLine>> {
+  using ColumnVectorType = IndirectColumnVector;
+  using FunctorSpecializer = FuncSpec;
+
+  using NativeType = TypedValue;
+  using NativeTypeConst = const TypedValue;
+  using NativeTypeConstRef = const TypedValue&;
+  using NativeTypeConstPtr = const TypedValue;
+
+  template <typename ArgumentGen>
+  inline static TypedValue ApplyUnaryTypedValue(
+      typename ArgumentGen::NativeTypeConstRef argument,
+      const Type &result_type,
+      const typename FuncSpec::FunctorType &functor) {
+    return FuncSpec::Invoke(functor, argument);
+  }
+
+  template <typename ArgumentGen>
+  inline static void ApplyUnaryColumnVector(
+      const typename ArgumentGen::NativeTypeConstRef argument,
+      const typename FuncSpec::FunctorType &functor,
+      ColumnVectorType *cv) {
+    cv->appendTypedValue(FuncSpec::Invoke(functor, argument));
+  }
+
+  template <typename LeftGen, typename RightGen>
+  inline static TypedValue ApplyBinaryTypedValue(
+      typename LeftGen::NativeTypeConstRef left,
+      typename RightGen::NativeTypeConstRef right,
+      const Type &result_type,
+      const typename FuncSpec::FunctorType &functor) {
+    return FuncSpec::Invoke(functor, left, right);
+  }
+
+  template <typename LeftGen, typename RightGen>
+  inline static void ApplyBinaryColumnVector(
+      const typename LeftGen::NativeTypeConstRef left,
+      const typename RightGen::NativeTypeConstRef right,
+      const typename FuncSpec::FunctorType &functor,
+      ColumnVectorType *cv) {
+    cv->appendTypedValue(FuncSpec::Invoke(functor, left, right));
+  }
+
+  template <bool nullable, typename AccessorT>
+  inline static NativeTypeConstPtr GetValuePtr(
+      const AccessorT *accessor,
+      const attribute_id attr_id) {
+    return accessor->getTypedValue(attr_id);
+  }
+
+  inline static bool IsNull(NativeTypeConstPtr &value) {
+    return value.isNull();
+  }
+
+  // Dereference: NativeTypeConstPtr& -> const NativeType&
+  inline static const NativeType& Dereference(NativeTypeConstPtr &value) {
+    return value;
+  }
+
+  inline static const NativeType& ToNativeValueConst(const TypedValue &value) {
+    return value;
+  }
+};
+
+template <typename ...FunctorTypes>
+struct FunctorPack {
+  template <typename Dispatcher>
+  inline static std::list<OperationPtr> GenerateOperations() {
+    std::vector<std::list<OperationPtr>> op_list_groups =
+        { Dispatcher::template Generate<FunctorTypes>()... };
+
+    std::list<OperationPtr> operations;
+    for (std::list<OperationPtr> &op_list : op_list_groups) {
+      operations.splice(operations.end(), std::move(op_list));
+    }
+    return operations;
+  }
+};
+
+struct OperationPack {
+  virtual std::vector<OperationPtr> generateOperations() = 0;
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_OPERATION_UTIL_HPP_