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/04/12 19:34:19 UTC

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/unary_operations/UnaryOperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationFactory.cpp b/types/operations/unary_operations/UnaryOperationFactory.cpp
deleted file mode 100644
index b306061..0000000
--- a/types/operations/unary_operations/UnaryOperationFactory.cpp
+++ /dev/null
@@ -1,120 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#include "types/operations/unary_operations/UnaryOperationFactory.hpp"
-
-#include <string>
-
-#include "types/TypeFactory.hpp"
-#include "types/operations/Operation.pb.h"
-#include "types/operations/unary_operations/ArithmeticUnaryOperations.hpp"
-#include "types/operations/unary_operations/NumericCastOperation.hpp"
-#include "types/operations/unary_operations/DateExtractOperation.hpp"
-#include "types/operations/unary_operations/SubstringOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-const UnaryOperation& UnaryOperationFactory::GetUnaryOperation(const UnaryOperationID id) {
-  switch (id) {
-    case UnaryOperationID::kNegate:
-      return NegateUnaryOperation::Instance();
-    case UnaryOperationID::kCast:
-      FATAL_ERROR("Getting a CastOperation through GetUnaryOperation is not supported");
-    case UnaryOperationID::kDateExtract:
-      FATAL_ERROR("Getting a DateExtractOperation through GetUnaryOperation is not supported");
-    case UnaryOperationID::kSubstring:
-      FATAL_ERROR("Getting a SubstringOperation through GetUnaryOperation is not supported");
-    default:
-      FATAL_ERROR("Unknown UnaryOperationID");
-  }
-}
-
-bool UnaryOperationFactory::ProtoIsValid(const serialization::UnaryOperation &proto) {
-  // Check that UnaryOperation is fully initialized.
-  if (!proto.IsInitialized()) {
-    return false;
-  }
-
-  // Check that the operation_id is a valid UnaryOperation.
-  if (!proto.UnaryOperationID_IsValid(proto.operation_id())) {
-    return false;
-  }
-
-  switch (proto.operation_id()) {
-    case serialization::UnaryOperation::NEGATE:
-      return true;
-    case serialization::UnaryOperation::CAST:
-      return proto.HasExtension(serialization::CastOperation::target_type)
-          && TypeFactory::ProtoIsValid(proto.GetExtension(serialization::CastOperation::target_type));
-    case serialization::UnaryOperation::DATE_EXTRACT:
-      return proto.HasExtension(serialization::DateExtractOperation::unit)
-          && DateExtractOperation_Unit_IsValid(proto.GetExtension(serialization::DateExtractOperation::unit));
-    case serialization::UnaryOperation::SUBSTRING:
-      return proto.HasExtension(serialization::SubstringOperation::start_position)
-          && proto.HasExtension(serialization::SubstringOperation::substring_length);
-    default:
-      return false;
-  }
-}
-
-const UnaryOperation& UnaryOperationFactory::ReconstructFromProto(
-    const serialization::UnaryOperation &proto) {
-  DCHECK(ProtoIsValid(proto))
-      << "Attempted to create UnaryOperation from an invalid proto description:\n"
-      << proto.DebugString();
-
-  switch (proto.operation_id()) {
-    case serialization::UnaryOperation::NEGATE:
-      return GetUnaryOperation(UnaryOperationID::kNegate);
-    case serialization::UnaryOperation::CAST:
-      return NumericCastOperation::Instance(
-          TypeFactory::ReconstructFromProto(
-              proto.GetExtension(
-                  serialization::CastOperation::target_type)));
-    case serialization::UnaryOperation::DATE_EXTRACT:
-      switch (proto.GetExtension(serialization::DateExtractOperation::unit)) {
-        case serialization::DateExtractOperation::YEAR:
-          return DateExtractOperation::Instance(DateExtractUnit::kYear);
-        case serialization::DateExtractOperation::MONTH:
-          return DateExtractOperation::Instance(DateExtractUnit::kMonth);
-        case serialization::DateExtractOperation::DAY:
-          return DateExtractOperation::Instance(DateExtractUnit::kDay);
-        case serialization::DateExtractOperation::HOUR:
-          return DateExtractOperation::Instance(DateExtractUnit::kHour);
-        case serialization::DateExtractOperation::MINUTE:
-          return DateExtractOperation::Instance(DateExtractUnit::kMinute);
-        case serialization::DateExtractOperation::SECOND:
-          return DateExtractOperation::Instance(DateExtractUnit::kSecond);
-        default:
-          FATAL_ERROR("Unrecognized DateExtractOperation unit in UnaryOperation::ReconstructFromProto");
-      }
-    case serialization::UnaryOperation::SUBSTRING:
-      return SubstringOperation::Instance(
-          proto.GetExtension(serialization::SubstringOperation::start_position),
-          proto.GetExtension(serialization::SubstringOperation::substring_length));
-    default:
-      FATAL_ERROR("Unrecognized UnaryOperationID in UnaryOperation::ReconstructFromProto");
-  }
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/unary_operations/UnaryOperationFactory.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationFactory.hpp b/types/operations/unary_operations/UnaryOperationFactory.hpp
deleted file mode 100644
index 2ce83d4..0000000
--- a/types/operations/unary_operations/UnaryOperationFactory.hpp
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#ifndef QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_FACTORY_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_FACTORY_HPP_
-
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-class UnaryOperation;
-namespace serialization { class UnaryOperation; }
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief All-static factory object that provides access to UnaryOperations.
- **/
-class UnaryOperationFactory {
- public:
-  /**
-   * @brief Convenience factory method to get a pointer to a UnaryOperation
-   *        from that UnaryOperation's ID.
-   *
-   * @param id The ID of the desired UnaryOperation.
-   * @return The UnaryOperation corresponding to id.
-   **/
-  static const UnaryOperation& GetUnaryOperation(const UnaryOperationID id);
-
-  /**
-   * @brief Get a reference to a UnaryOperation from that UnaryOperation's
-   *        serialized Protocol Buffer representation.
-   *
-   * @param proto A serialized Protocol Buffer representation of a UnaryOperation,
-   *        originally generated by getProto().
-   * @return The UnaryOperation described by proto.
-   **/
-  static const UnaryOperation& ReconstructFromProto(const serialization::UnaryOperation &proto);
-
-  /**
-   * @brief Check whether a serialization::UnaryOperation is fully-formed and
-   *        all parts are valid.
-   *
-   * @param proto A serialized Protocol Buffer representation of a UnaryOperation,
-   *        originally generated by getProto().
-   * @return Whether proto is fully-formed and valid.
-   **/
-  static bool ProtoIsValid(const serialization::UnaryOperation &proto);
-
- private:
-  UnaryOperationFactory();
-
-  DISALLOW_COPY_AND_ASSIGN(UnaryOperationFactory);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_FACTORY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/unary_operations/UnaryOperationID.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationID.cpp b/types/operations/unary_operations/UnaryOperationID.cpp
deleted file mode 100644
index b47a848..0000000
--- a/types/operations/unary_operations/UnaryOperationID.cpp
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
-
-namespace quickstep {
-
-const char *kUnaryOperationNames[] = {
-  "Negate", "Cast", "DateExtract", "Substring"
-};
-
-const char *kUnaryOperationShortNames[] = {
-  "-", "Cast", "DateExtract", "Substring"
-};
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/unary_operations/UnaryOperationID.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationID.hpp b/types/operations/unary_operations/UnaryOperationID.hpp
deleted file mode 100644
index fa50f50..0000000
--- a/types/operations/unary_operations/UnaryOperationID.hpp
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#ifndef QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_ID_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_ID_HPP_
-
-#include <type_traits>
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief Concrete UnaryOperations.
- **/
-enum class UnaryOperationID {
-  kNegate = 0,
-  kCast,
-  kDateExtract,
-  kSubstring,
-  kNumUnaryOperationIDs  // Not a real UnaryOperationID, exists for counting purposes.
-};
-
-/**
- * @brief Names of comparisons in the same order as UnaryOperationID.
- * @note Defined out-of-line in UnaryOperation.cpp
- **/
-extern const char *kUnaryOperationNames[
-    static_cast<typename std::underlying_type<UnaryOperationID>::type>(
-        UnaryOperationID::kNumUnaryOperationIDs)];
-
-/**
- * @brief Short names (i.e. mathematical symbols) of comparisons in the same
- *        order as UnaryOperationID.
- * @note Defined out-of-line in UnaryOperation.cpp
- **/
-extern const char *kUnaryOperationShortNames[
-    static_cast<typename std::underlying_type<UnaryOperationID>::type>(
-        UnaryOperationID::kNumUnaryOperationIDs)];
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_ID_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/unary_operations/UnaryOperationWrapper.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationWrapper.hpp b/types/operations/unary_operations/UnaryOperationWrapper.hpp
new file mode 100644
index 0000000..59b2cf0
--- /dev/null
+++ b/types/operations/unary_operations/UnaryOperationWrapper.hpp
@@ -0,0 +1,250 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_WRAPPER_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_WRAPPER_HPP_
+
+#include <cstddef>
+#include <string>
+#include <type_traits>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/operations/OperationSignature.hpp"
+#include "types/operations/OperationUtil.hpp"
+#include "types/operations/unary_operations/UnaryOperation.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename ArgumentT, typename ResultT>
+struct UnaryFunctor {
+  typedef ArgumentT ArgumentType;
+  typedef ResultT ResultType;
+
+  static constexpr Operation
+      ::OperationSuperTypeID kOperationSuperTypeID = Operation::kUnaryOperation;
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+class UncheckedUnaryOperatorWrapperCodegen : public UncheckedUnaryOperator {
+ public:
+  template <typename ...ConstructorArgs>
+  UncheckedUnaryOperatorWrapperCodegen(const Type &argument_type,
+                                       const Type &result_type,
+                                       ConstructorArgs &&...args)
+      : functor_(std::forward<ConstructorArgs>(args)...),
+        impl_(functor_, argument_type, result_type) {}
+
+  TypedValue applyToTypedValue(const TypedValue &argument) const override {
+    return impl_.applyToTypedValue(argument);
+  }
+
+  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override {
+    using ArgumentCVT = typename ArgumentGen::ColumnVectorType;
+    DCHECK_EQ(argument.isNative(), ArgumentCVT::kNative);
+
+    using ArgumentAccessorT = ColumnVectorValueAccessor<ArgumentCVT>;
+    ArgumentAccessorT accessor(static_cast<const ArgumentCVT&>(argument));
+    return impl_.applyToValueAccessor(&accessor, 0);
+  }
+
+  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
+                                     const attribute_id attr_id) const override {
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        accessor,
+        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      return impl_.applyToValueAccessor(accessor, attr_id);
+    });
+  }
+
+ private:
+  using ArgumentType = typename FunctorT::ArgumentType;
+  using ResultType = typename FunctorT::ResultType;
+
+  using FuncSpec = typename FunctorSpecializer<FunctorT, SpecArgs...>::type;
+  using ArgumentGen = Codegen<FuncSpec, ArgumentType>;
+  using ResultGen = Codegen<FuncSpec, ResultType>;
+
+  template <bool argument_nullable>
+  struct Implementation;
+
+  const FunctorT functor_;
+  const Implementation<true> impl_;
+
+  DISALLOW_COPY_AND_ASSIGN(UncheckedUnaryOperatorWrapperCodegen);
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <bool argument_nullable>
+struct UncheckedUnaryOperatorWrapperCodegen<FunctorT, SpecArgs...>
+    ::Implementation {
+  Implementation(const FunctorT &functor_in,
+                 const Type &argument_type_in,
+                 const Type &result_type_in)
+      : functor(functor_in),
+        argument_type(argument_type_in),
+        result_type(result_type_in) {}
+
+  inline TypedValue applyToTypedValue(const TypedValue &argument) const {
+    if (argument_nullable && argument.isNull()) {
+      return TypedValue(ResultType::kStaticTypeID);
+    }
+
+    return ResultGen::template ApplyUnaryTypedValue<ArgumentGen>(
+        ArgumentGen::ToNativeValueConst(argument),
+        result_type,
+        functor);
+  }
+
+  template <typename AccessorT>
+  inline ColumnVector* applyToValueAccessor(AccessorT *accessor,
+                                            const attribute_id attr_id) const {
+    using ResultCVT = typename ResultGen::ColumnVectorType;
+    ResultCVT *result_cv = new ResultCVT(result_type, accessor->getNumTuples());
+
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename ArgumentGen::NativeTypeConstPtr arg_value =
+          ArgumentGen::template GetValuePtr<
+              argument_nullable, AccessorT>(accessor, attr_id);
+
+      if (argument_nullable && ArgumentGen::IsNull(arg_value)) {
+        result_cv->appendNullValue();
+      } else {
+        ResultGen::template ApplyUnaryColumnVector<ArgumentGen>(
+            ArgumentGen::Dereference(arg_value), functor, result_cv);
+      }
+    }
+    return result_cv;
+  }
+
+  const FunctorT &functor;
+  const Type &argument_type;
+  const Type &result_type;
+};
+
+template <typename FunctorT>
+class UnaryOperationWrapper : public UnaryOperation {
+ public:
+  UnaryOperationWrapper()
+      : UnaryOperation(),
+        operation_name_(FunctorT::GetName()) {}
+
+  std::string getName() const override {
+    return operation_name_;
+  }
+
+  std::string getShortName() const override {
+    return getName();
+  }
+
+  std::vector<OperationSignaturePtr> getSignatures() const override {
+    return {
+        OperationSignature::Create(getName(), {ArgumentType::kStaticTypeID}, 0)
+    };
+  }
+
+  bool canApplyTo(const Type &argument_type,
+                  const std::vector<TypedValue> &static_arguments,
+                  std::string *message) const override {
+    DCHECK(argument_type.getTypeID() == ArgumentType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return true;
+  }
+
+  const Type* getResultType(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(argument_type.getTypeID() == ArgumentType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return getResultTypeImpl<ResultType::kParameterized>(
+        argument_type, static_arguments);
+  }
+
+  UncheckedUnaryOperator* makeUncheckedUnaryOperator(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(argument_type.getTypeID() == ArgumentType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return makeUncheckedUnaryOperatorImpl<
+        std::is_default_constructible<FunctorT>::value>(
+            argument_type, static_arguments);
+  }
+
+ private:
+  using ArgumentType = typename FunctorT::ArgumentType;
+  using ResultType = typename FunctorT::ResultType;
+
+  template <bool functor_use_default_constructor>
+  inline UncheckedUnaryOperator* makeUncheckedUnaryOperatorImpl(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<functor_use_default_constructor>* = 0) const {
+    return new UncheckedUnaryOperatorWrapperCodegen<FunctorT>(
+        argument_type, *getResultType(argument_type, static_arguments));
+  }
+
+  template <bool functor_use_default_constructor>
+  inline UncheckedUnaryOperator* makeUncheckedUnaryOperatorImpl(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<!functor_use_default_constructor>* = 0) const {
+    return new UncheckedUnaryOperatorWrapperCodegen<FunctorT>(
+        argument_type, *getResultType(argument_type, static_arguments),
+        static_cast<const ArgumentType&>(argument_type));
+  }
+
+  template <bool result_type_has_parameter>
+  inline const Type* getResultTypeImpl(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<!result_type_has_parameter>* = 0) const {
+    return &TypeFactory::GetType(ResultType::kStaticTypeID,
+                                 argument_type.isNullable());
+  }
+
+  template <bool result_type_has_parameter>
+  inline const Type* getResultTypeImpl(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<result_type_has_parameter>* = 0) const {
+    return FunctorT::GetResultType(argument_type);
+  }
+
+  const std::string operation_name_;
+
+  DISALLOW_COPY_AND_ASSIGN(UnaryOperationWrapper);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_WRAPPER_HPP_

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/utility/meta/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/meta/CMakeLists.txt b/utility/meta/CMakeLists.txt
new file mode 100644
index 0000000..1b72dd9
--- /dev/null
+++ b/utility/meta/CMakeLists.txt
@@ -0,0 +1,41 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+# Declare micro-libs:
+add_library(quickstep_utility_meta_Common ../../empty_src.cpp Common.hpp)
+add_library(quickstep_utility_meta_Dispatchers ../../empty_src.cpp Dispatchers.hpp)
+add_library(quickstep_utility_meta_TMP ../../empty_src.cpp TMP.hpp)
+add_library(quickstep_utility_meta_TransitiveClosure ../../empty_src.cpp TransitiveClosure.hpp)
+add_library(quickstep_utility_meta_TypeList ../../empty_src.cpp TypeList.hpp)
+add_library(quickstep_utility_meta_TypeListMetaFunctions ../../empty_src.cpp TypeListMetaFunctions.hpp)
+
+# Link dependencies:
+target_link_libraries(quickstep_utility_meta_Dispatchers
+                      quickstep_utility_meta_Common)
+target_link_libraries(quickstep_utility_meta_TMP
+                      quickstep_utility_meta_Common
+                      quickstep_utility_meta_Dispatchers
+                      quickstep_utility_meta_TransitiveClosure
+                      quickstep_utility_meta_TypeList)
+target_link_libraries(quickstep_utility_meta_TransitiveClosure
+                      quickstep_utility_meta_TypeList)
+target_link_libraries(quickstep_utility_meta_TypeList
+                      quickstep_utility_meta_Common
+                      quickstep_utility_meta_TypeListMetaFunctions)
+target_link_libraries(quickstep_utility_meta_TypeListMetaFunctions
+                      quickstep_utility_meta_Common)
+

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/utility/meta/Common.hpp
----------------------------------------------------------------------
diff --git a/utility/meta/Common.hpp b/utility/meta/Common.hpp
new file mode 100644
index 0000000..39c513e
--- /dev/null
+++ b/utility/meta/Common.hpp
@@ -0,0 +1,143 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_META_COMMON_HPP_
+#define QUICKSTEP_UTILITY_META_COMMON_HPP_
+
+namespace quickstep {
+namespace meta {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+template <typename T, T ...s>
+struct Sequence {
+  template <template <typename ...> class Host>
+  using bind_to = Host<std::integral_constant<T, s>...>;
+
+  template <template <T ...> class Host>
+  using bind_values_to = Host<s...>;
+
+  template <typename U>
+  using cast_to = Sequence<U, static_cast<U>(s)...>;
+
+  template <typename CollectionT>
+  inline static CollectionT Instantiate() {
+    return { s... };
+  }
+};
+
+template <std::size_t ...s>
+using IntegerSequence = Sequence<std::size_t, s...>;
+
+
+template <std::size_t n, std::size_t ...s>
+struct MakeSequence : MakeSequence<n-1, n-1, s...> {};
+
+template <std::size_t ...s>
+struct MakeSequence<0, s...> {
+  using type = IntegerSequence<s...>;
+};
+
+
+template <typename ...> struct Conjunction : std::true_type {};
+template <typename B> struct Conjunction<B> : B {};
+template <typename B, typename ...Bs>
+struct Conjunction<B, Bs...>
+    : std::conditional_t<B::value, Conjunction<Bs...>, B> {};
+
+template <typename ...> struct Disjunction : std::false_type {};
+template <typename B> struct Disjunction<B> : B {};
+template <typename B, typename ...Bs>
+struct Disjunction<B, Bs...>
+    : std::conditional_t<B::value, B, Disjunction<Bs...>> {};
+
+template <typename check, typename ...cases>
+struct EqualsAny {
+  static constexpr bool value =
+     Disjunction<std::is_same<check, cases>...>::value;
+};
+
+
+template <typename T, typename Enable = void>
+struct IsTrait {
+  static constexpr bool value = false;
+};
+
+template <typename T>
+struct IsTrait<T, std::enable_if_t<
+    std::is_same<typename T::type, typename T::type>::value>> {
+  static constexpr bool value = true;
+};
+
+template <typename T, template <typename> class Op, typename Enable = void>
+struct IsWellFormed {
+  static constexpr bool value = false;
+};
+
+template <typename T, template <typename> class Op>
+struct IsWellFormed<T, Op, std::enable_if_t<std::is_same<Op<T>, Op<T>>::value>> {
+  static constexpr bool value = true;
+};
+
+
+template <typename LeftT, typename RightT>
+struct PairSelectorLeft {
+  typedef LeftT type;
+};
+
+template <typename LeftT, typename RightT>
+struct PairSelectorRight {
+  typedef RightT type;
+};
+
+
+template <char ...c>
+struct StringLiteral {
+  inline static std::string ToString() {
+    return std::string({c...});
+  }
+};
+
+template <template <typename ...> class Op>
+class TraitWrapper {
+ private:
+  template <typename ...ArgTypes>
+  struct Implemenation {
+    using type = Op<ArgTypes...>;
+  };
+
+ public:
+  template <typename ...ArgTypes>
+  using type = Implemenation<ArgTypes...>;
+};
+
+template <template <typename ...> class Op>
+struct TraitUnwrapper {
+  template <typename ...ArgTypes>
+  using type = typename Op<ArgTypes...>::type;
+};
+
+/** @} */
+
+}  // namespace meta
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_META_COMMON_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/utility/meta/Dispatchers.hpp
----------------------------------------------------------------------
diff --git a/utility/meta/Dispatchers.hpp b/utility/meta/Dispatchers.hpp
new file mode 100644
index 0000000..5b0ee48
--- /dev/null
+++ b/utility/meta/Dispatchers.hpp
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_META_DISPATCHERS_HPP_
+#define QUICKSTEP_UTILITY_META_DISPATCHERS_HPP_
+
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+namespace meta {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+/**
+ * @brief A helper function for bool branched template specialization.
+ *
+ * Usage example:
+ * --
+ * bool c1 = true, c2 = false;
+ *
+ * InvokeOnBools(
+ *     c1, c2,
+ *     [&](auto c1, auto c2) -> SomeBaseClass* {
+ *   using T1 = decltype(c1);  // T1 == std::true_type
+ *   using T2 = decltype(c2);  // T2 == std::false_type
+ *
+ *   constexpr bool cv1 = T1::value;  // cv1 == true
+ *   constexpr bool cv2 = T2::value;  // cv2 == false
+ *
+ *   SomeFunction<cv1, cv2>(...);
+ *   return new SomeClass<cv1, cv2>(...);
+ * });
+ * --
+ */
+template <typename ...ArgTypes>
+inline auto InvokeOnBools(ArgTypes ...args);
+
+
+namespace internal {
+
+/**
+ * @brief Invoke the functor with the compile-time bool values wrapped as
+ *        integral_constant types.
+ */
+template <typename FunctorT, bool ...bool_values>
+inline auto InvokeOnBoolsInner(const FunctorT &functor) {
+  return functor(std::integral_constant<bool, bool_values>()...);
+}
+
+/**
+ * @brief Recursive dispatching.
+ */
+template <typename FunctorT, bool ...bool_values, typename ...Bools>
+inline auto InvokeOnBoolsInner(const FunctorT &functor,
+                               const bool tparam,
+                               const Bools ...rest_params) {
+  if (tparam) {
+    return InvokeOnBoolsInner<FunctorT, bool_values..., true>(
+        functor, rest_params...);
+  } else {
+    return InvokeOnBoolsInner<FunctorT, bool_values..., false>(
+        functor, rest_params...);
+  }
+}
+
+/**
+ * @brief Move the functor to the first position in argument list.
+ */
+template <std::size_t last, std::size_t ...i, typename TupleT>
+inline auto InvokeOnBoolsInner(TupleT &&args, IntegerSequence<i...> &&indices) {
+  return InvokeOnBoolsInner(std::get<last>(std::forward<TupleT>(args)),
+                            std::get<i>(std::forward<TupleT>(args))...);
+}
+
+}  // namespace internal
+
+template <typename ...ArgTypes>
+inline auto InvokeOnBools(ArgTypes ...args) {
+  constexpr std::size_t last = sizeof...(args) - 1;
+  return internal::InvokeOnBoolsInner<last>(std::forward_as_tuple(args...),
+                                            typename MakeSequence<last>::type());
+}
+
+/** @} */
+
+}  // namespace meta
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_META_DISPATCHERS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/utility/meta/TMP.hpp
----------------------------------------------------------------------
diff --git a/utility/meta/TMP.hpp b/utility/meta/TMP.hpp
new file mode 100644
index 0000000..5456479
--- /dev/null
+++ b/utility/meta/TMP.hpp
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_META_TMP_HPP_
+#define QUICKSTEP_UTILITY_META_TMP_HPP_
+
+#include "utility/meta/Common.hpp"
+#include "utility/meta/Dispatchers.hpp"
+#include "utility/meta/TransitiveClosure.hpp"
+#include "utility/meta/TypeList.hpp"
+
+#endif  // QUICKSTEP_UTILITY_META_TMP_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/utility/meta/TransitiveClosure.hpp
----------------------------------------------------------------------
diff --git a/utility/meta/TransitiveClosure.hpp b/utility/meta/TransitiveClosure.hpp
new file mode 100644
index 0000000..a5362bb
--- /dev/null
+++ b/utility/meta/TransitiveClosure.hpp
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_META_TRANSITIVE_CLOSURE_HPP_
+#define QUICKSTEP_UTILITY_META_TRANSITIVE_CLOSURE_HPP_
+
+#include "utility/meta/TypeList.hpp"
+
+namespace quickstep {
+namespace meta {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+template <typename Edges>
+struct TransitiveClosure;
+
+
+namespace internal {
+
+template <typename TL, typename Enable = void>
+struct EdgeMatcher {};
+
+template <typename TL>
+struct EdgeMatcher<
+    TL,
+    std::enable_if_t<std::is_same<typename TL::template at<0, 1>,
+                                  typename TL::template at<1, 0>>::value>> {
+  using type = TypeList<typename TL::template at<0, 0>,
+                        typename TL::template at<1, 1>>;
+};
+
+template <typename LeftEdges, typename RightEdges>
+struct JoinPath {
+  using type = typename LeftEdges::template cartesian_product<RightEdges>
+                                 ::template filtermap<EdgeMatcher>;
+};
+
+// Semi-naive
+template <typename Out, typename WorkSet, typename Edges, typename Enable = void>
+struct TransitiveClosureInner;
+
+template <typename Out, typename WorkSet, typename Edges>
+struct TransitiveClosureInner<Out, WorkSet, Edges,
+                              std::enable_if_t<WorkSet::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename WorkSet, typename Edges>
+struct TransitiveClosureInner<Out, WorkSet, Edges,
+                              std::enable_if_t<WorkSet::length != 0>>
+    : TransitiveClosureInner<typename Out::template append<WorkSet>,
+                             typename JoinPath<WorkSet, Edges>::type::template subtract<
+                                 typename Out::template append<WorkSet>>::template unique<>,
+                             Edges> {};
+
+template <typename Edge>
+struct TransitiveClosureInitializer {
+  using type = TypeList<TypeList<typename Edge::head, typename Edge::head>,
+                        TypeList<typename Edge::tail::head, typename Edge::tail::head>>;
+};
+
+template <typename Edges>
+using TransitiveClosureHelper =
+    typename TransitiveClosureInner<TypeList<>,
+                                    typename Edges::template flatmap<
+                                        TransitiveClosureInitializer>::template unique<>,
+                                    Edges>::type;
+
+}  // namespace internal
+
+template <typename Edges>
+struct TransitiveClosure : internal::TransitiveClosureHelper<Edges> {};
+
+/** @} */
+
+}  // namespace meta
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_META_TRANSITIVE_CLOSURE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/utility/meta/TypeList.hpp
----------------------------------------------------------------------
diff --git a/utility/meta/TypeList.hpp b/utility/meta/TypeList.hpp
new file mode 100644
index 0000000..fac3ce5
--- /dev/null
+++ b/utility/meta/TypeList.hpp
@@ -0,0 +1,124 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_META_TYPE_LIST_HPP_
+#define QUICKSTEP_UTILITY_META_TYPE_LIST_HPP_
+
+#include "utility/meta/Common.hpp"
+#include "utility/meta/TypeListMetaFunctions.hpp"
+
+namespace quickstep {
+namespace meta {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+template <typename ...Ts>
+class TypeList;
+
+template <typename ...Ts>
+class TypeListCommon {
+ private:
+  template <typename ...Tail> struct AppendHelper {
+    using type = TypeList<Ts..., Tail...>;
+  };
+
+ public:
+  static constexpr std::size_t length = sizeof...(Ts);
+
+  using type = TypeList<Ts...>;
+
+  template <template <typename ...> class Host>
+  using bind_to = Host<Ts...>;
+
+  template <std::size_t ...pos>
+  using at = typename internal::ElementAtImpl<
+      TypeList<Ts...>, TypeList<std::integral_constant<std::size_t, pos>...>>::type;
+
+  template <typename T>
+  using push_front = TypeList<T, Ts...>;
+
+  template <typename T>
+  using push_back = TypeList<Ts..., T>;
+
+  template <typename T>
+  using contains = EqualsAny<T, Ts...>;
+
+  template <typename ...DumbT>
+  using unique = typename internal::UniqueImpl<TypeList<>, TypeList<Ts...>, DumbT...>::type;
+
+  template <typename TL>
+  using append = typename TL::template bind_to<AppendHelper>::type;
+
+  template <typename TL>
+  using cartesian_product = typename internal::CartesianProductImpl<TypeList<Ts...>, TL>::type;
+
+  template <typename Subtrahend>
+  using subtract = typename internal::SubtractImpl<TypeList<>, TypeList<Ts...>, Subtrahend>::type;
+
+  template <template <typename ...> class Op>
+  using map = TypeList<typename Op<Ts>::type...>;
+
+  template <template <typename ...> class Op>
+  using flatmap = typename internal::FlatmapImpl<TypeList<>, TypeList<Ts...>, Op>::type;
+
+  template <template <typename ...> class Op>
+  using filter = typename internal::FilterImpl<TypeList<>, TypeList<Ts...>, Op>::type;
+
+  template <template <typename ...> class Op>
+  using filtermap = typename internal::FiltermapImpl<TypeList<>, TypeList<Ts...>, Op>::type;
+
+  template <typename ...DumbT>
+  using flatten_once = typename internal::FlattenOnceImpl<TypeList<>, TypeList<Ts...>, DumbT...>::type;
+
+  template <typename TL>
+  using zip = typename internal::ZipImpl<TypeList<>, TypeList<Ts...>, TL>::type;
+
+  template <typename TL, template <typename ...> class Op>
+  using zip_with = typename internal::ZipWithImpl<TypeList<>, TypeList<Ts...>, TL, Op>::type;
+
+  template <typename T>
+  using as_sequence = typename internal::AsSequenceImpl<T, Ts...>::type;
+};
+
+template <typename ...Ts>
+class TypeList : public TypeListCommon<Ts...> {
+ private:
+  template <typename Head, typename ...Tail>
+  struct HeadTailHelper {
+    using head = Head;
+    using tail = TypeList<Tail...>;
+  };
+
+ public:
+  using head = typename HeadTailHelper<Ts...>::head;
+  using tail = typename HeadTailHelper<Ts...>::tail;
+};
+
+template <>
+class TypeList<> : public TypeListCommon<> {
+};
+
+/** @} */
+
+}  // namespace meta
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_META_TYPE_LIST_HPP_