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/01 03:32:58 UTC

[1/3] incubator-quickstep git commit: Updates to types

Repository: incubator-quickstep
Updated Branches:
  refs/heads/new-op 959cd0d40 -> db611137e


Updates to types


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

Branch: refs/heads/new-op
Commit: 9db9fc869d808fba49274a3729113c6dcc80d147
Parents: 959cd0d
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Wed Mar 29 20:13:48 2017 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Wed Mar 29 20:13:48 2017 -0500

----------------------------------------------------------------------
 query_optimizer/LogicalGenerator.cpp            |   7 +-
 types/AsciiStringSuperType.hpp                  |  77 +++++++++++
 types/CMakeLists.txt                            |  12 +-
 types/CharType.cpp                              |  13 --
 types/CharType.hpp                              |  33 +----
 types/DateType.cpp                              |  19 ---
 types/DateType.hpp                              |  20 +--
 types/DatetimeIntervalType.cpp                  |  19 ---
 types/DatetimeIntervalType.hpp                  |  22 +---
 types/DatetimeType.cpp                          |  19 ---
 types/DatetimeType.hpp                          |  21 +--
 types/DoubleType.hpp                            |  12 +-
 types/FloatType.hpp                             |  12 +-
 types/IntType.hpp                               |  12 +-
 types/LongType.hpp                              |  12 +-
 types/NullType.hpp                              |  17 +--
 types/NumericSuperType.hpp                      |   7 +-
 types/Type.cpp                                  |  22 +++-
 types/Type.hpp                                  | 128 ++++++++-----------
 types/Type.proto                                |  18 +--
 types/TypeFactory.cpp                           | 116 ++++++-----------
 types/TypeFactory.hpp                           |  20 +--
 types/TypeID.hpp                                |   2 +-
 types/TypeUtil.hpp                              |  24 ++--
 types/VarCharType.cpp                           |  13 --
 types/VarCharType.hpp                           |  33 +----
 types/YearMonthIntervalType.cpp                 |  19 ---
 types/YearMonthIntervalType.hpp                 |  22 +---
 types/operations/OperationFactory.cpp           |   2 +-
 .../unary_operations/CMathUnaryOperations.hpp   |  11 +-
 .../unary_operations/CastOperation.cpp          |  12 +-
 31 files changed, 273 insertions(+), 503 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/query_optimizer/LogicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/LogicalGenerator.cpp b/query_optimizer/LogicalGenerator.cpp
index 940b728..01af806 100644
--- a/query_optimizer/LogicalGenerator.cpp
+++ b/query_optimizer/LogicalGenerator.cpp
@@ -51,10 +51,11 @@ L::LogicalPtr LogicalGenerator::generatePlan(
     const CatalogDatabase &catalog_database,
     const ParseStatement &parse_statement) {
   resolver::Resolver resolver(catalog_database, optimizer_context_);
-//  DVLOG(4) << "Parse tree:\n" << parse_statement.toString();
-  std::cerr << "Parse tree:\n" << parse_statement.toString();
+  DVLOG(4) << "Parse tree:\n" << parse_statement.toString();
+//  std::cerr << "Parse tree:\n" << parse_statement.toString();
   logical_plan_ = resolver.resolve(parse_statement);
-  std::cerr << "Initial logical plan:\n" << logical_plan_->toString();
+  DVLOG(4) << "Initial logical plan:\n" << logical_plan_->toString();
+//  std::cerr << "Initial logical plan:\n" << logical_plan_->toString();
 //  exit(0);
 
   optimizePlan();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/AsciiStringSuperType.hpp
----------------------------------------------------------------------
diff --git a/types/AsciiStringSuperType.hpp b/types/AsciiStringSuperType.hpp
new file mode 100644
index 0000000..82396b2
--- /dev/null
+++ b/types/AsciiStringSuperType.hpp
@@ -0,0 +1,77 @@
+/**
+ * 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_ASCII_STRING_SUPER_TYPE_HPP_
+#define QUICKSTEP_TYPES_ASCII_STRING_SUPER_TYPE_HPP_
+
+#include <cstddef>
+
+#include "types/Type.hpp"
+#include "types/TypeID.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+/**
+ * @brief A superclass for ASCII string types.
+ **/
+template <typename TypeClass, TypeID type_id, TypeStorageLayout layout>
+class AsciiStringSuperType : public TypeSynthesizer<TypeClass, type_id, true, layout> {
+ public:
+  bool isCoercibleFrom(const Type &original_type) const override {
+    if (original_type.isNullable() && !this->nullable_) {
+      return false;
+    }
+    return (original_type.getSuperTypeID() == Type::kAsciiString)
+           || (original_type.getTypeID() == kNullType);
+  }
+
+  /**
+   * @brief Get the character-length of this string type.
+   *
+   * @return The maximum length of a string of this type.
+   **/
+  inline std::size_t getStringLength() const {
+    return length_;
+  }
+
+ protected:
+  AsciiStringSuperType(const bool nullable,
+                       const std::size_t minimum_byte_length,
+                       const std::size_t maximum_byte_length,
+                       const std::size_t string_length)
+      : TypeSynthesizer<TypeClass, type_id, true, layout>(
+            Type::kAsciiString, type_id, nullable, minimum_byte_length, maximum_byte_length),
+        length_(string_length) {
+  }
+
+  const std::size_t length_;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(AsciiStringSuperType);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_ASCII_STRING_SUPER_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/CMakeLists.txt b/types/CMakeLists.txt
index 0c7ee07..96aea8c 100644
--- a/types/CMakeLists.txt
+++ b/types/CMakeLists.txt
@@ -32,6 +32,7 @@ QS_PROTOBUF_GENERATE_CPP(types_TypedValue_proto_srcs types_TypedValue_proto_hdrs
 QS_PROTOBUF_GENERATE_CPP(types_Type_proto_srcs types_Type_proto_hdrs Type.proto)
 
 # Declare micro-libs:
+add_library(quickstep_types_AsciiStringSuperType ../empty_src.cpp AsciiStringSuperType.hpp)
 add_library(quickstep_types_CharType CharType.cpp CharType.hpp)
 add_library(quickstep_types_DateOperatorOverloads ../empty_src.cpp DateOperatorOverloads.hpp)
 add_library(quickstep_types_DateType DateType.cpp DateType.hpp)
@@ -60,6 +61,9 @@ add_library(quickstep_types_VarCharType VarCharType.cpp VarCharType.hpp)
 add_library(quickstep_types_YearMonthIntervalType YearMonthIntervalType.cpp YearMonthIntervalType.hpp)
 
 # Link dependencies:
+target_link_libraries(quickstep_types_AsciiStringSuperType
+                      quickstep_types_Type
+                      quickstep_types_TypeID)
 target_link_libraries(quickstep_types_CharType
                       glog
                       quickstep_types_NullCoercibilityCheckMacro
@@ -78,7 +82,6 @@ target_link_libraries(quickstep_types_DateOperatorOverloads
 target_link_libraries(quickstep_types_DateType
                       glog
                       quickstep_types_DatetimeLit
-                      quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
@@ -88,7 +91,6 @@ target_link_libraries(quickstep_types_DatetimeIntervalType
                       glog
                       quickstep_types_IntervalLit
                       quickstep_types_IntervalParser
-                      quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
@@ -99,7 +101,6 @@ target_link_libraries(quickstep_types_DatetimeLit
 target_link_libraries(quickstep_types_DatetimeType
                       glog
                       quickstep_types_DatetimeLit
-                      quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
@@ -146,9 +147,6 @@ target_link_libraries(quickstep_types_LongType
                       quickstep_types_TypedValue
                       quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros)
-target_link_libraries(quickstep_types_NullCoercibilityCheckMacro
-                      quickstep_types_Type
-                      quickstep_types_TypeID)
 target_link_libraries(quickstep_types_NullType
                       glog
                       quickstep_types_Type
@@ -232,7 +230,6 @@ target_link_libraries(quickstep_types_VarCharType
 target_link_libraries(quickstep_types_YearMonthIntervalType
                       quickstep_types_IntervalLit
                       quickstep_types_IntervalParser
-                      quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
@@ -242,6 +239,7 @@ target_link_libraries(quickstep_types_YearMonthIntervalType
 # Module all-in-one library:
 add_library(quickstep_types ../empty_src.cpp TypesModule.hpp)
 target_link_libraries(quickstep_types
+                      quickstep_types_AsciiStringSuperType
                       quickstep_types_CharType
                       quickstep_types_DateOperatorOverloads
                       quickstep_types_DateType

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/CharType.cpp
----------------------------------------------------------------------
diff --git a/types/CharType.cpp b/types/CharType.cpp
index bc05711..c92d33e 100644
--- a/types/CharType.cpp
+++ b/types/CharType.cpp
@@ -60,19 +60,6 @@ const CharType& CharType::InstanceNullable(const std::size_t length) {
   return InstanceInternal<true>(length);
 }
 
-const CharType& CharType::InstanceFromProto(const serialization::Type &proto) {
-  return Instance(proto.GetExtension(serialization::CharType::length), proto.nullable());
-}
-
-serialization::Type CharType::getProto() const {
-  serialization::Type proto;
-  proto.mutable_type_id()->CopyFrom(TypeIDFactory::GetProto(type_id_));
-  proto.set_nullable(nullable_);
-
-  proto.SetExtension(serialization::CharType::length, length_);
-  return proto;
-}
-
 bool CharType::isSafelyCoercibleFrom(const Type &original_type) const {
   QUICKSTEP_NULL_COERCIBILITY_CHECK();
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/CharType.hpp
----------------------------------------------------------------------
diff --git a/types/CharType.hpp b/types/CharType.hpp
index e591953..fab0f61 100644
--- a/types/CharType.hpp
+++ b/types/CharType.hpp
@@ -24,8 +24,8 @@
 #include <cstdio>
 #include <string>
 
+#include "types/AsciiStringSuperType.hpp"
 #include "types/Type.hpp"
-#include "types/Type.pb.h"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
@@ -43,7 +43,7 @@ namespace quickstep {
  *       represented WITHOUT a null-terminator character. Any strings shorter
  *       than the maximum length will have a null-terminator.
  **/
-class CharType : public AsciiStringSuperType<kChar, kNonNativeInline> {
+class CharType : public AsciiStringSuperType<CharType, kChar, kNonNativeInline> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this Type
@@ -82,32 +82,6 @@ class CharType : public AsciiStringSuperType<kChar, kNonNativeInline> {
     }
   }
 
-  /**
-   * @brief Get a reference to the singleton instance of this Type described
-   *        by the given Protocol Buffer serialization.
-   *
-   * @param proto The serialized Protocol Buffer representation of the desired
-   *        CharType.
-   * @return A reference to the singleton instance of this Type for the given
-   *         Protocol Buffer.
-   **/
-  static const CharType& InstanceFromProto(const serialization::Type &proto);
-
-  /**
-   * @brief Generate a serialized Protocol Buffer representation of this Type.
-   *
-   * @return The serialized Protocol Buffer representation of this Type.
-   **/
-  serialization::Type getProto() const override;
-
-  const Type& getNullableVersion() const override {
-    return InstanceNullable(length_);
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable(length_);
-  }
-
   std::size_t estimateAverageByteLength() const override {
     return length_;
   }
@@ -134,7 +108,8 @@ class CharType : public AsciiStringSuperType<kChar, kNonNativeInline> {
 
  private:
   CharType(const std::size_t length, const bool nullable)
-      : AsciiStringSuperType<kChar, kNonNativeInline>(nullable, length, length, length) {
+      : AsciiStringSuperType<CharType, kChar, kNonNativeInline>(
+            nullable, length, length, length) {
   }
 
   template <bool nullable_internal>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/DateType.cpp
----------------------------------------------------------------------
diff --git a/types/DateType.cpp b/types/DateType.cpp
index 5bb982c..de1e554 100644
--- a/types/DateType.cpp
+++ b/types/DateType.cpp
@@ -30,7 +30,6 @@
 #include <string>
 
 #include "types/DatetimeLit.hpp"
-#include "types/NullCoercibilityCheckMacro.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
@@ -46,16 +45,6 @@ using std::snprintf;
 
 namespace quickstep {
 
-bool DateType::isCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kDate);
-}
-
-bool DateType::isSafelyCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kDate);
-}
-
 std::string DateType::printValueToString(const TypedValue &value) const {
   DCHECK(!value.isNull());
 
@@ -86,14 +75,6 @@ std::string DateType::printValueToString(const TypedValue &value) const {
   return std::string(datebuf);
 }
 
-void DateType::printValueToFile(const TypedValue &value,
-                                FILE *file,
-                                const int padding) const {
-  // We simply re-use the logic from printValueToString(), as trying to do
-  // padding on-the fly with so many different fields is too much of a hassle.
-  std::fprintf(file, "%*s", padding, printValueToString(value).c_str());
-}
-
 bool DateType::parseValueFromString(const std::string &value_string,
                                     TypedValue *value) const {
   std::int32_t year;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/DateType.hpp
----------------------------------------------------------------------
diff --git a/types/DateType.hpp b/types/DateType.hpp
index a39a28c..11e9908 100644
--- a/types/DateType.hpp
+++ b/types/DateType.hpp
@@ -40,7 +40,7 @@ class TypedValue;
 /**
  * @brief A type representing the date.
  **/
-class DateType : public TypeConcept<kDate, false, kNativeEmbedded, DateLit> {
+class DateType : public TypeSynthesizer<DateType, kDate, false, kNativeEmbedded, DateLit> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
@@ -77,32 +77,16 @@ class DateType : public TypeConcept<kDate, false, kNativeEmbedded, DateLit> {
     }
   }
 
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
   std::size_t estimateAverageByteLength() const override {
     return sizeof(DateLit);
   }
 
-  bool isCoercibleFrom(const Type &original_type) const override;
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override;
-
   int getPrintWidth() const override {
     return DateLit::kIsoChars;
   }
 
   std::string printValueToString(const TypedValue &value) const override;
 
-  void printValueToFile(const TypedValue &value,
-                        FILE *file,
-                        const int padding = 0) const override;
-
   /**
    * @note value_string is expected to be in (possibly extended) ISO-8601
    *       format. Extended ISO-8601 date format is "YYYY-MM-DD". YYYY is an
@@ -120,7 +104,7 @@ class DateType : public TypeConcept<kDate, false, kNativeEmbedded, DateLit> {
 
  private:
   explicit DateType(const bool nullable)
-      : TypeConcept<kDate, false, kNativeEmbedded, DateLit>(
+      : TypeSynthesizer<DateType, kDate, false, kNativeEmbedded, DateLit>(
              Type::kOther, kStaticTypeID, nullable,
              sizeof(DateLit), sizeof(DateLit)) {
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/DatetimeIntervalType.cpp
----------------------------------------------------------------------
diff --git a/types/DatetimeIntervalType.cpp b/types/DatetimeIntervalType.cpp
index 1eae03a..2c77f89 100644
--- a/types/DatetimeIntervalType.cpp
+++ b/types/DatetimeIntervalType.cpp
@@ -31,7 +31,6 @@
 
 #include "types/IntervalLit.hpp"
 #include "types/IntervalParser.hpp"
-#include "types/NullCoercibilityCheckMacro.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
@@ -47,16 +46,6 @@ using std::snprintf;
 
 namespace quickstep {
 
-bool DatetimeIntervalType::isCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kDatetimeInterval);
-}
-
-bool DatetimeIntervalType::isSafelyCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kDatetimeInterval);
-}
-
 std::string DatetimeIntervalType::printValueToString(const TypedValue &value) const {
   DCHECK(!value.isNull());
 
@@ -121,14 +110,6 @@ std::string DatetimeIntervalType::printValueToString(const TypedValue &value) co
   return std::string(interval_buf);
 }
 
-void DatetimeIntervalType::printValueToFile(const TypedValue &value,
-                                            FILE *file,
-                                            const int padding) const {
-  // We simply re-use the logic from printValueToString(), as trying to do
-  // padding on-the fly with so many different fields is too much of a hassle.
-  std::fprintf(file, "%*s", padding, printValueToString(value).c_str());
-}
-
 bool DatetimeIntervalType::parseValueFromString(const std::string &value_string,
                                                 TypedValue *value) const {
   // Try simple-format parse first.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/DatetimeIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeIntervalType.hpp b/types/DatetimeIntervalType.hpp
index 3b22471..ba05dce 100644
--- a/types/DatetimeIntervalType.hpp
+++ b/types/DatetimeIntervalType.hpp
@@ -41,7 +41,8 @@ namespace quickstep {
  * @brief A type representing the datetime interval.
  **/
 class DatetimeIntervalType :
-    public TypeConcept<kDatetimeInterval, false, kNativeEmbedded, DatetimeIntervalLit> {
+    public TypeSynthesizer<DatetimeIntervalType, kDatetimeInterval,
+                           false, kNativeEmbedded, DatetimeIntervalLit> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
@@ -78,32 +79,16 @@ class DatetimeIntervalType :
     }
   }
 
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
   std::size_t estimateAverageByteLength() const override {
     return sizeof(DatetimeIntervalLit);
   }
 
-  bool isCoercibleFrom(const Type &original_type) const override;
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override;
-
   int getPrintWidth() const override {
     return DatetimeIntervalLit::kPrintingChars;
   }
 
   std::string printValueToString(const TypedValue &value) const override;
 
-  void printValueToFile(const TypedValue &value,
-                        FILE *file,
-                        const int padding = 0) const override;
-
   TypedValue makeZeroValue() const override {
     return TypedValue(DatetimeIntervalLit{0});
   }
@@ -113,7 +98,8 @@ class DatetimeIntervalType :
 
  private:
   explicit DatetimeIntervalType(const bool nullable)
-      : TypeConcept<kDatetimeInterval, false, kNativeEmbedded, DatetimeIntervalLit>(
+      : TypeSynthesizer<DatetimeIntervalType, kDatetimeInterval,
+                        false, kNativeEmbedded, DatetimeIntervalLit>(
              Type::kOther, kStaticTypeID, nullable,
              sizeof(DatetimeIntervalLit), sizeof(DatetimeIntervalLit)) {
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/DatetimeType.cpp
----------------------------------------------------------------------
diff --git a/types/DatetimeType.cpp b/types/DatetimeType.cpp
index f54f318..723da61 100644
--- a/types/DatetimeType.cpp
+++ b/types/DatetimeType.cpp
@@ -33,7 +33,6 @@
 #include <string>
 
 #include "types/DatetimeLit.hpp"
-#include "types/NullCoercibilityCheckMacro.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
@@ -51,16 +50,6 @@ using std::snprintf;
 
 namespace quickstep {
 
-bool DatetimeType::isCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kDatetime);
-}
-
-bool DatetimeType::isSafelyCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kDatetime);
-}
-
 std::string DatetimeType::printValueToString(const TypedValue &value) const {
   DCHECK(!value.isNull());
 
@@ -114,14 +103,6 @@ std::string DatetimeType::printValueToString(const TypedValue &value) const {
   return std::string(datebuf);
 }
 
-void DatetimeType::printValueToFile(const TypedValue &value,
-                                    FILE *file,
-                                    const int padding) const {
-  // We simply re-use the logic from printValueToString(), as trying to do
-  // padding on-the fly with so many different fields is too much of a hassle.
-  std::fprintf(file, "%*s", padding, printValueToString(value).c_str());
-}
-
 bool DatetimeType::parseValueFromString(const std::string &value_string,
                                         TypedValue *value) const {
   int year;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/DatetimeType.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeType.hpp b/types/DatetimeType.hpp
index b6742f9..2189d22 100644
--- a/types/DatetimeType.hpp
+++ b/types/DatetimeType.hpp
@@ -40,7 +40,8 @@ class TypedValue;
 /**
  * @brief A type representing the datetime.
  **/
-class DatetimeType : public TypeConcept<kDatetime, false, kNativeEmbedded, DatetimeLit> {
+class DatetimeType
+    : public TypeSynthesizer<DatetimeType, kDatetime, false, kNativeEmbedded, DatetimeLit> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
@@ -77,32 +78,16 @@ class DatetimeType : public TypeConcept<kDatetime, false, kNativeEmbedded, Datet
     }
   }
 
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
   std::size_t estimateAverageByteLength() const override {
     return sizeof(DatetimeLit);
   }
 
-  bool isCoercibleFrom(const Type &original_type) const override;
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override;
-
   int getPrintWidth() const override {
     return DatetimeLit::kIsoChars;
   }
 
   std::string printValueToString(const TypedValue &value) const override;
 
-  void printValueToFile(const TypedValue &value,
-                        FILE *file,
-                        const int padding = 0) const override;
-
   /**
    * @note value_string is expected to be in (possibly extended) ISO-8601
    *       format. Extended ISO-8601 format is one of the following:
@@ -128,7 +113,7 @@ class DatetimeType : public TypeConcept<kDatetime, false, kNativeEmbedded, Datet
 
  private:
   explicit DatetimeType(const bool nullable)
-      : TypeConcept<kDatetime, false, kNativeEmbedded, quickstep::DatetimeLit>(
+      : TypeSynthesizer<DatetimeType, kDatetime, false, kNativeEmbedded, DatetimeLit>(
              Type::kOther, kStaticTypeID, nullable,
              sizeof(DatetimeLit), sizeof(DatetimeLit)) {
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/DoubleType.hpp
----------------------------------------------------------------------
diff --git a/types/DoubleType.hpp b/types/DoubleType.hpp
index 4d1dd7e..e8bb97a 100644
--- a/types/DoubleType.hpp
+++ b/types/DoubleType.hpp
@@ -40,7 +40,7 @@ class Type;
 /**
  * @brief A type representing a double-precision floating-point number.
  **/
-class DoubleType : public NumericSuperType<kDouble, double> {
+class DoubleType : public NumericSuperType<DoubleType, kDouble, double> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
@@ -77,14 +77,6 @@ class DoubleType : public NumericSuperType<kDouble, double> {
     }
   }
 
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
   bool isSafelyCoercibleFrom(const Type &original_type) const override;
 
   int getPrintWidth() const override {
@@ -120,7 +112,7 @@ class DoubleType : public NumericSuperType<kDouble, double> {
               // exponent never takes more than 3 base-10 digits to represent.
 
   explicit DoubleType(const bool nullable)
-      : NumericSuperType<kDouble, double>(nullable) {
+      : NumericSuperType<DoubleType, kDouble, double>(nullable) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(DoubleType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/FloatType.hpp
----------------------------------------------------------------------
diff --git a/types/FloatType.hpp b/types/FloatType.hpp
index ee9c45c..c1428b0 100644
--- a/types/FloatType.hpp
+++ b/types/FloatType.hpp
@@ -40,7 +40,7 @@ class Type;
 /**
  * @brief A type representing a single-precision floating-point number.
  **/
-class FloatType : public NumericSuperType<kFloat, float> {
+class FloatType : public NumericSuperType<FloatType, kFloat, float> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
@@ -77,14 +77,6 @@ class FloatType : public NumericSuperType<kFloat, float> {
     }
   }
 
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
   bool isSafelyCoercibleFrom(const Type &original_type) const override;
 
   int getPrintWidth() const override {
@@ -120,7 +112,7 @@ class FloatType : public NumericSuperType<kFloat, float> {
               // never takes more than 2 base-10 digits to represent.
 
   explicit FloatType(const bool nullable)
-      : NumericSuperType<kFloat, float>(nullable) {
+      : NumericSuperType<FloatType, kFloat, float>(nullable) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(FloatType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/IntType.hpp
----------------------------------------------------------------------
diff --git a/types/IntType.hpp b/types/IntType.hpp
index 25ce3c8..dda89bb 100644
--- a/types/IntType.hpp
+++ b/types/IntType.hpp
@@ -40,7 +40,7 @@ class Type;
 /**
  * @brief A type representing a 32-bit integer.
  **/
-class IntType : public NumericSuperType<kInt, int> {
+class IntType : public NumericSuperType<IntType, kInt, int> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
@@ -77,14 +77,6 @@ class IntType : public NumericSuperType<kInt, int> {
     }
   }
 
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
   bool isSafelyCoercibleFrom(const Type &original_type) const override;
 
   int getPrintWidth() const override {
@@ -107,7 +99,7 @@ class IntType : public NumericSuperType<kInt, int> {
 
  private:
   explicit IntType(const bool nullable)
-      : NumericSuperType<kInt, int>(nullable) {
+      : NumericSuperType<IntType, kInt, int>(nullable) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(IntType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/LongType.hpp
----------------------------------------------------------------------
diff --git a/types/LongType.hpp b/types/LongType.hpp
index 52244a7..f2814bf 100644
--- a/types/LongType.hpp
+++ b/types/LongType.hpp
@@ -41,7 +41,7 @@ class Type;
 /**
  * @brief A type representing a 64-bit integer.
  **/
-class LongType : public NumericSuperType<kLong, std::int64_t> {
+class LongType : public NumericSuperType<LongType, kLong, std::int64_t> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
@@ -78,14 +78,6 @@ class LongType : public NumericSuperType<kLong, std::int64_t> {
     }
   }
 
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
   bool isSafelyCoercibleFrom(const Type &original_type) const override;
 
   // Fully represented digits, single leading digit, and possible '-'
@@ -108,7 +100,7 @@ class LongType : public NumericSuperType<kLong, std::int64_t> {
 
  private:
   explicit LongType(const bool nullable)
-      : NumericSuperType<kLong, std::int64_t>(nullable) {
+      : NumericSuperType<LongType, kLong, std::int64_t>(nullable) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(LongType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/NullType.hpp
----------------------------------------------------------------------
diff --git a/types/NullType.hpp b/types/NullType.hpp
index 6c6dd06..0b9381c 100644
--- a/types/NullType.hpp
+++ b/types/NullType.hpp
@@ -48,7 +48,7 @@ class TypedValue;
  *       a particular operation may accept. It is also assumed that applying
  *       any operation to an argument of NullType always yields NULL values.
  **/
-class NullType : public TypeConcept<kNullType, false, kNativeEmbedded> {
+class NullType : public TypeSynthesizer<NullType, kNullType, false, kNativeEmbedded> {
  public:
   /**
    * @brief Get a reference to the nullable singleton instance of this Type.
@@ -62,12 +62,13 @@ class NullType : public TypeConcept<kNullType, false, kNativeEmbedded> {
     return instance;
   }
 
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    LOG(FATAL) << "Called NullType::getNonNullableVersion(), which is not allowed.";
+  static const NullType& Instance(const bool nullable) {
+    if (nullable) {
+      return InstanceNullable();
+    } else {
+      LOG(FATAL) << "Called NullType::Instance(nullable = true), "
+                 << "which is not allowed.";
+    }
   }
 
   std::size_t estimateAverageByteLength() const override {
@@ -105,7 +106,7 @@ class NullType : public TypeConcept<kNullType, false, kNativeEmbedded> {
   // NOTE(chasseur): NullType requires 0 bytes of inherent storage. It does,
   // however, require a bit in NULL bitmaps.
   NullType()
-      : TypeConcept<kNullType, false, kNativeEmbedded>(
+      : TypeSynthesizer<NullType, kNullType, false, kNativeEmbedded>(
             Type::kOther, kStaticTypeID, true, 0, 0) {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/NumericSuperType.hpp
----------------------------------------------------------------------
diff --git a/types/NumericSuperType.hpp b/types/NumericSuperType.hpp
index 77f028a..a3f1808 100644
--- a/types/NumericSuperType.hpp
+++ b/types/NumericSuperType.hpp
@@ -38,8 +38,9 @@ namespace quickstep {
  * @brief Templatized superclass for Numeric types. Contains code common to all
  *        Numeric types.
  **/
-template <TypeID type_id, typename CppType>
-class NumericSuperType : public TypeConcept<type_id, false, kNativeEmbedded, CppType> {
+template <typename TypeClass, TypeID type_id, typename CppType>
+class NumericSuperType
+    : public TypeSynthesizer<TypeClass, type_id, false, kNativeEmbedded, CppType> {
  public:
   std::size_t estimateAverageByteLength() const override {
     return sizeof(CppType);
@@ -56,7 +57,7 @@ class NumericSuperType : public TypeConcept<type_id, false, kNativeEmbedded, Cpp
 
  protected:
   explicit NumericSuperType(const bool nullable)
-      : TypeConcept<type_id, false, kNativeEmbedded, CppType>(
+      : TypeSynthesizer<TypeClass, type_id, false, kNativeEmbedded, CppType>(
             Type::kNumeric, type_id, nullable, sizeof(CppType), sizeof(CppType)) {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/Type.cpp
----------------------------------------------------------------------
diff --git a/types/Type.cpp b/types/Type.cpp
index 144f39a..af6503b 100644
--- a/types/Type.cpp
+++ b/types/Type.cpp
@@ -28,12 +28,24 @@
 
 namespace quickstep {
 
-serialization::Type Type::getProto() const {
-  serialization::Type proto;
-  proto.mutable_type_id()->CopyFrom(TypeIDFactory::GetProto(type_id_));
-  proto.set_nullable(nullable_);
+bool Type::isCoercibleFrom(const Type &original_type) const {
+  return isSafelyCoercibleFrom(original_type);
+}
+
+bool Type::isSafelyCoercibleFrom(const Type &original_type) const {
+  if (original_type.isNullable() && !this->nullable_) {
+    return false;
+  }
+  if (original_type.getTypeID() == kNullType) {
+    return true;
+  }
+  return (original_type.getTypeID() == type_id_);
+}
 
-  return proto;
+void Type::printValueToFile(const TypedValue &value,
+                            FILE *file,
+                            const int padding) const {
+  std::fprintf(file, "%*s", padding, printValueToString(value).c_str());
 }
 
 TypedValue Type::coerceValue(const TypedValue &original_value,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/Type.hpp
----------------------------------------------------------------------
diff --git a/types/Type.hpp b/types/Type.hpp
index 066b2b5..e293766 100644
--- a/types/Type.hpp
+++ b/types/Type.hpp
@@ -24,6 +24,7 @@
 #include <cstdint>
 #include <cstdio>
 #include <string>
+#include <type_traits>
 
 #include "types/Type.pb.h"
 #include "types/TypeID.hpp"
@@ -112,7 +113,7 @@ class Type {
    *
    * @return The serialized Protocol Buffer representation of this Type.
    **/
-  virtual serialization::Type getProto() const;
+  virtual serialization::Type getProto() const = 0;
 
   /**
    * @brief Determine what supertype this type belongs to.
@@ -154,16 +155,7 @@ class Type {
     TypeSignature sig;
     sig.id = type_id_;
     sig.nullable = nullable_;
-    switch (type_id_) {
-      case kChar:
-        sig.length = maximum_byte_length_;
-        break;
-      case kVarChar:
-        sig.length = maximum_byte_length_ - 1;
-        break;
-      default:
-        sig.length = 0;
-    }
+    sig.length = parameter_;
     return sig;
   }
 
@@ -255,7 +247,7 @@ class Type {
    * @param original_type The original Type for coercion to this Type.
    * @return true if coercion is supported, false otherwise.
    **/
-  virtual bool isCoercibleFrom(const Type &original_type) const = 0;
+  virtual bool isCoercibleFrom(const Type &original_type) const;
 
   /**
    * @brief Determine whether data items of another type can be coerced (used
@@ -277,7 +269,7 @@ class Type {
    * @param original_type The original Type for coercion to this Type.
    * @return true if coercion is supported, false otherwise.
    **/
-  virtual bool isSafelyCoercibleFrom(const Type &original_type) const = 0;
+  virtual bool isSafelyCoercibleFrom(const Type &original_type) const;
 
   /**
    * @brief Determine whether data items of this type are always guaranteed to
@@ -348,7 +340,7 @@ class Type {
    **/
   virtual void printValueToFile(const TypedValue &value,
                                 FILE *file,
-                                const int padding = 0) const = 0;
+                                const int padding = 0) const;
 
   /**
    * @brief Make a TypedValue of this Type.
@@ -453,10 +445,12 @@ class Type {
        const TypeID type_id,
        const bool nullable,
        const std::size_t minimum_byte_length,
-       const std::size_t maximum_byte_length)
+       const std::size_t maximum_byte_length,
+       const std::size_t parameter = 0)
       : super_type_id_(super_type_id),
         type_id_(type_id),
         nullable_(nullable),
+        parameter_(parameter),
         minimum_byte_length_(minimum_byte_length),
         maximum_byte_length_(maximum_byte_length) {
   }
@@ -464,6 +458,7 @@ class Type {
   const SuperTypeID super_type_id_;
   const TypeID type_id_;
   const bool nullable_;
+  const std::size_t parameter_;
   const std::size_t minimum_byte_length_;
   const std::size_t maximum_byte_length_;
 
@@ -471,11 +466,10 @@ class Type {
   DISALLOW_COPY_AND_ASSIGN(Type);
 };
 
-template <TypeID type_id,
-          bool parameterized,
-          TypeStorageLayout layout,
+template <typename TypeClass, TypeID type_id,
+          bool parameterized, TypeStorageLayout layout,
           typename CppType = void>
-class TypeConcept : public Type {
+class TypeSynthesizer : public Type {
  public:
   static constexpr TypeID kStaticTypeID = type_id;
   static constexpr bool kParameterized = parameterized;
@@ -483,73 +477,63 @@ class TypeConcept : public Type {
 
   typedef CppType cpptype;
 
- protected:
-  template <typename ...ArgTypes>
-  explicit TypeConcept(ArgTypes &&...args)
-      : Type(std::forward<ArgTypes>(args)...) {}
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(TypeConcept);
-};
+  serialization::Type getProto() const override {
+    serialization::Type proto;
 
+    proto.mutable_type_id()->CopyFrom(TypeIDFactory::GetProto(type_id_));
+    proto.set_nullable(nullable_);
 
-template <TypeID type_id,
-          bool parameterized,
-          TypeStorageLayout layout,
-          typename CppType>
-constexpr TypeID TypeConcept<type_id, parameterized, layout, CppType>::kStaticTypeID;
-
-template <TypeID type_id,
-          bool parameterized,
-          TypeStorageLayout layout,
-          typename CppType>
-constexpr bool TypeConcept<type_id, parameterized, layout, CppType>::kParameterized;
+    if (kParameterized) {
+      proto.set_length(parameter_);
+    }
 
-template <TypeID type_id,
-          bool parameterized,
-          TypeStorageLayout layout,
-          typename CppType>
-constexpr TypeStorageLayout TypeConcept<type_id, parameterized, layout, CppType>::kLayout;
+    return proto;
+  }
 
-/**
- * @brief A superclass for ASCII string types.
- **/
-template <TypeID type_id, TypeStorageLayout layout>
-class AsciiStringSuperType : public TypeConcept<type_id, true, layout> {
- public:
-  bool isCoercibleFrom(const Type &original_type) const override {
-    if (original_type.isNullable() && !this->nullable_) {
-      return false;
-    }
-    return (original_type.getSuperTypeID() == Type::kAsciiString)
-           || (original_type.getTypeID() == kNullType);
+  const Type& getNullableVersion() const override {
+    return getInstance<kParameterized>(true);
   }
 
-  /**
-   * @brief Get the character-length of this string type.
-   *
-   * @return The maximum length of a string of this type.
-   **/
-  inline std::size_t getStringLength() const {
-    return length_;
+  const Type& getNonNullableVersion() const override {
+    return getInstance<kParameterized>(false);
   }
 
  protected:
-  AsciiStringSuperType(const bool nullable,
-                       const std::size_t minimum_byte_length,
-                       const std::size_t maximum_byte_length,
-                       const std::size_t string_length)
-      : TypeConcept<type_id, true, layout>(
-            Type::kAsciiString, type_id, nullable, minimum_byte_length, maximum_byte_length),
-        length_(string_length) {
+  template <typename ...ArgTypes>
+  explicit TypeSynthesizer(ArgTypes &&...args)
+      : Type(std::forward<ArgTypes>(args)...) {}
+
+ private:
+  template <bool has_param>
+  inline const Type& getInstance(const bool nullable,
+                                 std::enable_if_t<has_param>* = 0) const {
+    return TypeClass::Instance(parameter_, nullable);
   }
 
-  const std::size_t length_;
+  template <bool has_param>
+  inline const Type& getInstance(const bool nullable,
+                                 std::enable_if_t<!has_param>* = 0) const {
+    return TypeClass::Instance(nullable);
+  }
 
- private:
-  DISALLOW_COPY_AND_ASSIGN(AsciiStringSuperType);
+  DISALLOW_COPY_AND_ASSIGN(TypeSynthesizer);
 };
 
+template <typename TypeClass, TypeID type_id,
+          bool parameterized, TypeStorageLayout layout, typename CppType>
+constexpr TypeID TypeSynthesizer<
+    TypeClass, type_id, parameterized, layout, CppType>::kStaticTypeID;
+
+template <typename TypeClass, TypeID type_id,
+          bool parameterized, TypeStorageLayout layout, typename CppType>
+constexpr bool TypeSynthesizer<
+    TypeClass, type_id, parameterized, layout, CppType>::kParameterized;
+
+template <typename TypeClass, TypeID type_id,
+          bool parameterized, TypeStorageLayout layout, typename CppType>
+constexpr TypeStorageLayout TypeSynthesizer<
+    TypeClass, type_id, parameterized, layout, CppType>::kLayout;
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/Type.proto
----------------------------------------------------------------------
diff --git a/types/Type.proto b/types/Type.proto
index d454923..ed8df36 100644
--- a/types/Type.proto
+++ b/types/Type.proto
@@ -26,22 +26,6 @@ message TypeID {
 message Type {
   required TypeID type_id = 1;
   required bool nullable = 2;
-
-  // The convention for extension numbering is that extensions for a particular
-  // TypeID should begin from (type_id + 1) * 32.
-  extensions 32 to max;
-}
-
-message CharType {
-  extend Type {
-    // Required when type_id == CHAR.
-    optional uint64 length = 160;
-  }
+  optional uint64 length = 3;
 }
 
-message VarCharType {
-  extend Type {
-    // Required when type_id == VAR_CHAR.
-    optional uint64 length = 192;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/TypeFactory.cpp
----------------------------------------------------------------------
diff --git a/types/TypeFactory.cpp b/types/TypeFactory.cpp
index df21d50..4a24cc6 100644
--- a/types/TypeFactory.cpp
+++ b/types/TypeFactory.cpp
@@ -34,6 +34,7 @@
 #include "types/Type.hpp"
 #include "types/Type.pb.h"
 #include "types/TypeID.hpp"
+#include "types/TypeUtil.hpp"
 #include "types/VarCharType.hpp"
 #include "types/YearMonthIntervalType.hpp"
 #include "utility/Macros.hpp"
@@ -42,46 +43,35 @@
 
 namespace quickstep {
 
+bool TypeFactory::TypeRequiresLengthParameter(const TypeID id) {
+  return TypeUtil::IsParameterized(id);
+}
+
 const Type& TypeFactory::GetType(const TypeID id,
                                  const bool nullable) {
-  switch (id) {
-    case kInt:
-      return IntType::Instance(nullable);
-    case kLong:
-      return LongType::Instance(nullable);
-    case kFloat:
-      return FloatType::Instance(nullable);
-    case kDouble:
-      return DoubleType::Instance(nullable);
-    case kDate:
-      return DateType::Instance(nullable);
-    case kDatetime:
-      return DatetimeType::Instance(nullable);
-    case kDatetimeInterval:
-      return DatetimeIntervalType::Instance(nullable);
-    case kYearMonthInterval:
-      return YearMonthIntervalType::Instance(nullable);
-    case kNullType:
-      DCHECK(nullable);
-      return NullType::InstanceNullable();
-    default:
-      FATAL_ERROR("Called TypeFactory::GetType() for a type which requires "
-                  " a length parameter without specifying one.");
-  }
+  DCHECK(!TypeRequiresLengthParameter(id))
+      << "Called TypeFactory::GetType() for a type which requires "
+      << " a length parameter without specifying one.";
+
+  return *InvokeOnTypeID<TypeIDSelectorNonParameterized>(
+      id,
+      [&](auto id) -> const Type* {  // NOLINT(build/c++11)
+    return &TypeClass<decltype(id)::value>::type::Instance(nullable);
+  });
 }
 
 const Type& TypeFactory::GetType(const TypeID id,
                                  const std::size_t length,
                                  const bool nullable) {
-  switch (id) {
-    case kChar:
-      return CharType::Instance(length, nullable);
-    case kVarChar:
-      return VarCharType::Instance(length, nullable);
-    default:
-      FATAL_ERROR("Provided a length parameter to TypeFactory::GetType() for "
-                  "a type which does not take one.");
-  }
+  DCHECK(TypeRequiresLengthParameter(id))
+      << "Provided a length parameter to TypeFactory::GetType() for "
+      << "a type which does not take one.";
+
+  return *InvokeOnTypeID<TypeIDSelectorParameterized>(
+      id,
+      [&](auto id) -> const Type* {  // NOLINT(build/c++11)
+    return &TypeClass<decltype(id)::value>::type::Instance(nullable, length);
+  });
 }
 
 bool TypeFactory::ProtoIsValid(const serialization::Type &proto) {
@@ -90,27 +80,18 @@ bool TypeFactory::ProtoIsValid(const serialization::Type &proto) {
     return false;
   }
 
-  // Check that the type_id is valid, and extensions if any.
+  // Check that the type_id is valid, and has length if parameterized.
   const TypeID type_id = TypeIDFactory::ReconstructFromProto(proto.type_id());
-  switch (type_id) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble:
-    case kDate:
-    case kDatetime:
-    case kDatetimeInterval:
-    case kYearMonthInterval:
-      return true;
-    case kChar:
-      return proto.HasExtension(serialization::CharType::length);
-    case kVarChar:
-      return proto.HasExtension(serialization::VarCharType::length);
-    case kNullType:
-      return proto.nullable();
-    default:
-      return false;
+
+  if (type_id == kNullType) {
+    return proto.nullable();
+  }
+
+  if (TypeRequiresLengthParameter(type_id)) {
+    return proto.has_length();
   }
+
+  return true;
 }
 
 const Type& TypeFactory::ReconstructFromProto(const serialization::Type &proto) {
@@ -119,32 +100,11 @@ const Type& TypeFactory::ReconstructFromProto(const serialization::Type &proto)
       << proto.DebugString();
 
   const TypeID type_id = TypeIDFactory::ReconstructFromProto(proto.type_id());
-  switch (type_id) {
-    case kInt:
-      return IntType::Instance(proto.nullable());
-    case kLong:
-      return LongType::Instance(proto.nullable());
-    case kFloat:
-      return FloatType::Instance(proto.nullable());
-    case kDouble:
-      return DoubleType::Instance(proto.nullable());
-    case kDate:
-      return DateType::Instance(proto.nullable());
-    case kDatetime:
-      return DatetimeType::Instance(proto.nullable());
-    case kDatetimeInterval:
-      return DatetimeIntervalType::Instance(proto.nullable());
-    case kYearMonthInterval:
-      return YearMonthIntervalType::Instance(proto.nullable());
-    case kChar:
-      return CharType::InstanceFromProto(proto);
-    case kVarChar:
-      return VarCharType::InstanceFromProto(proto);
-    case kNullType:
-      DCHECK(proto.nullable());
-      return NullType::InstanceNullable();
-    default:
-      FATAL_ERROR("Unrecognized TypeID in TypeFactory::ReconstructFromProto");
+
+  if (TypeRequiresLengthParameter(type_id)) {
+    return GetType(type_id, proto.length(), proto.nullable());
+  } else {
+    return GetType(type_id, proto.nullable());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/TypeFactory.hpp
----------------------------------------------------------------------
diff --git a/types/TypeFactory.hpp b/types/TypeFactory.hpp
index 742348e..89ff497 100644
--- a/types/TypeFactory.hpp
+++ b/types/TypeFactory.hpp
@@ -48,25 +48,7 @@ class TypeFactory {
    * @param id The id of the desired Type.
    * @return Whether a length must be specified for Types of the given id.
    **/
-  static bool TypeRequiresLengthParameter(const TypeID id) {
-    switch (id) {
-      case kInt:
-      case kLong:
-      case kFloat:
-      case kDouble:
-      case kDate:
-      case kDatetime:
-      case kDatetimeInterval:
-      case kYearMonthInterval:
-      case kNullType:
-        return false;
-      case kChar:
-      case kVarChar:
-        return true;
-      default:
-        FATAL_ERROR("Unrecognized TypeID in TypeFactory::TypeRequiresLengthParameter");
-    }
-  }
+  static bool TypeRequiresLengthParameter(const TypeID id);
 
   /**
    * @brief Factory method to get a Type by its TypeID.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/TypeID.hpp
----------------------------------------------------------------------
diff --git a/types/TypeID.hpp b/types/TypeID.hpp
index 26731b9..e5f023e 100644
--- a/types/TypeID.hpp
+++ b/types/TypeID.hpp
@@ -49,7 +49,7 @@ enum TypeID : int {
 };
 
 enum TypeStorageLayout {
-  kNativeEmbedded,
+  kNativeEmbedded = 0,
   kNativeInline,
   kNonNativeInline,
   kOutOfLine

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/TypeUtil.hpp
----------------------------------------------------------------------
diff --git a/types/TypeUtil.hpp b/types/TypeUtil.hpp
index 6c58ef1..d2183e8 100644
--- a/types/TypeUtil.hpp
+++ b/types/TypeUtil.hpp
@@ -44,10 +44,10 @@
 namespace quickstep {
 
 template <TypeID type_id>
-struct TypeGenerator {};
+struct TypeClass {};
 
 #define REGISTER_TYPE(T) \
-  template <> struct TypeGenerator<T::kStaticTypeID> { typedef T type; };
+  template <> struct TypeClass<T::kStaticTypeID> { typedef T type; };
 
 REGISTER_TYPE(IntType);
 REGISTER_TYPE(LongType);
@@ -69,11 +69,14 @@ template <bool require_parameterized>
 struct TypeIDSelectorParameterizedHelper {
   template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
   struct Implementation {
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wreturn-type"
     inline static auto Invoke(const FunctorT &functor)
         -> decltype(functor(TypeIDConstant())) {
-      DLOG(FATAL) << "Unexpected TypeID: "
-                  << kTypeNames[static_cast<int>(TypeIDConstant::value)];
+      LOG(FATAL) << "Unexpected TypeID: "
+                 << kTypeNames[static_cast<int>(TypeIDConstant::value)];
     }
+#pragma GCC diagnostic pop
   };
 };
 
@@ -81,7 +84,7 @@ 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
+    std::enable_if_t<TypeClass<TypeIDConstant::value>::type::kParameterized
                          ^ require_non_parameterized>> {
   inline static auto Invoke(const FunctorT &functor) {
     return functor(TypeIDConstant());
@@ -109,11 +112,14 @@ template <TypeID ...candidates>
 struct TypeIDSelectorEqualsAny {
   template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
   struct Implementation {
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wreturn-type"
     inline static auto Invoke(const FunctorT &functor)
         -> decltype(functor(TypeIDConstant())) {
-      DLOG(FATAL) << "Unexpected TypeID: "
-                  << kTypeNames[static_cast<int>(TypeIDConstant::value)];
+      LOG(FATAL) << "Unexpected TypeID: "
+                 << kTypeNames[static_cast<int>(TypeIDConstant::value)];
     }
+#pragma GCC diagnostic pop
   };
 };
 
@@ -150,7 +156,7 @@ inline auto InvokeOnTypeID(const TypeID type_id,
     REGISTER_TYPE_ID(kVarChar);
     REGISTER_TYPE_ID(kNullType);
     default:
-      FATAL_ERROR("Unrecognized TypeID in InvokeOnTypeID()");
+      LOG(FATAL) << "Unrecognized TypeID in InvokeOnTypeID()";
   }
 
 #undef REGISTER_TYPE_ID
@@ -162,7 +168,7 @@ class TypeUtil {
     return InvokeOnTypeID(
         type_id,
         [&](auto tid) -> bool {  // NOLINT(build/c++11)
-      return TypeGenerator<decltype(tid)::value>::type::kParameterized;
+      return TypeClass<decltype(tid)::value>::type::kParameterized;
     });
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/VarCharType.cpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.cpp b/types/VarCharType.cpp
index 05162cf..61fd88e 100644
--- a/types/VarCharType.cpp
+++ b/types/VarCharType.cpp
@@ -60,19 +60,6 @@ const VarCharType& VarCharType::InstanceNullable(const std::size_t length) {
   return InstanceInternal<true>(length);
 }
 
-const VarCharType& VarCharType::InstanceFromProto(const serialization::Type &proto) {
-  return Instance(proto.GetExtension(serialization::VarCharType::length), proto.nullable());
-}
-
-serialization::Type VarCharType::getProto() const {
-  serialization::Type proto;
-  proto.mutable_type_id()->CopyFrom(TypeIDFactory::GetProto(type_id_));
-  proto.set_nullable(nullable_);
-
-  proto.SetExtension(serialization::VarCharType::length, length_);
-  return proto;
-}
-
 size_t VarCharType::estimateAverageByteLength() const {
   if (length_ > 160) {
     return 80;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/VarCharType.hpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.hpp b/types/VarCharType.hpp
index af3ab64..5ca1bc1 100644
--- a/types/VarCharType.hpp
+++ b/types/VarCharType.hpp
@@ -24,8 +24,8 @@
 #include <cstdio>
 #include <string>
 
+#include "types/AsciiStringSuperType.hpp"
 #include "types/Type.hpp"
-#include "types/Type.pb.h"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.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<kVarChar, kOutOfLine> {
+class VarCharType : public AsciiStringSuperType<VarCharType, kVarChar, kOutOfLine> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this Type
@@ -83,32 +83,6 @@ class VarCharType : public AsciiStringSuperType<kVarChar, kOutOfLine> {
   }
 
   /**
-   * @brief Get a reference to the singleton instance of this Type described
-   *        by the given Protocol Buffer serialization.
-   *
-   * @param type The serialized Protocol Buffer representation of the desired
-   *        VarCharType.
-   * @return A reference to the singleton instance of this Type for the given
-   *         Protocol Buffer.
-   **/
-  static const VarCharType& InstanceFromProto(const serialization::Type &type);
-
-  /**
-   * @brief Generate a serialized Protocol Buffer representation of this Type.
-   *
-   * @return The serialized Protocol Buffer representation of this Type.
-   **/
-  serialization::Type getProto() const override;
-
-  const Type& getNullableVersion() const override {
-    return InstanceNullable(length_);
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable(length_);
-  }
-
-  /**
    * @note Includes an extra byte for a terminating null character.
    **/
   std::size_t estimateAverageByteLength() const override;
@@ -137,7 +111,8 @@ class VarCharType : public AsciiStringSuperType<kVarChar, kOutOfLine> {
 
  private:
   VarCharType(const std::size_t length, const bool nullable)
-      : AsciiStringSuperType<kVarChar, kOutOfLine>(nullable, 1, length + 1, length) {
+      : AsciiStringSuperType<VarCharType, kVarChar, kOutOfLine>(
+            nullable, 1, length + 1, length) {
   }
 
   template <bool nullable_internal>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/YearMonthIntervalType.cpp
----------------------------------------------------------------------
diff --git a/types/YearMonthIntervalType.cpp b/types/YearMonthIntervalType.cpp
index 3c15a91..d656fca 100644
--- a/types/YearMonthIntervalType.cpp
+++ b/types/YearMonthIntervalType.cpp
@@ -30,7 +30,6 @@
 
 #include "types/IntervalLit.hpp"
 #include "types/IntervalParser.hpp"
-#include "types/NullCoercibilityCheckMacro.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
@@ -46,16 +45,6 @@ using std::snprintf;
 
 namespace quickstep {
 
-bool YearMonthIntervalType::isCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kYearMonthInterval);
-}
-
-bool YearMonthIntervalType::isSafelyCoercibleFrom(const Type &original_type) const {
-  QUICKSTEP_NULL_COERCIBILITY_CHECK();
-  return (original_type.getTypeID() == kYearMonthInterval);
-}
-
 std::string YearMonthIntervalType::printValueToString(const TypedValue &value) const {
   DCHECK(!value.isNull());
 
@@ -127,14 +116,6 @@ std::string YearMonthIntervalType::printValueToString(const TypedValue &value) c
   return std::string(interval_buf);
 }
 
-void YearMonthIntervalType::printValueToFile(const TypedValue &value,
-                                             FILE *file,
-                                             const int padding) const {
-  // We simply re-use the logic from printValueToString(), as trying to do
-  // padding on-the fly with so many different fields is too much of a hassle.
-  std::fprintf(file, "%*s", static_cast<int>(padding), printValueToString(value).c_str());
-}
-
 bool YearMonthIntervalType::parseValueFromString(const std::string &value_string,
                                                  TypedValue *value) const {
   // Try simple-format parse first.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/YearMonthIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/YearMonthIntervalType.hpp b/types/YearMonthIntervalType.hpp
index f768046..a13b05f 100644
--- a/types/YearMonthIntervalType.hpp
+++ b/types/YearMonthIntervalType.hpp
@@ -40,7 +40,8 @@ namespace quickstep {
  * @brief A type representing the year-month interval.
  **/
 class YearMonthIntervalType :
-    public TypeConcept<kYearMonthInterval, false, kNativeEmbedded, YearMonthIntervalLit> {
+    public TypeSynthesizer<YearMonthIntervalType, kYearMonthInterval,
+                           false, kNativeEmbedded, YearMonthIntervalLit> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
@@ -77,32 +78,16 @@ class YearMonthIntervalType :
     }
   }
 
-  const Type& getNullableVersion() const override {
-    return InstanceNullable();
-  }
-
-  const Type& getNonNullableVersion() const override {
-    return InstanceNonNullable();
-  }
-
   std::size_t estimateAverageByteLength() const override {
     return sizeof(YearMonthIntervalLit);
   }
 
-  bool isCoercibleFrom(const Type &original_type) const override;
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override;
-
   int getPrintWidth() const override {
     return YearMonthIntervalLit::kPrintingChars;
   }
 
   std::string printValueToString(const TypedValue &value) const override;
 
-  void printValueToFile(const TypedValue &value,
-                        FILE *file,
-                        const int padding = 0) const override;
-
   TypedValue makeZeroValue() const override {
     return TypedValue(YearMonthIntervalLit{0});
   }
@@ -112,7 +97,8 @@ class YearMonthIntervalType :
 
  private:
   explicit YearMonthIntervalType(const bool nullable)
-      : TypeConcept<kYearMonthInterval, false, kNativeEmbedded, YearMonthIntervalLit>(
+      : TypeSynthesizer<YearMonthIntervalType, kYearMonthInterval,
+                        false, kNativeEmbedded, YearMonthIntervalLit>(
             Type::kOther, kStaticTypeID, nullable,
             sizeof(YearMonthIntervalLit), sizeof(YearMonthIntervalLit)) {
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/operations/OperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationFactory.cpp b/types/operations/OperationFactory.cpp
index 5d4295a..521e3ad 100644
--- a/types/operations/OperationFactory.cpp
+++ b/types/operations/OperationFactory.cpp
@@ -226,7 +226,7 @@ OperationFactory::ResolveStatus OperationFactory::resolveOperationWithPartialTyp
         coerced_static_args.emplace_back(arg_value);
       } else {
         const Type *expected_type = nullptr;
-        if (TypeUtil::IsParameterized(expected_type_id)) {
+        if (TypeFactory::TypeRequiresLengthParameter(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(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/operations/unary_operations/CMathUnaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CMathUnaryOperations.hpp b/types/operations/unary_operations/CMathUnaryOperations.hpp
index a4cb201..8e54c03 100644
--- a/types/operations/unary_operations/CMathUnaryOperations.hpp
+++ b/types/operations/unary_operations/CMathUnaryOperations.hpp
@@ -60,6 +60,13 @@ template <typename ArgumentT, typename ResultT,
 using CMathUnaryFunctor =
     typename CMathUnaryFunctorWrapper<ArgumentT, ResultT, f, FunctorNameT>::type;
 
+inline std::int64_t CMathRound(const float arg) {
+  return std::llround(arg);
+}
+inline std::int64_t CMathRound(const double arg) {
+  return std::llround(arg);
+}
+
 using CMathUnaryFunctorPack = FunctorPack<
 // abs
     CMathUnaryFunctor<IntType, IntType,
@@ -97,9 +104,9 @@ using CMathUnaryFunctorPack = FunctorPack<
                       std::floor, StringLiteral<'f','l','o','o','r'>>,
 // round
     CMathUnaryFunctor<FloatType, LongType,
-                      std::llround, StringLiteral<'r','o','u','n','d'>>,
+                      CMathRound, StringLiteral<'r','o','u','n','d'>>,
     CMathUnaryFunctor<DoubleType, LongType,
-                      std::llround, StringLiteral<'r','o','u','n','d'>>
+                      CMathRound, StringLiteral<'r','o','u','n','d'>>
 >;
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9db9fc86/types/operations/unary_operations/CastOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CastOperation.cpp b/types/operations/unary_operations/CastOperation.cpp
index b240b23..21378e4 100644
--- a/types/operations/unary_operations/CastOperation.cpp
+++ b/types/operations/unary_operations/CastOperation.cpp
@@ -195,7 +195,7 @@ UncheckedUnaryOperator* CastOperation::makeUncheckedUnaryOperator(
     return InvokeOnTypeID<TypeIDSelectorEqualsAny<kInt, kLong, kFloat, kDouble>>(
         argument_type_id,
         [&](auto arg_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
-      using ArgumentT = typename TypeGenerator<decltype(arg_tid)::value>::type;
+      using ArgumentT = typename TypeClass<decltype(arg_tid)::value>::type;
 
       switch (result_type_id) {
         case kInt:  // Fall through
@@ -205,7 +205,7 @@ UncheckedUnaryOperator* CastOperation::makeUncheckedUnaryOperator(
           return InvokeOnTypeID<TypeIDSelectorEqualsAny<kInt, kLong, kFloat, kDouble>>(
               result_type_id,
               [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
-            using ResultT = typename TypeGenerator<decltype(result_tid)::value>::type;
+            using ResultT = typename TypeClass<decltype(result_tid)::value>::type;
 
             return new UncheckedUnaryOperatorWrapperCodegen<
                 NumericCastToNumericFunctor<ArgumentT, ResultT>>(type, *result_type);
@@ -216,7 +216,7 @@ UncheckedUnaryOperator* CastOperation::makeUncheckedUnaryOperator(
           return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
               result_type_id,
               [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
-            using ResultT = typename TypeGenerator<decltype(result_tid)::value>::type;
+            using ResultT = typename TypeClass<decltype(result_tid)::value>::type;
 
             return new UncheckedUnaryOperatorWrapperCodegen<
                  CastToAsciiStringFunctor<ArgumentT, ResultT>>(
@@ -235,7 +235,7 @@ UncheckedUnaryOperator* CastOperation::makeUncheckedUnaryOperator(
     return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
         argument_type_id,
         [&](auto arg_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
-      using ArgumentT = typename TypeGenerator<decltype(arg_tid)::value>::type;
+      using ArgumentT = typename TypeClass<decltype(arg_tid)::value>::type;
 
       switch (result_type_id) {
         case kInt:  // Fall through
@@ -245,7 +245,7 @@ UncheckedUnaryOperator* CastOperation::makeUncheckedUnaryOperator(
           return InvokeOnTypeID<TypeIDSelectorEqualsAny<kInt, kLong, kFloat, kDouble>>(
               result_type_id,
               [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
-            using ResultT = typename TypeGenerator<decltype(result_tid)::value>::type;
+            using ResultT = typename TypeClass<decltype(result_tid)::value>::type;
 
             return new UncheckedUnaryOperatorWrapperCodegen<
                 AsciiStringCastToNumericFunctor<
@@ -260,7 +260,7 @@ UncheckedUnaryOperator* CastOperation::makeUncheckedUnaryOperator(
           return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
               result_type_id,
               [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
-            using ResultT = typename TypeGenerator<decltype(result_tid)::value>::type;
+            using ResultT = typename TypeClass<decltype(result_tid)::value>::type;
 
             return new UncheckedUnaryOperatorWrapperCodegen<
                  AsciiStringCastToAsciiStringFunctor<ArgumentT, ResultT>>(


[2/3] incubator-quickstep git commit: Initial type system

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/utility/TemplateUtil.hpp
----------------------------------------------------------------------
diff --git a/utility/TemplateUtil.hpp b/utility/TemplateUtil.hpp
index 40ebbcd..869373f 100644
--- a/utility/TemplateUtil.hpp
+++ b/utility/TemplateUtil.hpp
@@ -58,6 +58,19 @@ struct MakeSequence<0, S...> {
   typedef Sequence<S...> type;
 };
 
+
+template <typename T, typename EnableT = void>
+struct IsTypeTrait {
+  static constexpr bool value = false;
+};
+
+template <typename T>
+struct IsTypeTrait<T, std::enable_if_t<
+    std::is_same<typename T::type, typename T::type>::value>> {
+  static constexpr bool value = true;
+};
+
+
 template<class...> struct Disjunction : std::false_type {};
 template<class B1> struct Disjunction<B1> : B1 {};
 template<class B1, class... Bn>
@@ -70,19 +83,22 @@ struct EqualsAny {
       Disjunction<std::is_same<check, cases>...>::value;
 };
 
-namespace template_util_inner {
+template <char ...c>
+struct StringLiteral {
+  inline static std::string ToString() {
+    return std::string({c...});
+  }
+};
 
-/**
- * @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))...);
-}
+template <typename LeftT, typename RightT>
+struct PairSelectorLeft {
+  typedef LeftT type;
+};
+
+template <typename LeftT, typename RightT>
+struct PairSelectorRight {
+  typedef RightT type;
+};
 
 /**
  * @brief Invoke the functor with the compile-time bool values wrapped as
@@ -118,98 +134,82 @@ inline auto InvokeOnBoolsInner(TupleT &&args, Sequence<i...> &&indices) {
                             std::get<i>(std::forward<TupleT>(args))...);
 }
 
-}  // namespace template_util_inner
+template <typename Out, typename Rest,
+          template <typename> class Op, typename EnableT = void>
+struct MapInner;
 
-/**
- * @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 MakeSequence<n_args>::type());
-}
+template <typename Out, typename Rest,
+          template <typename> class Op, typename EnableT = void>
+struct FlatMapInner;
 
-/**
- * @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...);
-  }
-}
+template <typename Out, typename Rest,
+          template <typename> class Op, typename EnableT = void>
+struct FilterInner;
+
+template <typename Out, typename Rest,
+          template <typename> class Op, typename EnableT = void>
+struct FilterMapInner;
+
+template <typename Out, typename Rest, typename EnableT = void>
+struct UniqueInner;
+
+template <typename Out, typename Rest, typename ...DumbT>
+struct UniqueInnerHelper {
+  using type = typename UniqueInner<Out, Rest>::type;
+  static constexpr bool kDumbSize = sizeof...(DumbT);
+};
+
+template <typename Out, typename Rest,
+          typename Subtrahend, typename EnableT = void>
+struct SubtractInner;
+
+template <typename ...Ts>
+class TypeList;
+
+template <typename ...Ts>
+class TypeListCommon {
+ private:
+  template <typename ...Tail>
+  struct AppendInner {
+    using type = TypeList<Ts..., Tail...>;
+  };
+
+ public:
+  static constexpr std::size_t length = sizeof...(Ts);
+
+  template <template <typename ...> class Target>
+  using bind = Target<Ts...>;
+
+  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 TL>
+  using append = typename TL::template bind<AppendInner>::type;
+
+  template <template <typename> class Op>
+  using map = typename MapInner<TypeList<>, TypeList<Ts...>, Op>::type;
+
+  template <template <typename> class Op>
+  using flatmap = typename FlatMapInner<TypeList<>, TypeList<Ts...>, Op>::type;
+
+  template <template <typename> class Op>
+  using filter = typename FilterInner<TypeList<>, TypeList<Ts...>, Op>::type;
+
+  template <template <typename> class Op>
+  using filtermap = typename FilterMapInner<TypeList<>, TypeList<Ts...>, Op>::type;
+
+  template <typename ...DumbT>
+  using unique = typename UniqueInnerHelper<TypeList<>, TypeList<Ts...>, DumbT...>::type;
+
+  template <typename Subtrahend>
+  using subtract = typename SubtractInner<TypeList<>, TypeList<Ts...>, Subtrahend>::type;
+};
 
 /**
  * @brief A helper function for bool branched template specialization.
@@ -235,29 +235,178 @@ inline ReturnT* CreateBoolInstantiatedInstance(Tuple &&args,
 template <typename ...ArgTypes>
 inline auto InvokeOnBools(ArgTypes ...args) {
   constexpr std::size_t last = sizeof...(args) - 1;
-  return template_util_inner::InvokeOnBoolsInner<last>(
+  return InvokeOnBoolsInner<last>(
       std::forward_as_tuple(args...),
       typename MakeSequence<last>::type());
 }
 
+template <typename ...Ts>
+class TypeList : public TypeListCommon<Ts...> {
+ private:
+  template <typename Head, typename ...Tail>
+  struct HeadTailInner {
+    using head = Head;
+    using tail = TypeList<Tail...>;
+  };
 
-template <char ...c>
-struct StringLiteral {
-  inline static std::string ToString() {
-    return std::string({c...});
-  }
+ public:
+  using head = typename HeadTailInner<Ts...>::head;
+  using tail = typename HeadTailInner<Ts...>::tail;
 };
 
-template <typename LeftT, typename RightT>
-struct PairSelectorLeft {
-  typedef LeftT type;
+template <>
+class TypeList<> : public TypeListCommon<> {
 };
 
-template <typename LeftT, typename RightT>
-struct PairSelectorRight {
-  typedef RightT type;
+
+template <typename Out, typename Rest, template <typename> class Op>
+struct MapInner<Out, Rest, Op,
+                std::enable_if_t<Rest::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename Rest, template <typename> class Op>
+struct MapInner<Out, Rest, Op,
+                std::enable_if_t<Rest::length != 0>>
+    : MapInner<typename Out::template push_back<typename Op<typename Rest::head>::type>,
+               typename Rest::tail, Op> {};
+
+template <typename Out, typename Rest, template <typename> class Op>
+struct FlatMapInner<Out, Rest, Op,
+                    std::enable_if_t<Rest::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename Rest, template <typename> class Op>
+struct FlatMapInner<Out, Rest, Op,
+                    std::enable_if_t<Rest::length != 0>>
+    : FlatMapInner<typename Out::template append<typename Op<typename Rest::head>::type>,
+                   typename Rest::tail, Op> {};
+
+template <typename Out, typename Rest, template <typename> class Op>
+struct FilterInner<Out, Rest, Op,
+                   std::enable_if_t<Rest::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename Rest, template <typename> class Op>
+struct FilterInner<Out, Rest, Op,
+                   std::enable_if_t<Op<typename Rest::head>::value>>
+    : FilterInner<typename Out::template push_back<typename Rest::head>,
+                  typename Rest::tail, Op> {};
+
+template <typename Out, typename Rest, template <typename> class Op>
+struct FilterInner<Out, Rest, Op,
+                   std::enable_if_t<!Op<typename Rest::head>::value>>
+    : FilterInner<Out, typename Rest::tail, Op> {};
+
+template <typename Out, typename Rest, template <typename> class Op>
+struct FilterMapInner<Out, Rest, Op,
+                      std::enable_if_t<Rest::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename Rest, template <typename> class Op>
+struct FilterMapInner<Out, Rest, Op,
+                      std::enable_if_t<Rest::length != 0 &&
+                                       IsTypeTrait<Op<typename Rest::head>>::value>>
+    : FilterMapInner<typename Out::template push_back<typename Op<typename Rest::head>::type>,
+                     typename Rest::tail, Op> {};
+
+template <typename Out, typename Rest, template <typename> class Op>
+struct FilterMapInner<Out, Rest, Op,
+                      std::enable_if_t<Rest::length != 0 &&
+                                       !IsTypeTrait<Op<typename Rest::head>>::value>>
+    : FilterMapInner<Out, typename Rest::tail, Op> {};
+
+template <typename Out, typename Rest>
+struct UniqueInner<Out, Rest,
+                   std::enable_if_t<Rest::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename Rest>
+struct UniqueInner<Out, Rest,
+                   std::enable_if_t<Out::template contains<typename Rest::head>::value>>
+    : UniqueInner<Out, typename Rest::tail> {};
+
+template <typename Out, typename Rest>
+struct UniqueInner<Out, Rest,
+                   std::enable_if_t<!Out::template contains<typename Rest::head>::value>>
+    : UniqueInner<typename Out::template push_back<typename Rest::head>,
+                  typename Rest::tail> {};
+
+template <typename Out, typename Rest, typename Subtrahend>
+struct SubtractInner<Out, Rest, Subtrahend,
+                     std::enable_if_t<Rest::length == 0>> {
+  using type = Out;
+};
+
+template <typename Out, typename Rest, typename Subtrahend>
+struct SubtractInner<Out, Rest, Subtrahend,
+                     std::enable_if_t<Subtrahend::template contains<
+                         typename Rest::head>::value>>
+    : SubtractInner<Out, typename Rest::tail, Subtrahend> {};
+
+template <typename Out, typename Rest, typename Subtrahend>
+struct SubtractInner<Out, Rest, Subtrahend,
+                     std::enable_if_t<!Subtrahend::template contains<
+                         typename Rest::head>::value>>
+    : SubtractInner<typename Out::template push_back<typename Rest::head>,
+                    typename Rest::tail, Subtrahend> {};
+
+template <typename LeftEdge, typename RightEdge, typename EnableT = void>
+struct EdgeMatcher {};
+
+template <typename LeftEdge, typename RightEdge>
+struct EdgeMatcher<LeftEdge, RightEdge,
+                   std::enable_if_t<std::is_same<typename LeftEdge::tail::head,
+                                                 typename RightEdge::head>::value>> {
+  using type = TypeList<typename LeftEdge::head, typename RightEdge::tail::head>;
+};
+
+template <typename LeftEdges, typename RightEdges>
+struct JoinPath {
+  template <typename LeftEdge>
+  struct JoinPathLeftHelper {
+    template <typename RightEdge>
+    struct JoinPathRightHelper : EdgeMatcher<LeftEdge, RightEdge> {};
+
+    using type = typename RightEdges::template filtermap<JoinPathRightHelper>;
+  };
+  using type = typename LeftEdges::template flatmap<JoinPathLeftHelper>;
+};
+
+// Semi-naive
+template <typename Out, typename WorkSet, typename Edges, typename EnableT = 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 TransitiveClosure =
+    typename TransitiveClosureInner<
+        TypeList<>,
+        typename Edges::template flatmap<TransitiveClosureInitializer>::template unique<>,
+        Edges>::type;
 
 /** @} */
 


[3/3] incubator-quickstep git commit: Initial type system

Posted by ji...@apache.org.
Initial type system


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

Branch: refs/heads/new-op
Commit: db611137e07bf014900fbb175795e8ce8d1a0b8f
Parents: 9db9fc8
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Fri Mar 31 22:32:34 2017 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Fri Mar 31 22:32:34 2017 -0500

----------------------------------------------------------------------
 types/AsciiStringSuperType.hpp                  |   6 +-
 types/BoolType.cpp                              |   0
 types/BoolType.hpp                              |   0
 types/CMakeLists.txt                            |  32 +-
 types/CharType.cpp                              |  18 -
 types/CharType.hpp                              |  44 +--
 types/DateType.hpp                              |  41 +-
 types/DatetimeIntervalType.hpp                  |  41 +-
 types/DatetimeType.hpp                          |  41 +-
 types/DoubleType.hpp                            |  37 +-
 types/FloatType.hpp                             |  37 +-
 types/IntType.hpp                               |  37 +-
 types/LongType.hpp                              |  37 +-
 types/NullCoercibilityCheckMacro.hpp            |   3 -
 types/NullType.hpp                              |  30 +-
 types/NumericSuperType.hpp                      |  18 +-
 types/NumericTypeSafeCoercibility.hpp           |  59 +++
 types/NumericTypeUnifier.hpp                    | 105 ++----
 types/Type.cpp                                  |  11 +
 types/Type.hpp                                  |  99 ++++-
 types/TypeConcept.hpp                           |  33 --
 types/TypeIDSelectors.hpp                       | 151 ++++++++
 types/TypeRegistrar.hpp                         |  95 +++++
 types/TypeUtil.hpp                              | 124 +-----
 types/VarCharType.cpp                           |  18 -
 types/VarCharType.hpp                           |  40 +-
 types/YearMonthIntervalType.hpp                 |  41 +-
 .../comparisons/PatternMatchingComparison.cpp   |  16 +-
 .../operations/unary_operations/CMakeLists.txt  |   3 -
 .../unary_operations/SubstringOperation.cpp     | 121 +-----
 .../unary_operations/SubstringOperation.hpp     | 103 ++++-
 utility/TemplateUtil.hpp                        | 377 +++++++++++++------
 32 files changed, 893 insertions(+), 925 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/AsciiStringSuperType.hpp
----------------------------------------------------------------------
diff --git a/types/AsciiStringSuperType.hpp b/types/AsciiStringSuperType.hpp
index 82396b2..16a8285 100644
--- a/types/AsciiStringSuperType.hpp
+++ b/types/AsciiStringSuperType.hpp
@@ -35,7 +35,9 @@ namespace quickstep {
  * @brief A superclass for ASCII string types.
  **/
 template <typename TypeClass, TypeID type_id, TypeStorageLayout layout>
-class AsciiStringSuperType : public TypeSynthesizer<TypeClass, type_id, true, layout> {
+class AsciiStringSuperType
+    : public TypeSynthesizer<TypeClass, type_id,
+                             true, layout, void, Type::kAsciiString> {
  public:
   bool isCoercibleFrom(const Type &original_type) const override {
     if (original_type.isNullable() && !this->nullable_) {
@@ -59,7 +61,7 @@ class AsciiStringSuperType : public TypeSynthesizer<TypeClass, type_id, true, la
                        const std::size_t minimum_byte_length,
                        const std::size_t maximum_byte_length,
                        const std::size_t string_length)
-      : TypeSynthesizer<TypeClass, type_id, true, layout>(
+      : TypeSynthesizer<TypeClass, type_id, true, layout, void, Type::kAsciiString>(
             Type::kAsciiString, type_id, nullable, minimum_byte_length, maximum_byte_length),
         length_(string_length) {
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/BoolType.cpp
----------------------------------------------------------------------
diff --git a/types/BoolType.cpp b/types/BoolType.cpp
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/BoolType.hpp
----------------------------------------------------------------------
diff --git a/types/BoolType.hpp b/types/BoolType.hpp
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/CMakeLists.txt b/types/CMakeLists.txt
index 96aea8c..2419e0c 100644
--- a/types/CMakeLists.txt
+++ b/types/CMakeLists.txt
@@ -33,6 +33,7 @@ QS_PROTOBUF_GENERATE_CPP(types_Type_proto_srcs types_Type_proto_hdrs Type.proto)
 
 # Declare micro-libs:
 add_library(quickstep_types_AsciiStringSuperType ../empty_src.cpp AsciiStringSuperType.hpp)
+add_library(quickstep_types_BoolType BoolType.cpp BoolType.hpp)
 add_library(quickstep_types_CharType CharType.cpp CharType.hpp)
 add_library(quickstep_types_DateOperatorOverloads ../empty_src.cpp DateOperatorOverloads.hpp)
 add_library(quickstep_types_DateType DateType.cpp DateType.hpp)
@@ -48,11 +49,14 @@ add_library(quickstep_types_LongType LongType.cpp LongType.hpp)
 add_library(quickstep_types_NullCoercibilityCheckMacro ../empty_src.cpp NullCoercibilityCheckMacro.hpp)
 add_library(quickstep_types_NullType ../empty_src.cpp NullType.hpp)
 add_library(quickstep_types_NumericSuperType ../empty_src.cpp NumericSuperType.hpp)
+add_library(quickstep_types_NumericTypeSafeCoercibility ../empty_src.cpp NumericTypeSafeCoercibility.hpp)
 add_library(quickstep_types_NumericTypeUnifier ../empty_src.cpp NumericTypeUnifier.hpp)
 add_library(quickstep_types_Type Type.cpp Type.hpp)
 add_library(quickstep_types_TypeErrors ../empty_src.cpp TypeErrors.hpp)
 add_library(quickstep_types_TypeFactory TypeFactory.cpp TypeFactory.hpp)
 add_library(quickstep_types_TypeID TypeID.cpp TypeID.hpp)
+add_library(quickstep_types_TypeIDSelectors ../empty_src.cpp TypeIDSelectors.hpp)
+add_library(quickstep_types_TypeRegistrar ../empty_src.cpp TypeRegistrar.hpp)
 add_library(quickstep_types_TypeUtil ../empty_src.cpp TypeUtil.hpp)
 add_library(quickstep_types_Type_proto ${types_Type_proto_srcs})
 add_library(quickstep_types_TypedValue TypedValue.cpp TypedValue.hpp)
@@ -64,8 +68,15 @@ add_library(quickstep_types_YearMonthIntervalType YearMonthIntervalType.cpp Year
 target_link_libraries(quickstep_types_AsciiStringSuperType
                       quickstep_types_Type
                       quickstep_types_TypeID)
+target_link_libraries(quickstep_types_BoolType
+                      glog
+                      quickstep_types_Type
+                      quickstep_types_TypeID
+                      quickstep_types_TypedValue
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_types_CharType
                       glog
+                      quickstep_types_AsciiStringSuperType
                       quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_Type
                       quickstep_types_TypeID
@@ -158,12 +169,17 @@ target_link_libraries(quickstep_types_NumericSuperType
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_types_NumericTypeSafeCoercibility
+                      quickstep_utility_TemplateUtil)
+target_link_libraries(quickstep_types_NumericTypeUnifier
+                      quickstep_types_NumericTypeSafeCoercibility)
 target_link_libraries(quickstep_types_Type
                       glog
                       quickstep_types_Type_proto
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
-                      quickstep_utility_Macros)
+                      quickstep_utility_Macros
+                      quickstep_utility_PtrMap)
 target_link_libraries(quickstep_types_TypeFactory
                       glog
                       quickstep_types_CharType
@@ -177,6 +193,7 @@ target_link_libraries(quickstep_types_TypeFactory
                       quickstep_types_NullType
                       quickstep_types_Type
                       quickstep_types_TypeID
+                      quickstep_types_TypeUtil
                       quickstep_types_Type_proto
                       quickstep_types_VarCharType
                       quickstep_types_YearMonthIntervalType
@@ -184,6 +201,12 @@ target_link_libraries(quickstep_types_TypeFactory
 target_link_libraries(quickstep_types_TypeID
                       quickstep_types_Type_proto
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_types_TypeIDSelectors
+                      quickstep_types_TypeID
+                      quickstep_utility_TemplateUtil)
+target_link_libraries(quickstep_types_TypeRegistrar
+                      quickstep_types_TypeID
+                      quickstep_types_TypeIDSelectors)
 target_link_libraries(quickstep_types_TypeUtil
                       quickstep_types_CharType
                       quickstep_types_DateType
@@ -196,11 +219,11 @@ target_link_libraries(quickstep_types_TypeUtil
                       quickstep_types_NullType
                       quickstep_types_Type
                       quickstep_types_TypeID
+                      quickstep_types_TypeRegistrar
                       quickstep_types_Type_proto
                       quickstep_types_VarCharType
                       quickstep_types_YearMonthIntervalType
-                      quickstep_utility_Macros
-                      quickstep_utility_TemplateUtil)
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_types_TypedValue
                       farmhash
                       glog
@@ -219,6 +242,7 @@ target_link_libraries(quickstep_types_Type_proto
                       ${PROTOBUF_LIBRARY})
 target_link_libraries(quickstep_types_VarCharType
                       glog
+                      quickstep_types_AsciiStringSuperType
                       quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_Type
                       quickstep_types_TypeID
@@ -240,6 +264,7 @@ target_link_libraries(quickstep_types_YearMonthIntervalType
 add_library(quickstep_types ../empty_src.cpp TypesModule.hpp)
 target_link_libraries(quickstep_types
                       quickstep_types_AsciiStringSuperType
+                      quickstep_types_BoolType
                       quickstep_types_CharType
                       quickstep_types_DateOperatorOverloads
                       quickstep_types_DateType
@@ -255,6 +280,7 @@ target_link_libraries(quickstep_types
                       quickstep_types_NullCoercibilityCheckMacro
                       quickstep_types_NullType
                       quickstep_types_NumericSuperType
+                      quickstep_types_NumericTypeSafeCoercibility
                       quickstep_types_NumericTypeUnifier
                       quickstep_types_Type
                       quickstep_types_TypeUtil

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/CharType.cpp
----------------------------------------------------------------------
diff --git a/types/CharType.cpp b/types/CharType.cpp
index c92d33e..2ed469b 100644
--- a/types/CharType.cpp
+++ b/types/CharType.cpp
@@ -42,24 +42,6 @@ using std::string;
 
 namespace quickstep {
 
-template <bool nullable_internal>
-const CharType& CharType::InstanceInternal(const std::size_t length) {
-  static PtrMap<size_t, CharType> instance_map;
-  PtrMap<size_t, CharType>::iterator imit = instance_map.find(length);
-  if (imit == instance_map.end()) {
-    imit = instance_map.insert(length, new CharType(length, nullable_internal)).first;
-  }
-  return *(imit->second);
-}
-
-const CharType& CharType::InstanceNonNullable(const std::size_t length) {
-  return InstanceInternal<false>(length);
-}
-
-const CharType& CharType::InstanceNullable(const std::size_t length) {
-  return InstanceInternal<true>(length);
-}
-
 bool CharType::isSafelyCoercibleFrom(const Type &original_type) const {
   QUICKSTEP_NULL_COERCIBILITY_CHECK();
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/CharType.hpp
----------------------------------------------------------------------
diff --git a/types/CharType.hpp b/types/CharType.hpp
index fab0f61..919cc9c 100644
--- a/types/CharType.hpp
+++ b/types/CharType.hpp
@@ -45,47 +45,6 @@ namespace quickstep {
  **/
 class CharType : public AsciiStringSuperType<CharType, kChar, kNonNativeInline> {
  public:
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this Type
-   *        for the specified length.
-   *
-   * @param length The length parameter of the CharType.
-   * @return A reference to the non-nullable singleton instance of this Type
-   *         for the specified length.
-   **/
-  static const CharType& InstanceNonNullable(const std::size_t length);
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type for
-   *        the specified length.
-   *
-   * @param length The length parameter of the CharType.
-   * @return A reference to the nullable singleton instance of this Type for
-   *         the specified length.
-   **/
-  static const CharType& InstanceNullable(const std::size_t length);
-
-  /**
-   * @brief Get a reference to the singleton instance of this Type for the
-   *        specified length and nullability.
-   *
-   * @param length The length parameter of the CharType.
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the singleton instance of this Type for the
-   *         specified length.
-   **/
-  static const CharType& Instance(const std::size_t length, const bool nullable) {
-    if (nullable) {
-      return InstanceNullable(length);
-    } else {
-      return InstanceNonNullable(length);
-    }
-  }
-
-  std::size_t estimateAverageByteLength() const override {
-    return length_;
-  }
-
   bool isSafelyCoercibleFrom(const Type &original_type) const override;
 
   std::string getName() const override;
@@ -112,8 +71,7 @@ class CharType : public AsciiStringSuperType<CharType, kChar, kNonNativeInline>
             nullable, length, length, length) {
   }
 
-  template <bool nullable_internal>
-  static const CharType& InstanceInternal(const std::size_t length);
+  template <typename, bool> friend class TypeInstance;
 
   DISALLOW_COPY_AND_ASSIGN(CharType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/DateType.hpp
----------------------------------------------------------------------
diff --git a/types/DateType.hpp b/types/DateType.hpp
index 11e9908..d9356a2 100644
--- a/types/DateType.hpp
+++ b/types/DateType.hpp
@@ -42,45 +42,6 @@ class TypedValue;
  **/
 class DateType : public TypeSynthesizer<DateType, kDate, false, kNativeEmbedded, DateLit> {
  public:
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const DateType& InstanceNonNullable() {
-    static DateType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const DateType& InstanceNullable() {
-    static DateType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const DateType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
-  std::size_t estimateAverageByteLength() const override {
-    return sizeof(DateLit);
-  }
-
   int getPrintWidth() const override {
     return DateLit::kIsoChars;
   }
@@ -109,6 +70,8 @@ class DateType : public TypeSynthesizer<DateType, kDate, false, kNativeEmbedded,
              sizeof(DateLit), sizeof(DateLit)) {
   }
 
+  template <typename, bool> friend class TypeInstance;
+
   DISALLOW_COPY_AND_ASSIGN(DateType);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/DatetimeIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeIntervalType.hpp b/types/DatetimeIntervalType.hpp
index ba05dce..25eb231 100644
--- a/types/DatetimeIntervalType.hpp
+++ b/types/DatetimeIntervalType.hpp
@@ -44,45 +44,6 @@ class DatetimeIntervalType :
     public TypeSynthesizer<DatetimeIntervalType, kDatetimeInterval,
                            false, kNativeEmbedded, DatetimeIntervalLit> {
  public:
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const DatetimeIntervalType& InstanceNonNullable() {
-    static DatetimeIntervalType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const DatetimeIntervalType& InstanceNullable() {
-    static DatetimeIntervalType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const DatetimeIntervalType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
-  std::size_t estimateAverageByteLength() const override {
-    return sizeof(DatetimeIntervalLit);
-  }
-
   int getPrintWidth() const override {
     return DatetimeIntervalLit::kPrintingChars;
   }
@@ -104,6 +65,8 @@ class DatetimeIntervalType :
              sizeof(DatetimeIntervalLit), sizeof(DatetimeIntervalLit)) {
   }
 
+  template <typename, bool> friend class TypeInstance;
+
   DISALLOW_COPY_AND_ASSIGN(DatetimeIntervalType);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/DatetimeType.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeType.hpp b/types/DatetimeType.hpp
index 2189d22..e08e39b 100644
--- a/types/DatetimeType.hpp
+++ b/types/DatetimeType.hpp
@@ -43,45 +43,6 @@ class TypedValue;
 class DatetimeType
     : public TypeSynthesizer<DatetimeType, kDatetime, false, kNativeEmbedded, DatetimeLit> {
  public:
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const DatetimeType& InstanceNonNullable() {
-    static DatetimeType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const DatetimeType& InstanceNullable() {
-    static DatetimeType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const DatetimeType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
-  std::size_t estimateAverageByteLength() const override {
-    return sizeof(DatetimeLit);
-  }
-
   int getPrintWidth() const override {
     return DatetimeLit::kIsoChars;
   }
@@ -118,6 +79,8 @@ class DatetimeType
              sizeof(DatetimeLit), sizeof(DatetimeLit)) {
   }
 
+  template <typename, bool> friend class TypeInstance;
+
   DISALLOW_COPY_AND_ASSIGN(DatetimeType);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/DoubleType.hpp
----------------------------------------------------------------------
diff --git a/types/DoubleType.hpp b/types/DoubleType.hpp
index e8bb97a..a431e0a 100644
--- a/types/DoubleType.hpp
+++ b/types/DoubleType.hpp
@@ -42,41 +42,6 @@ class Type;
  **/
 class DoubleType : public NumericSuperType<DoubleType, kDouble, double> {
  public:
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const DoubleType& InstanceNonNullable() {
-    static DoubleType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const DoubleType& InstanceNullable() {
-    static DoubleType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const DoubleType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
   bool isSafelyCoercibleFrom(const Type &original_type) const override;
 
   int getPrintWidth() const override {
@@ -115,6 +80,8 @@ class DoubleType : public NumericSuperType<DoubleType, kDouble, double> {
       : NumericSuperType<DoubleType, kDouble, double>(nullable) {
   }
 
+  template <typename, bool> friend class TypeInstance;
+
   DISALLOW_COPY_AND_ASSIGN(DoubleType);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/FloatType.hpp
----------------------------------------------------------------------
diff --git a/types/FloatType.hpp b/types/FloatType.hpp
index c1428b0..29c05f0 100644
--- a/types/FloatType.hpp
+++ b/types/FloatType.hpp
@@ -42,41 +42,6 @@ class Type;
  **/
 class FloatType : public NumericSuperType<FloatType, kFloat, float> {
  public:
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type
-   **/
-  static const FloatType& InstanceNonNullable() {
-    static FloatType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type
-   *
-   * @return A reference to the nullable singleton instance of this Type
-   **/
-  static const FloatType& InstanceNullable() {
-    static FloatType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type
-   *
-   * @param nullable Whether to get the nullable version of this Type
-   * @return A reference to the desired singleton instance of this Type
-   **/
-  static const FloatType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
   bool isSafelyCoercibleFrom(const Type &original_type) const override;
 
   int getPrintWidth() const override {
@@ -115,6 +80,8 @@ class FloatType : public NumericSuperType<FloatType, kFloat, float> {
       : NumericSuperType<FloatType, kFloat, float>(nullable) {
   }
 
+  template <typename, bool> friend class TypeInstance;
+
   DISALLOW_COPY_AND_ASSIGN(FloatType);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/IntType.hpp
----------------------------------------------------------------------
diff --git a/types/IntType.hpp b/types/IntType.hpp
index dda89bb..0ef3da9 100644
--- a/types/IntType.hpp
+++ b/types/IntType.hpp
@@ -42,41 +42,6 @@ class Type;
  **/
 class IntType : public NumericSuperType<IntType, kInt, int> {
  public:
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const IntType& InstanceNonNullable() {
-    static IntType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const IntType& InstanceNullable() {
-    static IntType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const IntType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
   bool isSafelyCoercibleFrom(const Type &original_type) const override;
 
   int getPrintWidth() const override {
@@ -102,6 +67,8 @@ class IntType : public NumericSuperType<IntType, kInt, int> {
       : NumericSuperType<IntType, kInt, int>(nullable) {
   }
 
+  template <typename, bool> friend class TypeInstance;
+
   DISALLOW_COPY_AND_ASSIGN(IntType);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/LongType.hpp
----------------------------------------------------------------------
diff --git a/types/LongType.hpp b/types/LongType.hpp
index f2814bf..76569c1 100644
--- a/types/LongType.hpp
+++ b/types/LongType.hpp
@@ -43,41 +43,6 @@ class Type;
  **/
 class LongType : public NumericSuperType<LongType, kLong, std::int64_t> {
  public:
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const LongType& InstanceNonNullable() {
-    static LongType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const LongType& InstanceNullable() {
-    static LongType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const LongType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
   bool isSafelyCoercibleFrom(const Type &original_type) const override;
 
   // Fully represented digits, single leading digit, and possible '-'
@@ -103,6 +68,8 @@ class LongType : public NumericSuperType<LongType, kLong, std::int64_t> {
       : NumericSuperType<LongType, kLong, std::int64_t>(nullable) {
   }
 
+  template <typename, bool> friend class TypeInstance;
+
   DISALLOW_COPY_AND_ASSIGN(LongType);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/NullCoercibilityCheckMacro.hpp
----------------------------------------------------------------------
diff --git a/types/NullCoercibilityCheckMacro.hpp b/types/NullCoercibilityCheckMacro.hpp
index 2dcbec3..9cdd152 100644
--- a/types/NullCoercibilityCheckMacro.hpp
+++ b/types/NullCoercibilityCheckMacro.hpp
@@ -20,9 +20,6 @@
 #ifndef QUICKSTEP_TYPES_NULL_COERCIBILITY_CHECK_MACRO_HPP_
 #define QUICKSTEP_TYPES_NULL_COERCIBILITY_CHECK_MACRO_HPP_
 
-#include "types/Type.hpp"
-#include "types/TypeID.hpp"
-
 /** \addtogroup Types
  *  @{
  */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/NullType.hpp
----------------------------------------------------------------------
diff --git a/types/NullType.hpp b/types/NullType.hpp
index 0b9381c..647c2f7 100644
--- a/types/NullType.hpp
+++ b/types/NullType.hpp
@@ -50,16 +50,9 @@ class TypedValue;
  **/
 class NullType : public TypeSynthesizer<NullType, kNullType, false, kNativeEmbedded> {
  public:
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   * @note Unlike other Types, there is no corresponding method to get a
-   *       non-nullable version of NullType. NullType is ALWAYS nullable.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const NullType& InstanceNullable() {
-    static NullType instance;
-    return instance;
+  static const NullType& InstanceNonNullable() {
+    LOG(FATAL) << "Called NullType::InstanceNonNullable(), "
+               << "which is not allowed.";
   }
 
   static const NullType& Instance(const bool nullable) {
@@ -71,18 +64,6 @@ class NullType : public TypeSynthesizer<NullType, kNullType, false, kNativeEmbed
     }
   }
 
-  std::size_t estimateAverageByteLength() const override {
-    return 0;
-  }
-
-  bool isCoercibleFrom(const Type &original_type) const override {
-    return original_type.getTypeID() == kNullType;
-  }
-
-  bool isSafelyCoercibleFrom(const Type &original_type) const override {
-    return original_type.getTypeID() == kNullType;
-  }
-
   int getPrintWidth() const override {
     return 0;
   }
@@ -105,11 +86,14 @@ class NullType : public TypeSynthesizer<NullType, kNullType, false, kNativeEmbed
  private:
   // NOTE(chasseur): NullType requires 0 bytes of inherent storage. It does,
   // however, require a bit in NULL bitmaps.
-  NullType()
+  NullType(const bool nullable)
       : TypeSynthesizer<NullType, kNullType, false, kNativeEmbedded>(
             Type::kOther, kStaticTypeID, true, 0, 0) {
+    DCHECK(nullable);
   }
 
+  template <typename, bool> friend class TypeInstance;
+
   DISALLOW_COPY_AND_ASSIGN(NullType);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/NumericSuperType.hpp
----------------------------------------------------------------------
diff --git a/types/NumericSuperType.hpp b/types/NumericSuperType.hpp
index a3f1808..066ca1e 100644
--- a/types/NumericSuperType.hpp
+++ b/types/NumericSuperType.hpp
@@ -21,8 +21,10 @@
 #define QUICKSTEP_TYPES_NUMERIC_SUPER_TYPE_HPP_
 
 #include <cstddef>
+#include <unordered_set>
 
 #include "types/NullCoercibilityCheckMacro.hpp"
+#include "types/NumericTypeSafeCoercibility.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
@@ -40,10 +42,12 @@ namespace quickstep {
  **/
 template <typename TypeClass, TypeID type_id, typename CppType>
 class NumericSuperType
-    : public TypeSynthesizer<TypeClass, type_id, false, kNativeEmbedded, CppType> {
+    : public TypeSynthesizer<TypeClass, type_id,
+                             false, kNativeEmbedded, CppType, Type::kNumeric> {
  public:
-  std::size_t estimateAverageByteLength() const override {
-    return sizeof(CppType);
+  bool isSafelyCoercibleFrom(const Type &original_type) const override {
+    const auto it = safe_coerce_cache_.find(original_type.getTypeID());
+    return it != safe_coerce_cache_.end();
   }
 
   bool isCoercibleFrom(const Type &original_type) const override {
@@ -57,11 +61,13 @@ class NumericSuperType
 
  protected:
   explicit NumericSuperType(const bool nullable)
-      : TypeSynthesizer<TypeClass, type_id, false, kNativeEmbedded, CppType>(
-            Type::kNumeric, type_id, nullable, sizeof(CppType), sizeof(CppType)) {
-  }
+      : TypeSynthesizer<TypeClass, type_id,
+                        false, kNativeEmbedded, CppType, Type::kNumeric>(
+            Type::kNumeric, type_id, nullable, sizeof(CppType), sizeof(CppType)) {}
 
  private:
+  std::unordered_set<TypeID> safe_coerce_cache_;
+
   DISALLOW_COPY_AND_ASSIGN(NumericSuperType);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/NumericTypeSafeCoercibility.hpp
----------------------------------------------------------------------
diff --git a/types/NumericTypeSafeCoercibility.hpp b/types/NumericTypeSafeCoercibility.hpp
new file mode 100644
index 0000000..2ef29b7
--- /dev/null
+++ b/types/NumericTypeSafeCoercibility.hpp
@@ -0,0 +1,59 @@
+/**
+ * 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_NUMERIC_TYPE_SAFE_COERCIBILITY_HPP_
+#define QUICKSTEP_TYPES_NUMERIC_TYPE_SAFE_COERCIBILITY_HPP_
+
+#include "utility/TemplateUtil.hpp"
+
+namespace quickstep {
+
+class BoolType;
+class DoubleType;
+class FloatType;
+class IntType;
+class LongType;
+
+/** \addtogroup Types
+ *  @{
+ */
+
+using NumericTypeSafeCoersions = TypeList<
+    TypeList<BoolType, IntType>,
+    TypeList<IntType, FloatType>,
+    TypeList<IntType, LongType>,
+    TypeList<FloatType, DoubleType>,
+    TypeList<LongType, DoubleType>
+>;
+
+using NumericTypeSafeCoersionClosure = TransitiveClosure<NumericTypeSafeCoersions>;
+
+
+template <typename LeftType, typename RightType>
+struct NumericTypeSafeCoercibility {
+  static constexpr bool value =
+      NumericTypeSafeCoersionClosure::contains<
+          TypeList<LeftType, RightType>>::value;
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_NUMERIC_TYPE_SAFE_COERCIBILITY_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/NumericTypeUnifier.hpp
----------------------------------------------------------------------
diff --git a/types/NumericTypeUnifier.hpp b/types/NumericTypeUnifier.hpp
index 168dfb1..6253a18 100644
--- a/types/NumericTypeUnifier.hpp
+++ b/types/NumericTypeUnifier.hpp
@@ -20,16 +20,9 @@
 #ifndef QUICKSTEP_TYPES_NUMERIC_TYPE_UNIFIER_HPP_
 #define QUICKSTEP_TYPES_NUMERIC_TYPE_UNIFIER_HPP_
 
-namespace quickstep {
-
-class DoubleType;
-class FloatType;
-class IntType;
-class LongType;
+#include "types/NumericTypeSafeCoercibility.hpp"
 
-/** \addtogroup Types
- *  @{
- */
+namespace quickstep {
 
 /**
  * @brief A traits template that resolves what the "unifying" Type of two
@@ -51,92 +44,52 @@ class LongType;
  * @tparam RightType The second Quickstep numeric Type class to unify.
  **/
 template <typename LeftType, typename RightType>
-struct NumericTypeUnifier {
-};
-
-/** @} */
-
-// Explicit template specializations for all combinations of builtin numeric
-// types.
-template<>
-struct NumericTypeUnifier<IntType, IntType> {
-  typedef IntType type;
-};
-
-template<>
-struct NumericTypeUnifier<IntType, LongType> {
-  typedef LongType type;
-};
-
-template<>
-struct NumericTypeUnifier<IntType, FloatType> {
-  typedef FloatType type;
-};
+struct NumericTypeUnifier;
 
-template<>
-struct NumericTypeUnifier<IntType, DoubleType> {
-  typedef DoubleType type;
-};
-
-template<>
-struct NumericTypeUnifier<LongType, IntType> {
-  typedef LongType type;
-};
-
-template<>
-struct NumericTypeUnifier<LongType, LongType> {
-  typedef LongType type;
-};
 
-template<>
-struct NumericTypeUnifier<LongType, FloatType> {
-  typedef DoubleType type;
-};
+namespace internal {
 
-template<>
-struct NumericTypeUnifier<LongType, DoubleType> {
-  typedef DoubleType type;
-};
+using NumericTypeSafeCoersionClosure = TransitiveClosure<NumericTypeSafeCoersions>;
 
-template<>
-struct NumericTypeUnifier<FloatType, IntType> {
-  typedef FloatType type;
-};
+template <typename LeftType, typename RightType, typename EnableT = void>
+struct NumericTypeUnifierHelper;
 
-template<>
-struct NumericTypeUnifier<FloatType, LongType> {
-  typedef DoubleType type;
+template <typename LeftType, typename RightType>
+struct NumericTypeUnifierHelper<
+    LeftType, RightType,
+    std::enable_if_t<NumericTypeSafeCoersionClosure::contains<
+                         TypeList<LeftType, RightType>>::value>> {
+  typedef RightType type;
 };
 
-template<>
-struct NumericTypeUnifier<FloatType, FloatType> {
-  typedef FloatType type;
+template <typename LeftType, typename RightType>
+struct NumericTypeUnifierHelper<
+    LeftType, RightType,
+    std::enable_if_t<!std::is_same<LeftType, RightType>::value &&
+                     NumericTypeSafeCoersionClosure::contains<
+                         TypeList<RightType, LeftType>>::value>> {
+  typedef LeftType type;
 };
 
+// Explicit template specializations for all combinations of builtin numeric
+// types.
 template<>
-struct NumericTypeUnifier<FloatType, DoubleType> {
+struct NumericTypeUnifierHelper<LongType, FloatType> {
   typedef DoubleType type;
 };
 
 template<>
-struct NumericTypeUnifier<DoubleType, IntType> {
+struct NumericTypeUnifierHelper<FloatType, LongType> {
   typedef DoubleType type;
 };
 
-template<>
-struct NumericTypeUnifier<DoubleType, LongType> {
-  typedef DoubleType type;
-};
+}  // namespace internal
 
-template<>
-struct NumericTypeUnifier<DoubleType, FloatType> {
-  typedef DoubleType type;
-};
+template <typename LeftType, typename RightType>
+struct NumericTypeUnifier
+    : internal::NumericTypeUnifierHelper<LeftType, RightType> {};
 
-template<>
-struct NumericTypeUnifier<DoubleType, DoubleType> {
-  typedef DoubleType type;
-};
+/** @} */
 
 }  // namespace quickstep
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/Type.cpp
----------------------------------------------------------------------
diff --git a/types/Type.cpp b/types/Type.cpp
index af6503b..b69cb65 100644
--- a/types/Type.cpp
+++ b/types/Type.cpp
@@ -42,6 +42,17 @@ bool Type::isSafelyCoercibleFrom(const Type &original_type) const {
   return (original_type.getTypeID() == type_id_);
 }
 
+std::size_t Type::estimateAverageByteLength() const {
+  if (minimum_byte_length_ == maximum_byte_length_) {
+    return maximum_byte_length_;
+  }
+  if (maximum_byte_length_ > 160) {
+    return 80;
+  } else {
+    return (maximum_byte_length_ >> 1) + 1;
+  }
+}
+
 void Type::printValueToFile(const TypedValue &value,
                             FILE *file,
                             const int padding) const {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/Type.hpp
----------------------------------------------------------------------
diff --git a/types/Type.hpp b/types/Type.hpp
index e293766..4f2042a 100644
--- a/types/Type.hpp
+++ b/types/Type.hpp
@@ -30,6 +30,7 @@
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
+#include "utility/PtrMap.hpp"
 
 #include "glog/logging.h"
 
@@ -219,7 +220,7 @@ class Type {
    * @return An estimate of the average number of bytes used by data items of
    *         this type.
    **/
-  virtual std::size_t estimateAverageByteLength() const = 0;
+  virtual std::size_t estimateAverageByteLength() const;
 
   /**
    * @brief Determine whether this Type is exactly the same as another.
@@ -466,11 +467,76 @@ class Type {
   DISALLOW_COPY_AND_ASSIGN(Type);
 };
 
+
+template <typename TypeClass, bool parameterized>
+class TypeInstance;
+
+template <typename TypeClass>
+class TypeInstance<TypeClass, false> {
+ public:
+  static const TypeClass& InstanceNonNullable() {
+    return InstanceInternal<false>();
+  }
+
+  static const TypeClass& InstanceNullable() {
+    return InstanceInternal<true>();
+  }
+
+  static const TypeClass& Instance(const bool nullable) {
+    if (nullable) {
+      return InstanceNullable();
+    } else {
+      return InstanceNonNullable();
+    }
+  }
+
+ private:
+  template <bool nullable>
+  inline static const TypeClass& InstanceInternal() {
+    static TypeClass instance(nullable);
+    return instance;
+  }
+};
+
+template <typename TypeClass>
+class TypeInstance<TypeClass, true> {
+ public:
+  static const TypeClass& InstanceNonNullable(const std::size_t length) {
+    return InstanceInternal<false>(length);
+  }
+
+  static const TypeClass& InstanceNullable(const std::size_t length) {
+    return InstanceInternal<true>(length);
+  }
+
+  static const TypeClass& Instance(const std::size_t length, const bool nullable) {
+    if (nullable) {
+      return InstanceNullable(length);
+    } else {
+      return InstanceNonNullable(length);
+    }
+  }
+
+ private:
+  template <bool nullable>
+  inline static const TypeClass& InstanceInternal(const std::size_t length) {
+    static PtrMap<size_t, TypeClass> instance_map;
+    auto imit = instance_map.find(length);
+    if (imit == instance_map.end()) {
+      imit = instance_map.insert(length, new TypeClass(length, nullable)).first;
+    }
+    return *(imit->second);
+  }
+};
+
+
 template <typename TypeClass, TypeID type_id,
           bool parameterized, TypeStorageLayout layout,
-          typename CppType = void>
-class TypeSynthesizer : public Type {
+          typename CppType = void,
+          Type::SuperTypeID super_type_id = Type::kOther>
+class TypeSynthesizer : public Type, public TypeInstance<TypeClass, parameterized> {
  public:
+  static constexpr Type::SuperTypeID kStaticSuperTypeID = super_type_id;
   static constexpr TypeID kStaticTypeID = type_id;
   static constexpr bool kParameterized = parameterized;
   static constexpr TypeStorageLayout kLayout = layout;
@@ -507,32 +573,43 @@ class TypeSynthesizer : public Type {
   template <bool has_param>
   inline const Type& getInstance(const bool nullable,
                                  std::enable_if_t<has_param>* = 0) const {
-    return TypeClass::Instance(parameter_, nullable);
+    return TypeInstance<TypeClass, kParameterized>::Instance(parameter_, nullable);
   }
 
   template <bool has_param>
   inline const Type& getInstance(const bool nullable,
                                  std::enable_if_t<!has_param>* = 0) const {
-    return TypeClass::Instance(nullable);
+    return TypeInstance<TypeClass, kParameterized>::Instance(nullable);
   }
 
+  friend class TypeInstance<TypeClass, kParameterized>;
+
   DISALLOW_COPY_AND_ASSIGN(TypeSynthesizer);
 };
 
 template <typename TypeClass, TypeID type_id,
-          bool parameterized, TypeStorageLayout layout, typename CppType>
+          bool parameterized, TypeStorageLayout layout,
+          typename CppType, Type::SuperTypeID super_type_id>
+constexpr Type::SuperTypeID TypeSynthesizer<
+    TypeClass, type_id, parameterized, layout, CppType, super_type_id>::kStaticSuperTypeID;
+
+template <typename TypeClass, TypeID type_id,
+          bool parameterized, TypeStorageLayout layout,
+          typename CppType, Type::SuperTypeID super_type_id>
 constexpr TypeID TypeSynthesizer<
-    TypeClass, type_id, parameterized, layout, CppType>::kStaticTypeID;
+    TypeClass, type_id, parameterized, layout, CppType, super_type_id>::kStaticTypeID;
 
 template <typename TypeClass, TypeID type_id,
-          bool parameterized, TypeStorageLayout layout, typename CppType>
+          bool parameterized, TypeStorageLayout layout,
+          typename CppType, Type::SuperTypeID super_type_id>
 constexpr bool TypeSynthesizer<
-    TypeClass, type_id, parameterized, layout, CppType>::kParameterized;
+    TypeClass, type_id, parameterized, layout, CppType, super_type_id>::kParameterized;
 
 template <typename TypeClass, TypeID type_id,
-          bool parameterized, TypeStorageLayout layout, typename CppType>
+          bool parameterized, TypeStorageLayout layout,
+          typename CppType, Type::SuperTypeID super_type_id>
 constexpr TypeStorageLayout TypeSynthesizer<
-    TypeClass, type_id, parameterized, layout, CppType>::kLayout;
+    TypeClass, type_id, parameterized, layout, CppType, super_type_id>::kLayout;
 
 /** @} */
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/TypeConcept.hpp
----------------------------------------------------------------------
diff --git a/types/TypeConcept.hpp b/types/TypeConcept.hpp
deleted file mode 100644
index d1ec79d..0000000
--- a/types/TypeConcept.hpp
+++ /dev/null
@@ -1,33 +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_TYPE_CONCEPT_HPP_
-#define QUICKSTEP_TYPES_TYPE_CONCEPT_HPP_
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_TYPE_CONCEPT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/TypeIDSelectors.hpp
----------------------------------------------------------------------
diff --git a/types/TypeIDSelectors.hpp b/types/TypeIDSelectors.hpp
new file mode 100644
index 0000000..8e0ac9f
--- /dev/null
+++ b/types/TypeIDSelectors.hpp
@@ -0,0 +1,151 @@
+/**
+ * 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_ID_SELECTORS_HPP_
+#define QUICKSTEP_TYPES_TYPE_ID_SELECTORS_HPP_
+
+#include <type_traits>
+
+#include "types/TypeID.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+struct TypeIDSelectorAll;
+
+struct TypeIDSelectorNumeric;
+
+struct TypeIDSelectorParameterized;
+
+struct TypeIDSelectorNonParameterized;
+
+template <TypeID ...candidates>
+struct TypeIDSelectorEqualsAny;
+
+
+// Forward declaration
+template <TypeID type_id>
+struct TypeClass;
+
+struct TypeIDSelectorNumeric {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wreturn-type"
+    inline static auto Invoke(const FunctorT &functor)
+        -> decltype(functor(TypeIDConstant())) {
+      DLOG(FATAL) << "Unexpected TypeID: "
+                  << kTypeNames[static_cast<int>(TypeIDConstant::value)];
+    }
+#pragma GCC diagnostic pop
+  };
+};
+
+template <typename TypeIDConstant, typename FunctorT>
+struct TypeIDSelectorNumeric::Implementation<
+    TypeIDConstant, FunctorT,
+    std::enable_if_t<TypeClass<TypeIDConstant::value>::type
+                         ::kStaticSuperTypeID == Type::kNumeric>> {
+  inline static auto Invoke(const FunctorT &functor) {
+    return functor(TypeIDConstant());
+  }
+};
+
+template <TypeID ...candidates>
+struct TypeIDSelectorEqualsAny {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wreturn-type"
+    inline static auto Invoke(const FunctorT &functor)
+        -> decltype(functor(TypeIDConstant())) {
+      DLOG(FATAL) << "Unexpected TypeID: "
+                  << kTypeNames[static_cast<int>(TypeIDConstant::value)];
+    }
+#pragma GCC diagnostic pop
+  };
+};
+
+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());
+  }
+};
+
+namespace internal {
+
+template <bool require_parameterized>
+struct TypeIDSelectorParameterizedHelper {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wreturn-type"
+    inline static auto Invoke(const FunctorT &functor)
+        -> decltype(functor(TypeIDConstant())) {
+      DLOG(FATAL) << "Unexpected TypeID: "
+                  << kTypeNames[static_cast<int>(TypeIDConstant::value)];
+    }
+#pragma GCC diagnostic pop
+  };
+};
+
+template <bool require_non_parameterized>
+template <typename TypeIDConstant, typename FunctorT>
+struct TypeIDSelectorParameterizedHelper<require_non_parameterized>::Implementation<
+    TypeIDConstant, FunctorT,
+    std::enable_if_t<TypeClass<TypeIDConstant::value>::type::kParameterized
+                         ^ require_non_parameterized>> {
+  inline static auto Invoke(const FunctorT &functor) {
+    return functor(TypeIDConstant());
+  }
+};
+
+}  // namespace internal
+
+struct TypeIDSelectorAll {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+    inline static auto Invoke(const FunctorT &functor) {
+      return functor(TypeIDConstant());
+    }
+  };
+};
+
+struct TypeIDSelectorNonParameterized
+    : internal::TypeIDSelectorParameterizedHelper<true> {};
+
+struct TypeIDSelectorParameterized
+    : internal::TypeIDSelectorParameterizedHelper<false> {};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_TYPE_ID_SELECTORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/TypeRegistrar.hpp
----------------------------------------------------------------------
diff --git a/types/TypeRegistrar.hpp b/types/TypeRegistrar.hpp
new file mode 100644
index 0000000..3366cd9
--- /dev/null
+++ b/types/TypeRegistrar.hpp
@@ -0,0 +1,95 @@
+/**
+ * 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_REGISTRAR_HPP_
+#define QUICKSTEP_TYPES_TYPE_REGISTRAR_HPP_
+
+#include <type_traits>
+
+#include "types/TypeID.hpp"
+#include "types/TypeIDSelectors.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <TypeID type_id>
+struct TypeClass;
+
+#define REGISTER_TYPE(T, TID) \
+  class T;\
+  template <> struct TypeClass<TID> { typedef T type; };
+
+REGISTER_TYPE(IntType, kInt);
+REGISTER_TYPE(LongType, kLong);
+REGISTER_TYPE(FloatType, kFloat);
+REGISTER_TYPE(DoubleType, kDouble);
+REGISTER_TYPE(DateType, kDate);
+REGISTER_TYPE(DatetimeType, kDatetime);
+REGISTER_TYPE(DatetimeIntervalType, kDatetimeInterval);
+REGISTER_TYPE(YearMonthIntervalType, kYearMonthInterval);
+REGISTER_TYPE(CharType, kChar);
+REGISTER_TYPE(VarCharType, kVarChar);
+REGISTER_TYPE(NullType, kNullType);
+
+#undef REGISTER_TYPE
+
+
+template <typename Selector = TypeIDSelectorAll, typename FunctorT>
+auto InvokeOnTypeID(const TypeID type_id, const FunctorT &functor);
+
+
+namespace internal {
+
+template <int l, int r, typename Selector, typename FunctorT>
+inline auto InvokeOnTypeIDInner(const int value,
+                                const FunctorT &functor) {
+  DCHECK_LE(l, r);
+  if (l == r) {
+    constexpr TypeID type_id = static_cast<TypeID>(r);
+    return Selector::template Implementation<
+        std::integral_constant<TypeID, type_id>, FunctorT>::Invoke(functor);
+  }
+  constexpr int m = (l + r) >> 1;
+  if (value <= m) {
+    return InvokeOnTypeIDInner<l, m, Selector, FunctorT>(value, functor);
+  } else {
+    return InvokeOnTypeIDInner<m+1, r, Selector, FunctorT>(value, functor);
+  }
+}
+
+}  // namespace internal
+
+template <typename Selector, typename FunctorT>
+auto InvokeOnTypeID(const TypeID type_id,
+                    const FunctorT &functor) {
+  return internal::InvokeOnTypeIDInner<0, static_cast<int>(kNumTypeIDs)-1,
+                                       Selector, FunctorT>(
+      static_cast<int>(type_id), functor);
+}
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_TYPE_REGISTRAR_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/TypeUtil.hpp
----------------------------------------------------------------------
diff --git a/types/TypeUtil.hpp b/types/TypeUtil.hpp
index d2183e8..c1cd3fa 100644
--- a/types/TypeUtil.hpp
+++ b/types/TypeUtil.hpp
@@ -32,135 +32,19 @@
 #include "types/LongType.hpp"
 #include "types/NullType.hpp"
 #include "types/Type.hpp"
-#include "types/Type.pb.h"
 #include "types/TypeID.hpp"
+#include "types/TypeRegistrar.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 TypeClass {};
-
-#define REGISTER_TYPE(T) \
-  template <> struct TypeClass<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 {
-#pragma GCC diagnostic push
-#pragma GCC diagnostic ignored "-Wreturn-type"
-    inline static auto Invoke(const FunctorT &functor)
-        -> decltype(functor(TypeIDConstant())) {
-      LOG(FATAL) << "Unexpected TypeID: "
-                 << kTypeNames[static_cast<int>(TypeIDConstant::value)];
-    }
-#pragma GCC diagnostic pop
-  };
-};
-
-template <bool require_non_parameterized>
-template <typename TypeIDConstant, typename FunctorT>
-struct TypeIDSelectorParameterizedHelper<require_non_parameterized>::Implementation<
-    TypeIDConstant, FunctorT,
-    std::enable_if_t<TypeClass<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 {
-#pragma GCC diagnostic push
-#pragma GCC diagnostic ignored "-Wreturn-type"
-    inline static auto Invoke(const FunctorT &functor)
-        -> decltype(functor(TypeIDConstant())) {
-      LOG(FATAL) << "Unexpected TypeID: "
-                 << kTypeNames[static_cast<int>(TypeIDConstant::value)];
-    }
-#pragma GCC diagnostic pop
-  };
-};
-
-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:
-      LOG(FATAL) << "Unrecognized TypeID in InvokeOnTypeID()";
-  }
-
-#undef REGISTER_TYPE_ID
-}
+/** \addtogroup Types
+ *  @{
+ */
 
 class TypeUtil {
  public:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/VarCharType.cpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.cpp b/types/VarCharType.cpp
index 61fd88e..7eeb04c 100644
--- a/types/VarCharType.cpp
+++ b/types/VarCharType.cpp
@@ -42,24 +42,6 @@ using std::string;
 
 namespace quickstep {
 
-template <bool nullable_internal>
-const VarCharType& VarCharType::InstanceInternal(const std::size_t length) {
-  static PtrMap<size_t, VarCharType> instance_map;
-  PtrMap<size_t, VarCharType>::iterator imit = instance_map.find(length);
-  if (imit == instance_map.end()) {
-    imit = instance_map.insert(length, new VarCharType(length, nullable_internal)).first;
-  }
-  return *(imit->second);
-}
-
-const VarCharType& VarCharType::InstanceNonNullable(const std::size_t length) {
-  return InstanceInternal<false>(length);
-}
-
-const VarCharType& VarCharType::InstanceNullable(const std::size_t length) {
-  return InstanceInternal<true>(length);
-}
-
 size_t VarCharType::estimateAverageByteLength() const {
   if (length_ > 160) {
     return 80;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/VarCharType.hpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.hpp b/types/VarCharType.hpp
index 5ca1bc1..a8f1a8d 100644
--- a/types/VarCharType.hpp
+++ b/types/VarCharType.hpp
@@ -46,43 +46,6 @@ namespace quickstep {
 class VarCharType : public AsciiStringSuperType<VarCharType, kVarChar, kOutOfLine> {
  public:
   /**
-   * @brief Get a reference to the non-nullable singleton instance of this Type
-   *        for the specified length.
-   *
-   * @param length The length parameter of the VarCharType.
-   * @return A reference to the non-nullable singleton instance of this Type
-   *         for the specified length.
-   **/
-  static const VarCharType& InstanceNonNullable(const std::size_t length);
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type for
-   *        the specified length.
-   *
-   * @param length The length parameter of the VarCharType.
-   * @return A reference to the nullable singleton instance of this Type for
-   *         the specified length.
-   **/
-  static const VarCharType& InstanceNullable(const std::size_t length);
-
-  /**
-   * @brief Get a reference to the singleton instance of this Type for the
-   *        specified length and nullability.
-   *
-   * @param length The length parameter of the VarCharType.
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the singleton instance of this Type for the
-   *         specified length and nullability.
-   **/
-  static const VarCharType& Instance(const std::size_t length, const bool nullable) {
-    if (nullable) {
-      return InstanceNullable(length);
-    } else {
-      return InstanceNonNullable(length);
-    }
-  }
-
-  /**
    * @note Includes an extra byte for a terminating null character.
    **/
   std::size_t estimateAverageByteLength() const override;
@@ -115,8 +78,7 @@ class VarCharType : public AsciiStringSuperType<VarCharType, kVarChar, kOutOfLin
             nullable, 1, length + 1, length) {
   }
 
-  template <bool nullable_internal>
-  static const VarCharType& InstanceInternal(const std::size_t length);
+  template <typename, bool> friend class TypeInstance;
 
   DISALLOW_COPY_AND_ASSIGN(VarCharType);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/YearMonthIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/YearMonthIntervalType.hpp b/types/YearMonthIntervalType.hpp
index a13b05f..3a22541 100644
--- a/types/YearMonthIntervalType.hpp
+++ b/types/YearMonthIntervalType.hpp
@@ -43,45 +43,6 @@ class YearMonthIntervalType :
     public TypeSynthesizer<YearMonthIntervalType, kYearMonthInterval,
                            false, kNativeEmbedded, YearMonthIntervalLit> {
  public:
-  /**
-   * @brief Get a reference to the non-nullable singleton instance of this
-   *        Type.
-   *
-   * @return A reference to the non-nullable singleton instance of this Type.
-   **/
-  static const YearMonthIntervalType& InstanceNonNullable() {
-    static YearMonthIntervalType instance(false);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to the nullable singleton instance of this Type.
-   *
-   * @return A reference to the nullable singleton instance of this Type.
-   **/
-  static const YearMonthIntervalType& InstanceNullable() {
-    static YearMonthIntervalType instance(true);
-    return instance;
-  }
-
-  /**
-   * @brief Get a reference to a singleton instance of this Type.
-   *
-   * @param nullable Whether to get the nullable version of this Type.
-   * @return A reference to the desired singleton instance of this Type.
-   **/
-  static const YearMonthIntervalType& Instance(const bool nullable) {
-    if (nullable) {
-      return InstanceNullable();
-    } else {
-      return InstanceNonNullable();
-    }
-  }
-
-  std::size_t estimateAverageByteLength() const override {
-    return sizeof(YearMonthIntervalLit);
-  }
-
   int getPrintWidth() const override {
     return YearMonthIntervalLit::kPrintingChars;
   }
@@ -103,6 +64,8 @@ class YearMonthIntervalType :
             sizeof(YearMonthIntervalLit), sizeof(YearMonthIntervalLit)) {
   }
 
+  template <typename, bool> friend class TypeInstance;
+
   DISALLOW_COPY_AND_ASSIGN(YearMonthIntervalType);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/operations/comparisons/PatternMatchingComparison.cpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/PatternMatchingComparison.cpp b/types/operations/comparisons/PatternMatchingComparison.cpp
index 4207f0f..7fa7031 100644
--- a/types/operations/comparisons/PatternMatchingComparison.cpp
+++ b/types/operations/comparisons/PatternMatchingComparison.cpp
@@ -121,11 +121,19 @@ UncheckedComparator* PatternMatchingComparison::makeUncheckedComparatorForTypes(
                  << " in PatternMatchinComparison::makeUncheckedComparatorForTypes()";
   }
 
-  return CreateBoolInstantiatedInstance<PatternMatchingUncheckedComparator, UncheckedComparator>(
-      std::forward_as_tuple(left_max_length, right_max_length),
+  return InvokeOnBools(
       is_like_pattern, is_negation,
-      left.isNullable(), right.isNullable());
+      left.isNullable(), right.isNullable(),
+      [&](auto is_like_pattern,  // NOLINT(build/c++11)
+          auto is_negation,
+          auto is_left_nullable,
+          auto is_right_nullable) -> UncheckedComparator* {
+    return new PatternMatchingUncheckedComparator<
+        decltype(is_like_pattern)::value,
+        decltype(is_negation)::value,
+        decltype(is_left_nullable)::value,
+        decltype(is_right_nullable)::value>(left_max_length, right_max_length);
+  });
 }
 
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/operations/unary_operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CMakeLists.txt b/types/operations/unary_operations/CMakeLists.txt
index fa47a5c..e605ca7 100644
--- a/types/operations/unary_operations/CMakeLists.txt
+++ b/types/operations/unary_operations/CMakeLists.txt
@@ -112,12 +112,9 @@ target_link_libraries(quickstep_types_operations_unaryoperations_SubstringOperat
                       quickstep_types_VarCharType
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorUtil
-                      quickstep_types_operations_Operation_proto
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_types_port_strnlen
-                      quickstep_utility_HashPair
                       quickstep_utility_Macros
-                      quickstep_utility_StringUtil
                       quickstep_utility_TemplateUtil)
 target_link_libraries(quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_catalog_CatalogTypedefs

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/operations/unary_operations/SubstringOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/SubstringOperation.cpp b/types/operations/unary_operations/SubstringOperation.cpp
index 3d0d139..2693194 100644
--- a/types/operations/unary_operations/SubstringOperation.cpp
+++ b/types/operations/unary_operations/SubstringOperation.cpp
@@ -19,20 +19,12 @@
 
 #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/port/strnlen.hpp"
 #include "utility/TemplateUtil.hpp"
 
 #include "glog/logging.h"
@@ -57,107 +49,18 @@ UncheckedUnaryOperator* SubstringOperation::makeUncheckedUnaryOperator(
   const Type *result_type = getResultType(type, static_arguments);
   DCHECK(result_type != nullptr);
 
-  return CreateBoolInstantiatedInstance<SubstringUncheckedOperator, UncheckedUnaryOperator>(
-      std::forward_as_tuple(start_position,
-                            ComputeMaximumSubstringLength(type, start_position, substring_length),
-                            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>
-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;
+  return 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);
   });
 }
 
-#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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/db611137/types/operations/unary_operations/SubstringOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/SubstringOperation.hpp b/types/operations/unary_operations/SubstringOperation.hpp
index fd5868f..afaf74d 100644
--- a/types/operations/unary_operations/SubstringOperation.hpp
+++ b/types/operations/unary_operations/SubstringOperation.hpp
@@ -27,22 +27,23 @@
 #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/VarCharType.hpp"
-#include "types/operations/Operation.pb.h"
+#include "types/containers/ColumnVector.hpp"
+#include "types/containers/ColumnVectorUtil.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "utility/HashPair.hpp"
+#include "types/port/strnlen.hpp"
 #include "utility/Macros.hpp"
-#include "utility/StringUtil.hpp"
 
 #include "glog/logging.h"
 
@@ -202,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_ */