You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by ji...@apache.org on 2017/10/05 22:03:28 UTC

[33/51] [abbrv] incubator-quickstep git commit: Updates for adding generic types

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/TypeRegistrar.hpp
----------------------------------------------------------------------
diff --git a/types/TypeRegistrar.hpp b/types/TypeRegistrar.hpp
index ffe2b7e..9e6c50b 100644
--- a/types/TypeRegistrar.hpp
+++ b/types/TypeRegistrar.hpp
@@ -20,13 +20,14 @@
 #ifndef QUICKSTEP_TYPES_TYPE_REGISTRAR_HPP_
 #define QUICKSTEP_TYPES_TYPE_REGISTRAR_HPP_
 
+#include <cstddef>
 #include <cstdint>
 #include <type_traits>
+#include <vector>
 
 #include "types/DatetimeLit.hpp"
 #include "types/IntervalLit.hpp"
 #include "types/NullLit.hpp"
-#include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypeIDSelectors.hpp"
 #include "utility/meta/Common.hpp"
@@ -39,6 +40,14 @@ namespace quickstep {
  *  @{
  */
 
+class Type;
+class TypedValue;
+
+using UntypedLiteral = void;
+
+using ArrayLiteral = std::vector<UntypedLiteral*>;
+using MetaTypeLiteral = const Type*;
+
 template <TypeID type_id>
 struct TypeIDTrait;
 
@@ -48,36 +57,40 @@ struct TypeIDTrait;
     typedef type_class TypeClass; \
     typedef cpp_type cpptype; \
     static constexpr TypeID kStaticTypeID = type_id; \
-    static constexpr Type::SuperTypeID kStaticSuperTypeID = super_type_id; \
+    static constexpr SuperTypeID kStaticSuperTypeID = super_type_id; \
     static constexpr MemoryLayout kMemoryLayout = memory_layout; \
-    static constexpr bool kIsParameterizedPod = \
-        (memory_layout == kParNativePod || memory_layout == kParIndirectPod); \
+    static constexpr bool kIsParPod = \
+        (memory_layout == kParInlinePod || memory_layout == kParOutOfLinePod); \
   };
 
-REGISTER_TYPE(BoolType, kBool, \
-              Type::kNumeric, kCxxNativePod, bool);
-REGISTER_TYPE(IntType, kInt, \
-              Type::kNumeric, kCxxNativePod, int);
-REGISTER_TYPE(LongType, kLong, \
-              Type::kNumeric, kCxxNativePod, std::int64_t);
-REGISTER_TYPE(FloatType, kFloat, \
-              Type::kNumeric, kCxxNativePod, float);
-REGISTER_TYPE(DoubleType, kDouble, \
-              Type::kNumeric, kCxxNativePod, double);
-REGISTER_TYPE(DateType, kDate, \
-              Type::kOther, kCxxNativePod, DateLit);
-REGISTER_TYPE(DatetimeType, kDatetime, \
-              Type::kOther, kCxxNativePod, DatetimeLit);
-REGISTER_TYPE(DatetimeIntervalType, kDatetimeInterval, \
-              Type::kOther, kCxxNativePod, DatetimeIntervalLit);
-REGISTER_TYPE(YearMonthIntervalType, kYearMonthInterval, \
-              Type::kOther, kCxxNativePod, YearMonthIntervalLit);
-REGISTER_TYPE(CharType, kChar, \
-              Type::kAsciiString, kParNativePod, void);
-REGISTER_TYPE(VarCharType, kVarChar, \
-              Type::kAsciiString, kParIndirectPod, void);
-REGISTER_TYPE(NullType, kNullType, \
-              Type::kOther, kCxxNativePod, NullLit);
+REGISTER_TYPE(BoolType, kBool,
+              SuperTypeID::kNumeric, kCxxInlinePod, bool);
+REGISTER_TYPE(IntType, kInt,
+              SuperTypeID::kNumeric, kCxxInlinePod, int);
+REGISTER_TYPE(LongType, kLong,
+              SuperTypeID::kNumeric, kCxxInlinePod, std::int64_t);
+REGISTER_TYPE(FloatType, kFloat,
+              SuperTypeID::kNumeric, kCxxInlinePod, float);
+REGISTER_TYPE(DoubleType, kDouble,
+              SuperTypeID::kNumeric, kCxxInlinePod, double);
+REGISTER_TYPE(DateType, kDate,
+              SuperTypeID::kOther, kCxxInlinePod, DateLit);
+REGISTER_TYPE(DatetimeType, kDatetime,
+              SuperTypeID::kOther, kCxxInlinePod, DatetimeLit);
+REGISTER_TYPE(DatetimeIntervalType, kDatetimeInterval,
+              SuperTypeID::kOther, kCxxInlinePod, DatetimeIntervalLit);
+REGISTER_TYPE(YearMonthIntervalType, kYearMonthInterval,
+              SuperTypeID::kOther, kCxxInlinePod, YearMonthIntervalLit);
+REGISTER_TYPE(CharType, kChar,
+              SuperTypeID::kAsciiString, kParInlinePod, TypedValue);
+REGISTER_TYPE(VarCharType, kVarChar,
+              SuperTypeID::kAsciiString, kParOutOfLinePod, TypedValue);
+REGISTER_TYPE(ArrayType, kArray,
+              SuperTypeID::kOther, kCxxGeneric, ArrayLiteral);
+REGISTER_TYPE(MetaType, kMetaType,
+              SuperTypeID::kOther, kCxxGeneric, MetaTypeLiteral);
+REGISTER_TYPE(NullType, kNullType,
+              SuperTypeID::kOther, kCxxInlinePod, NullLit);
 
 #undef REGISTER_TYPE
 
@@ -91,8 +104,8 @@ 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) {
+inline auto InvokeOnTypeIDInternal(const int value,
+                                   const FunctorT &functor) {
   DCHECK_LE(l, r);
   if (l == r) {
     constexpr TypeID type_id = static_cast<TypeID>(r);
@@ -101,9 +114,9 @@ inline auto InvokeOnTypeIDInner(const int value,
   }
   constexpr int m = (l + r) >> 1;
   if (value <= m) {
-    return InvokeOnTypeIDInner<l, m, Selector, FunctorT>(value, functor);
+    return InvokeOnTypeIDInternal<l, m, Selector, FunctorT>(value, functor);
   } else {
-    return InvokeOnTypeIDInner<m+1, r, Selector, FunctorT>(value, functor);
+    return InvokeOnTypeIDInternal<m+1, r, Selector, FunctorT>(value, functor);
   }
 }
 
@@ -112,8 +125,8 @@ inline auto InvokeOnTypeIDInner(const int value,
 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>(
+  return internal::InvokeOnTypeIDInternal<0, static_cast<int>(kNumTypeIDs)-1,
+                                          Selector, FunctorT>(
       static_cast<int>(type_id), functor);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/TypeSynthesizer.hpp
----------------------------------------------------------------------
diff --git a/types/TypeSynthesizer.hpp b/types/TypeSynthesizer.hpp
index 16d59c9..7c39e47 100644
--- a/types/TypeSynthesizer.hpp
+++ b/types/TypeSynthesizer.hpp
@@ -21,14 +21,21 @@
 #define QUICKSTEP_TYPES_TYPE_SYNTHESIZER_HPP_
 
 #include <cstddef>
+#include <cstdio>
+#include <memory>
+#include <string>
 #include <type_traits>
+#include <unordered_map>
+#include <vector>
 
+#include "types/GenericValue.hpp"
 #include "types/Type.hpp"
 #include "types/Type.pb.h"
 #include "types/TypeID.hpp"
 #include "types/TypeRegistrar.hpp"
+#include "types/TypedValue.hpp"
+#include "utility/HashPair.hpp"
 #include "utility/Macros.hpp"
-#include "utility/PtrMap.hpp"
 
 #include "glog/logging.h"
 
@@ -38,25 +45,26 @@ namespace quickstep {
  *  @{
  */
 
-template <typename TypeClass, bool parameterized>
-class TypeInstance;
+template <TypeID type_id, typename Enable = void>
+class TypeInstancePolicy;
 
 
 template <TypeID type_id>
 class TypeSynthesizer
     : public Type,
-      public TypeInstance<typename TypeIDTrait<type_id>::TypeClass,
-                          TypeIDTrait<type_id>::kIsParameterizedPod> {
- public:
+      public TypeInstancePolicy<type_id> {
+ private:
   using Trait = TypeIDTrait<type_id>;
-  using TypeClass = typename Trait::TypeClass;
+  using InstancePolicy = TypeInstancePolicy<type_id>;
 
-  static constexpr Type::SuperTypeID kStaticSuperTypeID = Trait::kStaticSuperTypeID;
+ public:
+  static constexpr SuperTypeID kStaticSuperTypeID = Trait::kStaticSuperTypeID;
   static constexpr TypeID kStaticTypeID = Trait::kStaticTypeID;
-  static constexpr bool kIsParameterizedPod = Trait::kIsParameterizedPod;
+  static constexpr bool kIsParPod = Trait::kIsParPod;
   static constexpr MemoryLayout kMemoryLayout = Trait::kMemoryLayout;
 
-  typedef typename Trait::cpptype cpptype;
+  using TypeClass = typename Trait::TypeClass;
+  using cpptype = typename Trait::cpptype;
 
   serialization::Type getProto() const override {
     serialization::Type proto;
@@ -64,72 +72,170 @@ class TypeSynthesizer
     proto.mutable_type_id()->CopyFrom(TypeIDFactory::GetProto(type_id_));
     proto.set_nullable(nullable_);
 
-    if (kIsParameterizedPod) {
-      proto.set_length(parameter_);
-    }
+    InstancePolicy::fillProto(&proto);
 
     return proto;
   }
 
   const Type& getNullableVersion() const override {
-    return getInstance<kIsParameterizedPod>(true);
+    return InstancePolicy::getInstance(true);
   }
 
   const Type& getNonNullableVersion() const override {
-    return getInstance<kIsParameterizedPod>(false);
+    return InstancePolicy::getInstance(false);
+  }
+
+  const cpptype& castValueToLiteral(const UntypedLiteral *value) const {
+    return *static_cast<const cpptype*>(value);
+  }
+
+  cpptype& castValueToLiteral(UntypedLiteral *value) const {
+    return *static_cast<cpptype*>(value);
+  }
+
+  UntypedLiteral* unmarshallValue(const TypedValue &value) const override {
+    return unmarshallInternal<kMemoryLayout>(value);
+  }
+
+  UntypedLiteral* unmarshallValue(TypedValue &&value) const override {
+    return unmarshallInternal<kMemoryLayout>(std::move(value));
+  }
+
+  std::string printTypedValueToString(const TypedValue &value) const override {
+    return invokeOnUnmarshalledValue<kMemoryLayout>(
+        value,
+        [&](const UntypedLiteral *value) -> std::string {
+      return this->printValueToString(value);
+    });
+  }
+
+  void printTypedValueToFile(const TypedValue &value,
+                             FILE *file,
+                             const int padding = 0) const override {
+    invokeOnUnmarshalledValue<kMemoryLayout>(
+        value,
+        [&](const UntypedLiteral *value) -> void {
+      this->printValueToFile(value, file, padding);
+    });
   }
 
  protected:
-  template <MemoryLayout layout = kMemoryLayout, bool par = kIsParameterizedPod>
+  template <MemoryLayout layout = kMemoryLayout>
   explicit TypeSynthesizer(const bool nullable,
-                           std::enable_if_t<layout == kCxxNativePod>* = 0)
+                           std::enable_if_t<layout == kCxxInlinePod>* = 0)
       : Type(kStaticSuperTypeID, kStaticTypeID, nullable,
-             sizeof(cpptype), sizeof(cpptype)) {
+             sizeof(cpptype), sizeof(cpptype)),
+        TypeInstancePolicy<type_id>() {
   }
 
-  template <MemoryLayout layout = kMemoryLayout, bool par = kIsParameterizedPod>
+  template <MemoryLayout layout = kMemoryLayout>
   TypeSynthesizer(const bool nullable,
                   const std::size_t minimum_byte_length,
                   const std::size_t maximum_byte_length,
                   const std::size_t parameter,
-                  std::enable_if_t<par>* = 0)
+                  std::enable_if_t<layout == kParInlinePod || layout == kParOutOfLinePod>* = 0)
+      : Type(kStaticSuperTypeID, kStaticTypeID, nullable,
+             minimum_byte_length, maximum_byte_length),
+        TypeInstancePolicy<type_id>(parameter) {
+  }
+
+  template <MemoryLayout layout = kMemoryLayout>
+  TypeSynthesizer(const bool nullable,
+                  const std::size_t minimum_byte_length,
+                  const std::size_t maximum_byte_length,
+                  const std::vector<GenericValue> &parameters = {},
+                  std::enable_if_t<layout == kCxxGeneric>* = 0)
       : Type(kStaticSuperTypeID, kStaticTypeID, nullable,
-             minimum_byte_length, maximum_byte_length, parameter) {
+             minimum_byte_length, maximum_byte_length),
+        TypeInstancePolicy<type_id>(parameters) {
   }
 
  private:
-  template <bool has_param>
-  inline const Type& getInstance(const bool nullable,
-                                 std::enable_if_t<has_param>* = 0) const {
-    return TypeInstance<TypeClass, kIsParameterizedPod>::Instance(parameter_, nullable);
+  template <MemoryLayout layout>
+  inline UntypedLiteral* unmarshallInternal(
+      const TypedValue &value,
+      std::enable_if_t<layout == kCxxInlinePod> * = 0) const {
+    return cloneValue(value.getDataPtr());
   }
 
-  template <bool has_param>
-  inline const Type& getInstance(const bool nullable,
-                                 std::enable_if_t<!has_param>* = 0) const {
-    return TypeInstance<TypeClass, kIsParameterizedPod>::Instance(nullable);
+  template <MemoryLayout layout>
+  inline UntypedLiteral* unmarshallInternal(
+      const TypedValue &value,
+      std::enable_if_t<layout == kParInlinePod ||
+                       layout == kParOutOfLinePod> * = 0) const {
+    return cloneValue(&value);
   }
 
-  friend class TypeInstance<TypeClass, kIsParameterizedPod>;
+  template <MemoryLayout layout>
+  inline UntypedLiteral* unmarshallInternal(
+      TypedValue &&value,
+      std::enable_if_t<layout == kParInlinePod ||
+                       layout == kParOutOfLinePod> * = 0) const {
+    return new TypedValue(std::move(value));
+  }
+
+  template <MemoryLayout layout>
+  inline UntypedLiteral* unmarshallInternal(
+      const TypedValue &value,
+      std::enable_if_t<layout == kCxxGeneric> * = 0) const {
+    return Type::unmarshallValue(value.getOutOfLineData(), value.getDataSize());
+  }
+
+
+  template <MemoryLayout layout, typename Functor>
+  inline auto invokeOnUnmarshalledValue(
+      const TypedValue &value,
+      const Functor &functor,
+      std::enable_if_t<layout == kCxxInlinePod> * = 0) const {
+    return functor(value.getDataPtr());
+  }
+
+  template <MemoryLayout layout, typename Functor>
+  inline auto invokeOnUnmarshalledValue(
+      const TypedValue &value,
+      const Functor &functor,
+      std::enable_if_t<layout == kParInlinePod ||
+                       layout == kParOutOfLinePod> * = 0) const {
+    return functor(&value);
+  }
+
+  template <MemoryLayout layout, typename Functor>
+  inline auto invokeOnUnmarshalledValue(
+      const TypedValue &value,
+      const Functor &functor,
+      std::enable_if_t<layout == kCxxGeneric> * = 0) const {
+    std::unique_ptr<cpptype> literal(
+        static_cast<cpptype*>(Type::unmarshallValue(value.getOutOfLineData(),
+                                                    value.getDataSize())));
+    return functor(literal.get());
+  }
+
+  template <TypeID, typename> friend class TypeInstancePolicy;
 
   DISALLOW_COPY_AND_ASSIGN(TypeSynthesizer);
 };
 
 template <TypeID type_id>
-constexpr Type::SuperTypeID TypeSynthesizer<type_id>::kStaticSuperTypeID;
+constexpr SuperTypeID TypeSynthesizer<type_id>::kStaticSuperTypeID;
 
 template <TypeID type_id>
 constexpr TypeID TypeSynthesizer<type_id>::kStaticTypeID;
 
 template <TypeID type_id>
-constexpr bool TypeSynthesizer<type_id>::kIsParameterizedPod;
+constexpr bool TypeSynthesizer<type_id>::kIsParPod;
 
 template <TypeID type_id>
 constexpr MemoryLayout TypeSynthesizer<type_id>::kMemoryLayout;
 
 
-template <typename TypeClass>
-class TypeInstance<TypeClass, false> {
+template <TypeID type_id>
+class TypeInstancePolicy<
+    type_id,
+    std::enable_if_t<TypeIDTrait<type_id>::kMemoryLayout == kCxxInlinePod>> {
+ private:
+  using Trait = TypeIDTrait<type_id>;
+  using TypeClass = typename Trait::TypeClass;
+
  public:
   static const TypeClass& InstanceNonNullable() {
     return InstanceInternal<false>();
@@ -147,6 +253,15 @@ class TypeInstance<TypeClass, false> {
     }
   }
 
+ protected:
+  TypeInstancePolicy() {}
+
+  inline const Type& getInstance(const bool nullable) const {
+    return nullable ? InstanceNullable() : InstanceNonNullable();
+  }
+
+  inline void fillProto(serialization::Type *proto) const {}
+
  private:
   template <bool nullable>
   inline static const TypeClass& InstanceInternal() {
@@ -155,8 +270,15 @@ class TypeInstance<TypeClass, false> {
   }
 };
 
-template <typename TypeClass>
-class TypeInstance<TypeClass, true> {
+template <TypeID type_id>
+class TypeInstancePolicy<
+    type_id,
+    std::enable_if_t<TypeIDTrait<type_id>::kMemoryLayout == kParInlinePod ||
+                     TypeIDTrait<type_id>::kMemoryLayout == kParOutOfLinePod>> {
+ private:
+  using Trait = TypeIDTrait<type_id>;
+  using TypeClass = typename Trait::TypeClass;
+
  public:
   static const TypeClass& InstanceNonNullable(const std::size_t length) {
     return InstanceInternal<false>(length);
@@ -174,18 +296,134 @@ class TypeInstance<TypeClass, true> {
     }
   }
 
+  inline std::size_t length() const {
+    return length_;
+  }
+
+ protected:
+  TypeInstancePolicy(const std::size_t length)
+      : length_(length) {}
+
+  const std::size_t length_;
+
+  inline const Type& getInstance(const bool nullable) const {
+    return nullable ? InstanceNullable(length_) : InstanceNonNullable(length_);
+  }
+
+  inline void fillProto(serialization::Type *proto) const {
+    proto->set_length(length_);
+  }
+
  private:
   template <bool nullable>
   inline static const TypeClass& InstanceInternal(const std::size_t length) {
-    static PtrMap<size_t, TypeClass> instance_map;
+    static std::unordered_map<size_t, std::unique_ptr<TypeClass>> instance_map;
     auto imit = instance_map.find(length);
     if (imit == instance_map.end()) {
-      imit = instance_map.insert(length, new TypeClass(length, nullable)).first;
+      std::unique_ptr<TypeClass> instance(new TypeClass(nullable, length));
+      imit = instance_map.emplace(length, std::move(instance)).first;
+    }
+    return *(imit->second);
+  }
+};
+
+template <TypeID type_id>
+class TypeInstancePolicy<
+    type_id,
+    std::enable_if_t<TypeIDTrait<type_id>::kMemoryLayout == kCxxGeneric>> {
+ private:
+  using Trait = TypeIDTrait<type_id>;
+  using TypeClass = typename Trait::TypeClass;
+
+ public:
+  static const TypeClass& InstanceNonNullable(
+      const std::vector<GenericValue> &parameters = {}) {
+    return InstanceInternal<false>(parameters);
+  }
+
+  static const TypeClass& InstanceNullable(
+      const std::vector<GenericValue> &parameters = {}) {
+    return InstanceInternal<true>(parameters);
+  }
+
+  static const TypeClass& Instance(
+      const bool nullable,
+      const std::vector<GenericValue> &parameters = {}) {
+    if (nullable) {
+      return InstanceNullable(parameters);
+    } else {
+      return InstanceNonNullable(parameters);
+    }
+  }
+
+  inline const std::vector<GenericValue>& parameters() const {
+    return parameters_;
+  }
+
+ protected:
+  TypeInstancePolicy(const std::vector<GenericValue> &parameters)
+      : parameters_(parameters) {}
+
+  const std::vector<GenericValue> parameters_;
+
+  inline const Type& getInstance(const bool nullable) const {
+    return nullable ? InstanceNullable(parameters_)
+                    : InstanceNonNullable(parameters_);
+  }
+
+  inline void fillProto(serialization::Type *proto) const {
+    LOG(FATAL) << "TODO";
+  }
+
+ private:
+  struct ParametersHasher {
+    inline std::size_t operator()(const std::vector<GenericValue> &parameters) const {
+      std::size_t hash_code = 0;
+      for (const GenericValue &value : parameters) {
+        hash_code = CombineHashes(hash_code, value.getHash());
+      }
+      return hash_code;
+    }
+  };
+
+  template <typename T>
+  inline static TypeClass* CreateInstance(
+      const bool nullable,
+      const std::vector<GenericValue> &parameters,
+      decltype(new T(nullable)) * = 0) {
+    DCHECK(parameters.empty());
+    return new T(nullable);
+  }
+
+  template <typename T>
+  inline static TypeClass* CreateInstance(
+      const bool nullable,
+      const std::vector<GenericValue> &parameters,
+      decltype(new T(nullable, parameters)) * = 0) {
+    return new T(nullable, parameters);
+  }
+
+  template <bool nullable>
+  inline static const TypeClass& InstanceInternal(
+      const std::vector<GenericValue> &parameters) {
+    static std::unordered_map<std::vector<GenericValue>,
+                              std::unique_ptr<TypeClass>,
+                              ParametersHasher> instance_map;
+    auto imit = instance_map.find(parameters);
+    if (imit == instance_map.end()) {
+      std::unique_ptr<TypeClass> instance(
+          TypeInstancePolicy<type_id>::template CreateInstance<TypeClass>(
+              nullable, parameters));
+      imit = instance_map.emplace(parameters, std::move(instance)).first;
     }
     return *(imit->second);
   }
 };
 
+#define QUICKSTEP_SYNTHESIZE_TYPE(type) \
+  template <TypeID, typename> friend class TypeInstancePolicy; \
+  DISALLOW_COPY_AND_ASSIGN(type)
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/TypeUtil.hpp
----------------------------------------------------------------------
diff --git a/types/TypeUtil.hpp b/types/TypeUtil.hpp
index 5a55280..3b16c70 100644
--- a/types/TypeUtil.hpp
+++ b/types/TypeUtil.hpp
@@ -22,6 +22,7 @@
 
 #include <type_traits>
 
+#include "types/ArrayType.hpp"
 #include "types/BoolType.hpp"
 #include "types/CharType.hpp"
 #include "types/DateType.hpp"
@@ -31,6 +32,7 @@
 #include "types/FloatType.hpp"
 #include "types/IntType.hpp"
 #include "types/LongType.hpp"
+#include "types/MetaType.hpp"
 #include "types/NullType.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
@@ -53,7 +55,7 @@ class TypeUtil {
     return InvokeOnTypeID(
         type_id,
         [&](auto tid) -> bool {  // NOLINT(build/c++11)
-      return TypeIDTrait<decltype(tid)::value>::kIsParameterizedPod;
+      return TypeIDTrait<decltype(tid)::value>::kIsParPod;
     });
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/TypedValue.hpp
----------------------------------------------------------------------
diff --git a/types/TypedValue.hpp b/types/TypedValue.hpp
index 196e8ec..d0e8b98 100644
--- a/types/TypedValue.hpp
+++ b/types/TypedValue.hpp
@@ -506,6 +506,28 @@ class TypedValue {
     return value_union_.out_of_line_data;
   }
 
+  inline void* releaseOutOfLineData() {
+    DCHECK(!(getTypeID() == kBool
+                   || getTypeID() == kInt
+                   || getTypeID() == kLong
+                   || getTypeID() == kFloat
+                   || getTypeID() == kDouble
+                   || getTypeID() == kDate
+                   || getTypeID() == kDatetime
+                   || getTypeID() == kDatetimeInterval
+                   || getTypeID() == kYearMonthInterval));
+    DCHECK(!isNull());
+    if (ownsOutOfLineData()) {
+      value_info_ &= ~kOwnershipMask;
+      return const_cast<void*>(value_union_.out_of_line_data);
+    } else {
+      const std::size_t length = value_info_ >> kSizeShift;
+      void *data = std::malloc(length);
+      std::memcpy(data, value_union_.out_of_line_data, length);
+      return data;
+    }
+  }
+
   /**
    * @brief Get the length of the ASCII string this TypedValue represents,
    *        not counting a null terminator character, if any (same behavior

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/VarCharType.cpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.cpp b/types/VarCharType.cpp
index 7eeb04c..f0f677d 100644
--- a/types/VarCharType.cpp
+++ b/types/VarCharType.cpp
@@ -86,21 +86,21 @@ string VarCharType::getName() const {
   return name;
 }
 
-std::string VarCharType::printValueToString(const TypedValue &value) const {
-  DCHECK(!value.isNull());
+std::string VarCharType::printValueToString(const UntypedLiteral *value) const {
+  DCHECK(value != nullptr);
 
-  return std::string(static_cast<const char*>(value.getOutOfLineData()));
+  return std::string(static_cast<const char*>(castValueToLiteral(value).getOutOfLineData()));
 }
 
-void VarCharType::printValueToFile(const TypedValue &value,
+void VarCharType::printValueToFile(const UntypedLiteral *value,
                                    FILE *file,
                                    const int padding) const {
-  DCHECK(!value.isNull());
+  DCHECK(value != nullptr);
 
   std::fprintf(file,
                "%*s",
                static_cast<int>(padding),
-               static_cast<const char*>(value.getOutOfLineData()));
+               static_cast<const char*>(castValueToLiteral(value).getOutOfLineData()));
 }
 
 bool VarCharType::parseValueFromString(const std::string &value_string,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/VarCharType.hpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.hpp b/types/VarCharType.hpp
index 05b2aae..47a2874 100644
--- a/types/VarCharType.hpp
+++ b/types/VarCharType.hpp
@@ -60,9 +60,9 @@ class VarCharType : public AsciiStringSuperType<kVarChar> {
     return length_;
   }
 
-  std::string printValueToString(const TypedValue &value) const override;
+  std::string printValueToString(const UntypedLiteral *value) const override;
 
-  void printValueToFile(const TypedValue &value,
+  void printValueToFile(const UntypedLiteral *value,
                         FILE *file,
                         const int padding = 0) const override;
 
@@ -73,12 +73,10 @@ class VarCharType : public AsciiStringSuperType<kVarChar> {
                          const Type &original_type) const override;
 
  private:
-  VarCharType(const std::size_t length, const bool nullable)
+  VarCharType(const bool nullable, const std::size_t length)
       : AsciiStringSuperType<kVarChar>(nullable, 1, length + 1, length) {}
 
-  template <typename, bool> friend class TypeInstance;
-
-  DISALLOW_COPY_AND_ASSIGN(VarCharType);
+  QUICKSTEP_SYNTHESIZE_TYPE(VarCharType);
 };
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/YearMonthIntervalType.cpp
----------------------------------------------------------------------
diff --git a/types/YearMonthIntervalType.cpp b/types/YearMonthIntervalType.cpp
index d656fca..b395dff 100644
--- a/types/YearMonthIntervalType.cpp
+++ b/types/YearMonthIntervalType.cpp
@@ -45,10 +45,10 @@ using std::snprintf;
 
 namespace quickstep {
 
-std::string YearMonthIntervalType::printValueToString(const TypedValue &value) const {
-  DCHECK(!value.isNull());
+std::string YearMonthIntervalType::printValueToString(const UntypedLiteral *value) const {
+  DCHECK(value != nullptr);
 
-  std::int64_t months = value.getLiteral<YearMonthIntervalLit>().months;
+  std::int64_t months = castValueToLiteral(value).months;
   const bool negative_interval = months < 0;
   if (negative_interval) {
     months = -months;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/YearMonthIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/YearMonthIntervalType.hpp b/types/YearMonthIntervalType.hpp
index e890ea9..ab06911 100644
--- a/types/YearMonthIntervalType.hpp
+++ b/types/YearMonthIntervalType.hpp
@@ -46,7 +46,7 @@ class YearMonthIntervalType : public TypeSynthesizer<kYearMonthInterval> {
     return YearMonthIntervalLit::kPrintingChars;
   }
 
-  std::string printValueToString(const TypedValue &value) const override;
+  std::string printValueToString(const UntypedLiteral *value) const override;
 
   TypedValue makeZeroValue() const override {
     return TypedValue(YearMonthIntervalLit{0});
@@ -59,9 +59,7 @@ class YearMonthIntervalType : public TypeSynthesizer<kYearMonthInterval> {
   explicit YearMonthIntervalType(const bool nullable)
       : TypeSynthesizer<kYearMonthInterval>(nullable) {}
 
-  template <typename, bool> friend class TypeInstance;
-
-  DISALLOW_COPY_AND_ASSIGN(YearMonthIntervalType);
+  QUICKSTEP_SYNTHESIZE_TYPE(YearMonthIntervalType);
 };
 
 /** @} */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/containers/ColumnVector.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.hpp b/types/containers/ColumnVector.hpp
index a312ee2..d524ff5 100644
--- a/types/containers/ColumnVector.hpp
+++ b/types/containers/ColumnVector.hpp
@@ -280,7 +280,8 @@ class NativeColumnVector : public ColumnVector {
    **/
   inline void appendTypedValue(const TypedValue &value) {
     DCHECK_LT(actual_length_, reserved_length_);
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+//    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+    // TODO(refactor-type): fix signature.
     if (null_bitmap_ && value.isNull()) {
       null_bitmap_->setBit(actual_length_, true);
     } else {
@@ -319,7 +320,8 @@ class NativeColumnVector : public ColumnVector {
    * @param value A value to fill this ColumnVector with.
    **/
   inline void fillWithValue(const TypedValue &value) {
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+//    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+    // TODO(refactor-type): fix signature.
     if (value.isNull()) {
       fillWithNulls();
     } else {
@@ -408,7 +410,8 @@ class NativeColumnVector : public ColumnVector {
   inline void positionalWriteTypedValue(const std::size_t position,
                                         const TypedValue &value) {
     DCHECK_LT(position, actual_length_);
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+//    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+    // TODO(refactor-type): fix signature.
     if (null_bitmap_ && value.isNull()) {
       null_bitmap_->setBit(position, true);
     } else {
@@ -515,7 +518,8 @@ class IndirectColumnVector : public ColumnVector {
    * @param value A value to append to this NativeColumnVector.
    **/
   inline void appendTypedValue(const TypedValue &value) {
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+//    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+    // TODO(refactor-type): fix signature.
     DCHECK_LT(values_.size(), reserved_length_);
     values_.emplace_back(value);
   }
@@ -526,7 +530,8 @@ class IndirectColumnVector : public ColumnVector {
    * @param value A value to append to this NativeColumnVector.
    **/
   inline void appendTypedValue(TypedValue &&value) {
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature())) << type_.getName();
+//    DCHECK(value.isPlausibleInstanceOf(type_.getSignature())) << type_.getName();
+    // TODO(refactor-type): fix signature.
     DCHECK_LT(values_.size(), reserved_length_);
     values_.emplace_back(std::move(value));
   }
@@ -547,7 +552,8 @@ class IndirectColumnVector : public ColumnVector {
    * @param value A value to fill this ColumnVector with.
    **/
   inline void fillWithValue(const TypedValue &value) {
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+//    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+    // TODO(refactor-type): fix signature.
     values_.assign(reserved_length_, value);
   }
 
@@ -576,7 +582,8 @@ class IndirectColumnVector : public ColumnVector {
    **/
   inline void positionalWriteTypedValue(const std::size_t position,
                                         const TypedValue &value) {
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+//    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+    // TODO(refactor-type): fix signature.
     DCHECK_LT(position, values_.size());
     values_[position] = value;
   }
@@ -594,7 +601,8 @@ class IndirectColumnVector : public ColumnVector {
    **/
   inline void positionalWriteTypedValue(const std::size_t position,
                                         TypedValue &&value) {  // NOLINT(whitespace/operators)
-    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+//    DCHECK(value.isPlausibleInstanceOf(type_.getSignature()));
+    // TODO(refactor-type): fix signature.
     DCHECK_LT(position, values_.size());
     values_[position] = std::move(value);
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/operations/OperationUtil.hpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationUtil.hpp b/types/operations/OperationUtil.hpp
index 8290061..af23b1c 100644
--- a/types/operations/OperationUtil.hpp
+++ b/types/operations/OperationUtil.hpp
@@ -17,8 +17,8 @@
  * under the License.
  **/
 
-#ifndef QUICKSTEP_TYPES_OPERATIONS_OPERATION_UTIL_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_OPERATION_UTIL_HPP_
+#ifndef QUICKSTEP_TYPES_OPERATIONS_OPERATION_SYNTHESIZE_UTIL_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_OPERATION_SYNTHESIZE_UTIL_HPP_
 
 #include <cstddef>
 #include <list>
@@ -103,7 +103,7 @@ template <typename FuncSpec, typename T, typename EnableT = void>
 struct Codegen;
 
 template <typename FuncSpec, typename T>
-struct Codegen<FuncSpec, T, std::enable_if_t<T::kMemoryLayout == kCxxNativePod>> {
+struct Codegen<FuncSpec, T, std::enable_if_t<T::kMemoryLayout == kCxxInlinePod>> {
   using ColumnVectorType = NativeColumnVector;
   using FunctorSpecializer = FuncSpec;
 
@@ -171,7 +171,7 @@ struct Codegen<FuncSpec, T, std::enable_if_t<T::kMemoryLayout == kCxxNativePod>>
 };
 
 template <typename FuncSpec, typename T>
-struct Codegen<FuncSpec, T, std::enable_if_t<T::kMemoryLayout == kParNativePod>> {
+struct Codegen<FuncSpec, T, std::enable_if_t<T::kMemoryLayout == kParInlinePod>> {
   using ColumnVectorType = NativeColumnVector;
   using FunctorSpecializer = FuncSpec;
 
@@ -244,7 +244,7 @@ struct Codegen<FuncSpec, T, std::enable_if_t<T::kMemoryLayout == kParNativePod>>
 };
 
 template <typename FuncSpec, typename T>
-struct Codegen<FuncSpec, T, std::enable_if_t<T::kMemoryLayout == kParIndirectPod>> {
+struct Codegen<FuncSpec, T, std::enable_if_t<T::kMemoryLayout == kParOutOfLinePod>> {
   using ColumnVectorType = IndirectColumnVector;
   using FunctorSpecializer = FuncSpec;
 
@@ -331,4 +331,4 @@ struct OperationPack {
 
 }  // namespace quickstep
 
-#endif  // QUICKSTEP_TYPES_OPERATIONS_OPERATION_UTIL_HPP_
+#endif  // QUICKSTEP_TYPES_OPERATIONS_OPERATION_SYNTHESIZE_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/operations/binary_operations/BinaryOperationWrapper.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationWrapper.hpp b/types/operations/binary_operations/BinaryOperationWrapper.hpp
index d819000..f5edf52 100644
--- a/types/operations/binary_operations/BinaryOperationWrapper.hpp
+++ b/types/operations/binary_operations/BinaryOperationWrapper.hpp
@@ -195,7 +195,7 @@ class UncheckedBinaryOperatorWrapperCodegen : public UncheckedBinaryOperator {
       std::size_t *num_tuples_applied) const override {
     constexpr bool is_supported =
         LeftType::kStaticTypeID == ResultType::kStaticTypeID &&
-        LeftType::kMemoryLayout == kCxxNativePod &&
+        LeftType::kMemoryLayout == kCxxInlinePod &&
         std::is_copy_assignable<typename LeftType::cpptype>::value;
 
     using RightCVT = typename RightGen::ColumnVectorType;
@@ -217,7 +217,7 @@ class UncheckedBinaryOperatorWrapperCodegen : public UncheckedBinaryOperator {
       std::size_t *num_tuples_applied) const override {
     constexpr bool is_supported =
         LeftType::kStaticTypeID == ResultType::kStaticTypeID &&
-        LeftType::kMemoryLayout == kCxxNativePod &&
+        LeftType::kMemoryLayout == kCxxInlinePod &&
         std::is_copy_assignable<typename LeftType::cpptype>::value;
 
     return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
@@ -511,7 +511,7 @@ class BinaryOperationWrapper : public BinaryOperation {
     DCHECK(left.getTypeID() == LeftType::kStaticTypeID);
     DCHECK(right.getTypeID() == RightType::kStaticTypeID);
     DCHECK(static_arguments.empty());
-    return getResultTypeImpl<ResultType::kIsParameterizedPod>(
+    return getResultTypeImpl<ResultType::kIsParPod>(
         left, right, static_arguments);
   }
 

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/operations/unary_operations/CastOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CastOperation.cpp b/types/operations/unary_operations/CastOperation.cpp
index 5091e89..8e0db55 100644
--- a/types/operations/unary_operations/CastOperation.cpp
+++ b/types/operations/unary_operations/CastOperation.cpp
@@ -62,7 +62,7 @@ class CastToAsciiStringFunctor : public UnaryFunctor<ArgumentT, ResultT> {
         max_string_length_(max_string_length) {}
 
   inline void apply(const typename ArgumentT::cpptype &argument, void *result) const {
-    std::string str = argument_type_.printValueToString(TypedValue(argument));
+    std::string str = argument_type_.printValueToString(&argument);
     const std::size_t str_len = str.length();
 
     if (str_len < max_string_length_) {
@@ -74,7 +74,7 @@ class CastToAsciiStringFunctor : public UnaryFunctor<ArgumentT, ResultT> {
   }
 
   inline TypedValue apply(const typename ArgumentT::cpptype &argument) const {
-    std::string str = argument_type_.printValueToString(TypedValue(argument));
+    std::string str = argument_type_.printValueToString(&argument);
     const std::size_t len = std::min(str.length(), max_string_length_);
     const std::size_t buf_len = len + 1;
 
@@ -225,7 +225,7 @@ UncheckedUnaryOperator* CastOperation::makeUncheckedUnaryOperator(
         }
         case kChar:  // Fall through
         case kVarChar: {
-          return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
+          return InvokeOnTypeID<TypeIDSelector<kChar, kVarChar>>(
               result_type_id,
               [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
             using ResultT = typename TypeIDTrait<decltype(result_tid)::value>::TypeClass;
@@ -244,7 +244,7 @@ UncheckedUnaryOperator* CastOperation::makeUncheckedUnaryOperator(
       }
     });
   } else if (QUICKSTEP_EQUALS_ANY_CONSTANT(argument_type_id, kChar, kVarChar)) {
-    return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
+    return InvokeOnTypeID<TypeIDSelector<kChar, kVarChar>>(
         argument_type_id,
         [&](auto arg_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
       using ArgumentT = typename TypeIDTrait<decltype(arg_tid)::value>::TypeClass;
@@ -270,7 +270,7 @@ UncheckedUnaryOperator* CastOperation::makeUncheckedUnaryOperator(
         }
         case kChar:  // Fall through
         case kVarChar: {
-          return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
+          return InvokeOnTypeID<TypeIDSelector<kChar, kVarChar>>(
               result_type_id,
               [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
             using ResultT = typename TypeIDTrait<decltype(result_tid)::value>::TypeClass;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/types/operations/unary_operations/UnaryOperationWrapper.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationWrapper.hpp b/types/operations/unary_operations/UnaryOperationWrapper.hpp
index 09e7b05..00562a6 100644
--- a/types/operations/unary_operations/UnaryOperationWrapper.hpp
+++ b/types/operations/unary_operations/UnaryOperationWrapper.hpp
@@ -184,7 +184,7 @@ class UnaryOperationWrapper : public UnaryOperation {
       const std::vector<TypedValue> &static_arguments) const override {
     DCHECK(argument_type.getTypeID() == ArgumentType::kStaticTypeID);
     DCHECK(static_arguments.empty());
-    return getResultTypeImpl<ResultType::kIsParameterizedPodz>(
+    return getResultTypeImpl<ResultType::kIsParPod>(
         argument_type, static_arguments);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/utility/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/utility/CMakeLists.txt b/utility/CMakeLists.txt
index 9093910..573ea7e 100644
--- a/utility/CMakeLists.txt
+++ b/utility/CMakeLists.txt
@@ -163,6 +163,9 @@ add_subdirectory(meta)
 
 # Declare micro-libs:
 add_library(quickstep_utility_Alignment ../empty_src.cpp Alignment.hpp)
+add_library(quickstep_utility_BarrieredReadWriteConcurrentBitVector
+            ../empty_src.cpp
+            BarrieredReadWriteConcurrentBitVector.hpp)
 add_library(quickstep_utility_BitManipulation ../empty_src.cpp BitManipulation.hpp)
 add_library(quickstep_utility_BitVector ../empty_src.cpp BitVector.hpp)
 add_library(quickstep_utility_BloomFilter ../empty_src.cpp BloomFilter.hpp)
@@ -172,12 +175,10 @@ add_library(quickstep_utility_BloomFilter_proto
 add_library(quickstep_utility_BulkIoConfiguration BulkIoConfiguration.cpp BulkIoConfiguration.hpp)
 add_library(quickstep_utility_CalculateInstalledMemory CalculateInstalledMemory.cpp CalculateInstalledMemory.hpp)
 add_library(quickstep_utility_Cast ../empty_src.cpp Cast.hpp)
+add_library(quickstep_utility_CharStream ../empty_src.cpp CharStream.hpp)
 add_library(quickstep_utility_CheckSnprintf ../empty_src.cpp CheckSnprintf.hpp)
 add_library(quickstep_utility_ColumnVectorCache ../empty_src.cpp ColumnVectorCache.hpp)
 add_library(quickstep_utility_CompositeHash ../empty_src.cpp CompositeHash.hpp)
-add_library(quickstep_utility_BarrieredReadWriteConcurrentBitVector
-            ../empty_src.cpp
-            BarrieredReadWriteConcurrentBitVector.hpp)
 add_library(quickstep_utility_DAG ../empty_src.cpp DAG.hpp)
 add_library(quickstep_utility_DisjointTreeForest ../empty_src.cpp DisjointTreeForest.hpp)
 add_library(quickstep_utility_EqualsAnyConstant ../empty_src.cpp EqualsAnyConstant.hpp)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/8a3b2cf1/utility/CharStream.hpp
----------------------------------------------------------------------
diff --git a/utility/CharStream.hpp b/utility/CharStream.hpp
new file mode 100644
index 0000000..060b1a6
--- /dev/null
+++ b/utility/CharStream.hpp
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_UTILITY_CHAR_STREAM_HPP_
+#define QUICKSTEP_UTILITY_CHAR_STREAM_HPP_
+
+#include <cstddef>
+#include <cstdlib>
+#include <cstring>
+#include <type_traits>
+
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Utility
+ *  @{
+ */
+
+class CharStream {
+ public:
+  template <typename T>
+  CharStream(const T &value,
+             std::enable_if_t<std::is_pod<T>::value && sizeof(T) <= sizeof(std::uint64_t)> * = 0)
+      : length_(sizeof(T)),
+        delete_function_(nullptr) {
+    std::memcpy(&value_union_.inline_data, &value, sizeof(T));
+  }
+
+  CharStream(std::vector<char> &&value)
+      : length_(value.size()),
+        delete_function_(&DeleteObject<std::vector<char>>) {
+    value_union_.out_of_line_data = new std::vector<char>(std::move(value));
+  }
+
+  CharStream(const void *value, const std::size_t length, const bool take_ownership)
+      : length_(length),
+        delete_function_(std::free) {
+    if (take_ownership) {
+      value_union_.out_of_line_data = value;
+    } else {
+      void *copy_of_value = std::malloc(length);
+      std::memcpy(copy_of_value, value, length);
+      value_union_.out_of_line_data = copy_of_value;
+    }
+  }
+
+  ~CharStream() {
+    if (delete_function_ != nullptr) {
+      delete_function_(const_cast<void*>(value_union_.out_of_line_data));
+    }
+  }
+
+  std::size_t length() const {
+    return length_;
+  }
+
+  const void* getDataPtr() const {
+    return delete_function_ == nullptr ? &value_union_.inline_data
+                                       : value_union_.out_of_line_data;
+  }
+
+ private:
+  union ValueUnion {
+    std::uint64_t inline_data;
+    const void *out_of_line_data;
+  };
+
+  typedef void (*DeleterFunction)(void*);
+
+  template <typename T>
+  static void DeleteObject(void *object) {
+    delete static_cast<T*>(object);
+  }
+
+  std::size_t length_;
+  ValueUnion value_union_;
+  DeleterFunction delete_function_;
+
+  DISALLOW_COPY_AND_ASSIGN(CharStream);
+};
+
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_UTILITY_CHAR_STREAM_HPP_