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/05/04 03:23:08 UTC

[24/32] incubator-quickstep git commit: Refactor type system and operations.

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/AddBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/AddBinaryOperation.cpp b/types/operations/binary_operations/AddBinaryOperation.cpp
deleted file mode 100644
index 8f56a61..0000000
--- a/types/operations/binary_operations/AddBinaryOperation.cpp
+++ /dev/null
@@ -1,418 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#include "types/operations/binary_operations/AddBinaryOperation.hpp"
-
-#include <string>
-#include <utility>
-
-#include "types/DateOperatorOverloads.hpp"
-#include "types/DateType.hpp"
-#include "types/DatetimeIntervalType.hpp"
-#include "types/DatetimeLit.hpp"
-#include "types/DatetimeType.hpp"
-#include "types/IntervalLit.hpp"
-#include "types/Type.hpp"
-#include "types/TypeErrors.hpp"
-#include "types/TypeFactory.hpp"
-#include "types/TypeID.hpp"
-#include "types/YearMonthIntervalType.hpp"
-#include "types/operations/binary_operations/ArithmeticBinaryOperators.hpp"
-#include "utility/EqualsAnyConstant.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-bool AddBinaryOperation::canApplyToTypes(const Type &left, const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:  // Fall through.
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      return (right.getSuperTypeID() == Type::kNumeric);
-    }
-    case kDate: {
-      return (right.getTypeID() == kYearMonthInterval);
-    }
-    case kDatetime: {
-      return (right.getTypeID() == kDatetimeInterval ||
-              right.getTypeID() == kYearMonthInterval);
-    }
-    case kDatetimeInterval: {
-      return (right.getTypeID() == kDatetime ||
-              right.getTypeID() == kDatetimeInterval);
-    }
-    case kYearMonthInterval: {
-      return (right.getTypeID() == kDate ||
-              right.getTypeID() == kDatetime ||
-              right.getTypeID() == kYearMonthInterval);
-    }
-    default:
-      return false;
-  }
-}
-
-const Type* AddBinaryOperation::resultTypeForArgumentTypes(const Type &left, const Type &right) const {
-  if (left.getSuperTypeID() == Type::kNumeric && right.getSuperTypeID() == Type::kNumeric) {
-    return TypeFactory::GetUnifyingType(left, right);
-  } else if ((left.getTypeID() == kDatetime && right.getTypeID() == kDatetimeInterval)  ||
-             (left.getTypeID() == kDatetimeInterval && right.getTypeID() == kDatetime)  ||
-             (left.getTypeID() == kDatetime && right.getTypeID() == kYearMonthInterval) ||
-             (left.getTypeID() == kYearMonthInterval && right.getTypeID() == kDatetime)) {
-    return &(DatetimeType::Instance(left.isNullable() || right.isNullable()));
-  } else if ((left.getTypeID() == kDate && right.getTypeID() == kYearMonthInterval) ||
-             (left.getTypeID() == kYearMonthInterval && right.getTypeID() == kDate)) {
-    return &(DateType::Instance(left.isNullable() || right.isNullable()));
-  } else if (left.getTypeID() == kDatetimeInterval && right.getTypeID() == kDatetimeInterval) {
-    return &(DatetimeIntervalType::Instance(left.isNullable() || right.isNullable()));
-  } else if (left.getTypeID() == kYearMonthInterval && right.getTypeID() == kYearMonthInterval) {
-    return &(YearMonthIntervalType::Instance(left.isNullable() || right.isNullable()));
-  } else {
-    return nullptr;
-  }
-}
-
-const Type* AddBinaryOperation::resultTypeForPartialArgumentTypes(const Type *left,
-                                                                  const Type *right) const {
-  if ((left == nullptr) && (right == nullptr)) {
-    return nullptr;
-  }
-
-  if ((left != nullptr) && (right != nullptr)) {
-    return resultTypeForArgumentTypes(*left, *right);
-  }
-
-  // Addition is commutative, so we just determine based on the known type,
-  // left or right.
-  const Type *known_type = (left != nullptr) ? left : right;
-  switch (known_type->getTypeID()) {
-    case kDouble:
-      // Double has highest precedence of the numeric types.
-      return &TypeFactory::GetType(kDouble, true);
-    case kDatetime:
-      // Datetime can be added with either interval type, and always yields
-      // Datetime.
-      return &TypeFactory::GetType(kDatetime, true);
-    case kDate:
-      // Date can be added with YearMonthInterval type only, and always yields
-      // Date.
-      return &TypeFactory::GetType(kDate, true);
-    default:
-      // Ambiguous or inapplicable.
-      return nullptr;
-  }
-}
-
-bool AddBinaryOperation::partialTypeSignatureIsPlausible(
-    const Type *result_type,
-    const Type *left_argument_type,
-    const Type *right_argument_type) const {
-  if ((left_argument_type == nullptr) && (right_argument_type == nullptr)) {
-    if (result_type == nullptr) {
-      return true;
-    } else if (!result_type->isNullable()) {
-      // Unknown arguments are assumed to be nullable, since they arise from
-      // untyped NULL literals in the parser. Therefore, a non-nullable result
-      // Type is not plausible with unknown arguments.
-      return false;
-    } else {
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(result_type->getTypeID(),
-                                           kInt,
-                                           kLong,
-                                           kFloat,
-                                           kDouble,
-                                           kDate,
-                                           kDatetime,
-                                           kDatetimeInterval,
-                                           kYearMonthInterval);
-    }
-  }
-
-  if ((left_argument_type != nullptr) && (right_argument_type != nullptr)) {
-    const Type *actual_result_type = resultTypeForArgumentTypes(*left_argument_type,
-                                                                *right_argument_type);
-    if (actual_result_type == nullptr) {
-      // Both argument Types are known, but this operation is NOT applicable to
-      // them. No matter what the result_type is, the signature is not
-      // plausible.
-      return false;
-    } else if (result_type == nullptr) {
-      return true;
-    } else {
-      return result_type->equals(*actual_result_type);
-    }
-  }
-
-  // Addition is commutative, so we just determine based on the known type,
-  // left or right.
-  const Type *known_argument_type = (left_argument_type != nullptr)
-                                    ? left_argument_type
-                                    : right_argument_type;
-  if (result_type == nullptr) {
-    return QUICKSTEP_EQUALS_ANY_CONSTANT(known_argument_type->getTypeID(),
-                                         kInt,
-                                         kLong,
-                                         kFloat,
-                                         kDouble,
-                                         kDate,
-                                         kDatetime,
-                                         kDatetimeInterval,
-                                         kYearMonthInterval);
-  }
-
-  if (!result_type->isNullable()) {
-    // One of the arguments is unknown, but it is nevertheless assumed
-    // nullable, since unknown argument Types arise from untyped NULL literals
-    // in the parser. Therefore, a non-nullable result Type is not plausible
-    // with an unknown argument.
-    return false;
-  }
-
-  switch (result_type->getTypeID()) {
-    case kInt:
-      return (known_argument_type->getTypeID() == kInt);
-    case kLong:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kInt, kLong);
-    case kFloat:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kInt, kFloat);
-    case kDouble:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kInt, kLong, kFloat, kDouble);
-    case kDate:
-      return (known_argument_type->getTypeID() == kDate);
-    case kDatetime:
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          known_argument_type->getTypeID(),
-          kDatetime, kDatetimeInterval);
-    case kDatetimeInterval:
-      return (known_argument_type->getTypeID() == kDatetimeInterval);
-    case kYearMonthInterval:
-      return (known_argument_type->getTypeID() == kYearMonthInterval);
-    default:
-      return false;
-  }
-}
-
-std::pair<const Type*, const Type*> AddBinaryOperation::pushDownTypeHint(
-    const Type *result_type_hint) const {
-  if (result_type_hint == nullptr) {
-    return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-
-  switch (result_type_hint->getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble:
-    case kDatetimeInterval:
-    case kYearMonthInterval:
-      // Hint the same as the result type. Note that, for numeric types, one of
-      // the argument Types can be a less precise Type and still yield the
-      // specified result Type (e.g. DoubleType + IntType = DoubleType). We
-      // choose the highest-precision suitable Type (i.e. the same as the
-      // result type) in such cases.
-      return std::pair<const Type*, const Type*>(result_type_hint, result_type_hint);
-    case kDate:
-      // Hint is ambiguous: one argument should be a Date, other has to be
-      // kYearMonthInterval, but order is not important.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-    case kDatetime:
-      // Hint is ambiguous: one argument should be a Datetime, the other should
-      // be one of the interval types, but either order is acceptable.
-      // Fortunately, the 3 types in question have syntactically distinct
-      // representations in the SQL parser, so their literals don't need
-      // disambiguation anyway.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-    default:
-      // Inapplicable.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-}
-
-TypedValue AddBinaryOperation::applyToChecked(const TypedValue &left,
-                                              const Type &left_type,
-                                              const TypedValue &right,
-                                              const Type &right_type) const {
-  switch (left_type.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      switch (right_type.getTypeID()) {
-        case kInt:
-        case kLong:
-        case kFloat:
-        case kDouble:
-          return applyToCheckedNumericHelper<AddFunctor>(left, left_type,
-                                                         right, right_type);
-        default:
-          break;
-      }
-      break;
-    }
-    case kDate: {
-      if (right_type.getTypeID() == kYearMonthInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDate);
-        }
-
-        return TypedValue(left.getLiteral<DateLit>() + right.getLiteral<YearMonthIntervalLit>());
-      }
-      break;
-    }
-    case kDatetime: {
-      if (right_type.getTypeID() == kDatetimeInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetime);
-        }
-
-        return TypedValue(left.getLiteral<DatetimeLit>() + right.getLiteral<DatetimeIntervalLit>());
-      } else if (right_type.getTypeID() == kYearMonthInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetime);
-        }
-
-        return TypedValue(left.getLiteral<DatetimeLit>() + right.getLiteral<YearMonthIntervalLit>());
-      }
-      break;
-    }
-    case kDatetimeInterval: {
-      if (right_type.getTypeID() == kDatetime) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetime);
-        }
-
-        return TypedValue(left.getLiteral<DatetimeIntervalLit>() + right.getLiteral<DatetimeLit>());
-      } else if (right_type.getTypeID() == kDatetimeInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetimeInterval);
-        }
-
-        return TypedValue(left.getLiteral<DatetimeIntervalLit>() + right.getLiteral<DatetimeIntervalLit>());
-      }
-      break;
-    }
-    case kYearMonthInterval: {
-      if (right_type.getTypeID() == kDate) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetime);
-        }
-
-        return TypedValue(left.getLiteral<YearMonthIntervalLit>() + right.getLiteral<DateLit>());
-      } else if (right_type.getTypeID() == kDatetime) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kDatetime);
-        }
-
-        return TypedValue(left.getLiteral<YearMonthIntervalLit>() + right.getLiteral<DatetimeLit>());
-      } else if (right_type.getTypeID() == kYearMonthInterval) {
-        if (left.isNull() || right.isNull()) {
-          return TypedValue(kYearMonthInterval);
-        }
-
-        return TypedValue(left.getLiteral<YearMonthIntervalLit>() + right.getLiteral<YearMonthIntervalLit>());
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-             << left_type.getName() << " and " << right_type.getName();
-}
-
-UncheckedBinaryOperator* AddBinaryOperation::makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                                                 const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble: {
-      if (right.getSuperTypeID() == Type::kNumeric) {
-        return makeNumericBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator>(left, right);
-      }
-      break;
-    }
-    case kDate: {
-      if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<
-            AddArithmeticUncheckedBinaryOperator,
-            DateType,
-            DateLit,
-            YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kDatetime: {
-      if (right.getTypeID() == kDatetimeInterval) {
-        return makeDateBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator,
-                                                 DatetimeType,
-                                                 DatetimeLit, DatetimeIntervalLit>(left, right);
-      } else if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator,
-                                                 DatetimeType,
-                                                 DatetimeLit, YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kDatetimeInterval: {
-      if (right.getTypeID() == kDatetime) {
-        return makeDateBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator,
-                                                 DatetimeType,
-                                                 DatetimeIntervalLit, DatetimeLit>(left, right);
-      } else if (right.getTypeID() == kDatetimeInterval) {
-        return makeDateBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator,
-                                                 DatetimeIntervalType,
-                                                 DatetimeIntervalLit, DatetimeIntervalLit>(left, right);
-      }
-      break;
-    }
-    case kYearMonthInterval: {
-      if (right.getTypeID() == kDate) {
-        return makeDateBinaryOperatorOuterHelper<
-            AddArithmeticUncheckedBinaryOperator,
-            DateType,
-            YearMonthIntervalLit,
-            DateLit>(left, right);
-      } else if (right.getTypeID() == kDatetime) {
-        return makeDateBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator,
-                                                 DatetimeType,
-                                                 YearMonthIntervalLit, DatetimeLit>(left, right);
-      } else if (right.getTypeID() == kYearMonthInterval) {
-        return makeDateBinaryOperatorOuterHelper<AddArithmeticUncheckedBinaryOperator,
-                                                 YearMonthIntervalType,
-                                                 YearMonthIntervalLit, YearMonthIntervalLit>(left, right);
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/ArithmeticBinaryFunctorOverloads.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryFunctorOverloads.hpp b/types/operations/binary_operations/ArithmeticBinaryFunctorOverloads.hpp
new file mode 100644
index 0000000..4c6f76c
--- /dev/null
+++ b/types/operations/binary_operations/ArithmeticBinaryFunctorOverloads.hpp
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_FUNCTOR_OVERLOADS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_FUNCTOR_OVERLOADS_HPP_
+
+#include <cmath>
+#include <cstddef>
+#include <cstdint>
+#include <type_traits>
+#include <utility>
+
+#include "types/DateOperatorOverloads.hpp"
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+// We use these functors instead of the standard-library ones, because the
+// standard-library functors in <functional> have to be instantiated for the
+// most specific argument type, which would unnecessisarily introduce
+// multiple copies of distinct template instantiations of operators.
+template <typename LeftCppType, typename RightCppType, typename EnableT = void>
+struct AddFunctorOverloads {
+  inline auto operator() (const LeftCppType &left,
+                          const RightCppType &right) const -> decltype(left + right) {
+    return left + right;
+  }
+};
+
+// NOTE(zuyu): The C++ compiler in general converts all integers to floats
+//             when doing the following operations,
+//             but we could like to return double instead.
+template <>
+struct AddFunctorOverloads<std::int64_t, float> {
+  inline double operator() (const std::int64_t &left, const float &right) const {
+    return static_cast<double>(left) + static_cast<double>(right);
+  }
+};
+
+template <>
+struct AddFunctorOverloads<float, std::int64_t> {
+  inline double operator() (const float &left, const std::int64_t &right) const {
+    return static_cast<double>(left) + static_cast<double>(right);
+  }
+};
+
+template <typename LeftCppType, typename RightCppType, typename EnableT = void>
+struct SubtractFunctorOverloads {
+  inline auto operator() (const LeftCppType &left,
+                          const RightCppType &right) const -> decltype(left - right) {
+    return left - right;
+  }
+};
+
+// NOTE(zuyu): The C++ compiler in general converts all integers to floats
+//             when doing the following operations,
+//             but we could like to return double instead.
+template <>
+struct SubtractFunctorOverloads<std::int64_t, float> {
+  inline double operator() (const std::int64_t &left, const float &right) const {
+    return static_cast<double>(left) - static_cast<double>(right);
+  }
+};
+
+template <>
+struct SubtractFunctorOverloads<float, std::int64_t> {
+  inline double operator() (const float &left, const std::int64_t &right) const {
+    return static_cast<double>(left) - static_cast<double>(right);
+  }
+};
+
+template <typename LeftCppType, typename RightCppType, typename EnableT = void>
+struct MultiplyFunctorOverloads {
+  inline auto operator() (const LeftCppType &left,
+                          const RightCppType &right) const -> decltype(left * right) {
+    return left * right;
+  }
+};
+
+// NOTE(zuyu): The C++ compiler in general converts all integers to floats
+//             when doing the following operations,
+//             but we could like to return double instead.
+template <>
+struct MultiplyFunctorOverloads<std::int64_t, float> {
+  inline double operator() (const std::int64_t &left, const float &right) const {
+    return static_cast<double>(left) * static_cast<double>(right);
+  }
+};
+
+template <>
+struct MultiplyFunctorOverloads<float, std::int64_t> {
+  inline double operator() (const float &left, const std::int64_t &right) const {
+    return static_cast<double>(left) * static_cast<double>(right);
+  }
+};
+
+template <typename LeftCppType, typename RightCppType, typename EnableT = void>
+struct DivideFunctorOverloads {
+  inline auto operator() (const LeftCppType &left,
+                          const RightCppType &right) const -> decltype(left / right) {
+    return left / right;
+  }
+};
+
+// NOTE(zuyu): The C++ compiler in general converts all integers to floats
+//             when doing the following operations,
+//             but we could like to return double instead.
+template <>
+struct DivideFunctorOverloads<std::int64_t, float> {
+  inline double operator() (const std::int64_t &left, const float &right) const {
+    return static_cast<double>(left) / static_cast<double>(right);
+  }
+};
+
+template <>
+struct DivideFunctorOverloads<float, std::int64_t> {
+  inline double operator() (const float &left, const std::int64_t &right) const {
+    return static_cast<double>(left) / static_cast<double>(right);
+  }
+};
+
+template <typename LeftCppType, typename RightCppType, typename EnableT = void>
+struct ModuloFunctorOverloads;
+
+template <typename LeftCppType, typename RightCppType>
+struct ModuloFunctorOverloads<
+    LeftCppType, RightCppType,
+    std::enable_if_t<meta::EqualsAny<LeftCppType, int, std::int64_t>::value &&
+                     meta::EqualsAny<RightCppType, int, std::int64_t>::value>> {
+  inline auto operator() (const LeftCppType &left,
+                          const RightCppType &right) const -> decltype(left % right) {
+    return left % right;
+  }
+};
+
+// NOTE(jianqiao): The C++11 standard specifies the following type signatures for fmod:
+// (1) (double, double) -> double
+// (2) (float, float) -> float
+// (3) (long double, long double) -> long double
+// (3) (Arithmetic, Arithmetic) -> double
+template <typename LeftCppType, typename RightCppType>
+struct ModuloFunctorOverloads<
+    LeftCppType, RightCppType,
+    std::enable_if_t<meta::EqualsAny<LeftCppType, float, double>::value ||
+                     meta::EqualsAny<RightCppType, float, double>::value>> {
+  inline auto operator() (const LeftCppType &left,
+                          const RightCppType &right) const -> decltype(std::fmod(left, right)) {
+    return std::fmod(left, right);
+  }
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_FUNCTOR_OVERLOADS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/ArithmeticBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryOperation.hpp b/types/operations/binary_operations/ArithmeticBinaryOperation.hpp
deleted file mode 100644
index f9a27a8..0000000
--- a/types/operations/binary_operations/ArithmeticBinaryOperation.hpp
+++ /dev/null
@@ -1,404 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- **/
-
-#ifndef QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATION_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATION_HPP_
-
-#include <string>
-
-#include "types/DoubleType.hpp"
-#include "types/FloatType.hpp"
-#include "types/IntType.hpp"
-#include "types/LongType.hpp"
-#include "types/NumericTypeUnifier.hpp"
-#include "types/Type.hpp"
-#include "types/TypeErrors.hpp"
-#include "types/TypeFactory.hpp"
-#include "types/TypeID.hpp"
-#include "types/TypedValue.hpp"
-#include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief A BinaryOperation which applies to and yields values
- *        including numeric, datetime, and intervals.
- **/
-class ArithmeticBinaryOperation : public BinaryOperation {
- protected:
-  explicit ArithmeticBinaryOperation(const BinaryOperationID operation_id)
-      : BinaryOperation(operation_id) {
-  }
-
-  template <template <typename LeftCppType, typename RightCppType> class OperationFunctor>
-  TypedValue applyToCheckedIntegerHelper(const TypedValue &left,
-                                         const Type &left_type,
-                                         const TypedValue &right,
-                                         const Type &right_type) const;
-
-  template <template <typename LeftCppType, typename RightCppType> class OperationFunctor>
-  TypedValue applyToCheckedNumericHelper(const TypedValue &left,
-                                         const Type &left_type,
-                                         const TypedValue &right,
-                                         const Type &right_type) const;
-
-  template <template <class ResultType,
-                      typename LeftCppType, bool left_nullable,
-                      typename RightCppType, bool right_nullable> class OperatorType>
-  UncheckedBinaryOperator* makeIntegerBinaryOperatorOuterHelper(const Type &left,
-                                                                const Type &right) const;
-
-  template <template <class ResultType,
-                      typename LeftCppType, bool left_nullable,
-                      typename RightCppType, bool right_nullable> class OperatorType,
-            typename LeftType, bool left_nullable>
-  UncheckedBinaryOperator* makeIntegerBinaryOperatorInnerHelper(const Type &left,
-                                                                const Type &right) const;
-
-  template <template <class ResultType,
-                      typename LeftCppType, bool left_nullable,
-                      typename RightCppType, bool right_nullable> class OperatorType>
-  UncheckedBinaryOperator* makeNumericBinaryOperatorOuterHelper(const Type &left,
-                                                                const Type &right) const;
-
-  template <template <class ResultType,
-                      typename LeftCppType, bool left_nullable,
-                      typename RightCppType, bool right_nullable> class OperatorType,
-            typename LeftType, bool left_nullable>
-  UncheckedBinaryOperator* makeNumericBinaryOperatorInnerHelper(const Type &left,
-                                                                const Type &right) const;
-
-  template <template <typename ResultType,
-                      typename LeftCppType, bool left_nullable,
-                      typename RightCppType, bool right_nullable> class OperatorType,
-            typename ResultType,
-            typename LeftCppType,
-            typename RightCppType>
-  UncheckedBinaryOperator* makeDateBinaryOperatorOuterHelper(const Type &left,
-                                                             const Type &right) const;
-
-  template <template <typename ResultType,
-                      typename LeftCppType, bool left_nullable,
-                      typename RightCppType, bool right_nullable> class OperatorType,
-            typename ResultType,
-            typename LeftCppType,
-            bool left_nullable,
-            typename RightCppType>
-  UncheckedBinaryOperator* makeDateBinaryOperatorInnerHelper(const Type &right) const;
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(ArithmeticBinaryOperation);
-};
-
-/** @} */
-
-// ----------------------------------------------------------------------------
-// Templated method implementations follow:
-
-template <template <typename LeftCppType, typename RightCppType> class OperationFunctor>
-TypedValue ArithmeticBinaryOperation::applyToCheckedIntegerHelper(
-    const TypedValue &left,
-    const Type &left_type,
-    const TypedValue &right,
-    const Type &right_type) const {
-  DCHECK(left_type.getTypeID() == TypeID::kInt
-         || left_type.getTypeID() == TypeID::kLong);
-  DCHECK(right_type.getTypeID() == TypeID::kInt
-         || right_type.getTypeID() == TypeID::kLong);
-
-  const Type *unifier = TypeFactory::GetUnifyingType(left_type, right_type);
-  DCHECK(unifier != nullptr);
-
-  if (left.isNull() || right.isNull()) {
-    return unifier->makeNullValue();
-  }
-
-  const TypedValue left_coerced = unifier->coerceValue(left, left_type);
-  const TypedValue right_coerced = unifier->coerceValue(right, right_type);
-
-  switch (unifier->getTypeID()) {
-    case kInt: {
-      OperationFunctor<IntType::cpptype, IntType::cpptype> operation_functor;
-      return TypedValue(operation_functor(left_coerced.getLiteral<IntType::cpptype>(),
-                                          right_coerced.getLiteral<IntType::cpptype>()));
-    }
-    case kLong: {
-      OperationFunctor<LongType::cpptype, LongType::cpptype> operation_functor;
-      return TypedValue(operation_functor(left_coerced.getLiteral<LongType::cpptype>(),
-                                          right_coerced.getLiteral<LongType::cpptype>()));
-    }
-    default: {
-      LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-                 << left_type.getName() << " and " << right_type.getName();
-    }
-  }
-
-  QUICKSTEP_UNREACHABLE();
-}
-
-template <template <typename LeftCppType, typename RightCppType> class OperationFunctor>
-TypedValue ArithmeticBinaryOperation::applyToCheckedNumericHelper(
-    const TypedValue &left,
-    const Type &left_type,
-    const TypedValue &right,
-    const Type &right_type) const {
-  DCHECK_EQ(Type::kNumeric, left_type.getSuperTypeID());
-  DCHECK_EQ(Type::kNumeric, right_type.getSuperTypeID());
-
-  const Type *unifier = TypeFactory::GetUnifyingType(left_type, right_type);
-  DCHECK(unifier != nullptr);
-
-  if (left.isNull() || right.isNull()) {
-    return unifier->makeNullValue();
-  }
-
-  const TypedValue left_coerced = unifier->coerceValue(left, left_type);
-  const TypedValue right_coerced = unifier->coerceValue(right, right_type);
-
-  switch (unifier->getTypeID()) {
-    case kInt: {
-      OperationFunctor<IntType::cpptype, IntType::cpptype> operation_functor;
-      return TypedValue(operation_functor(left_coerced.getLiteral<IntType::cpptype>(),
-                                          right_coerced.getLiteral<IntType::cpptype>()));
-    }
-    case kLong: {
-      OperationFunctor<LongType::cpptype, LongType::cpptype> operation_functor;
-      return TypedValue(operation_functor(left_coerced.getLiteral<LongType::cpptype>(),
-                                          right_coerced.getLiteral<LongType::cpptype>()));
-    }
-    case kFloat: {
-      OperationFunctor<FloatType::cpptype, FloatType::cpptype> operation_functor;
-      return TypedValue(operation_functor(left_coerced.getLiteral<FloatType::cpptype>(),
-                                          right_coerced.getLiteral<FloatType::cpptype>()));
-    }
-    case kDouble: {
-      OperationFunctor<DoubleType::cpptype, DoubleType::cpptype> operation_functor;
-      return TypedValue(operation_functor(left_coerced.getLiteral<DoubleType::cpptype>(),
-                                          right_coerced.getLiteral<DoubleType::cpptype>()));
-    }
-    default: {
-      LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-                 << left_type.getName() << " and " << right_type.getName();
-    }
-  }
-
-  QUICKSTEP_UNREACHABLE();
-}
-
-template <template <class ResultType,
-                    typename LeftCppType, bool left_nullable,
-                    typename RightCppType, bool right_nullable> class OperatorType>
-UncheckedBinaryOperator* ArithmeticBinaryOperation::makeIntegerBinaryOperatorOuterHelper(
-    const Type &left,
-    const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-      if (left.isNullable()) {
-        return makeIntegerBinaryOperatorInnerHelper<OperatorType, IntType, true>(
-            left, right);
-      } else {
-        return makeIntegerBinaryOperatorInnerHelper<OperatorType, IntType, false>(
-            left, right);
-      }
-    case kLong:
-      if (left.isNullable()) {
-        return makeIntegerBinaryOperatorInnerHelper<OperatorType, LongType, true>(
-            left, right);
-      } else {
-        return makeIntegerBinaryOperatorInnerHelper<OperatorType, LongType, false>(
-            left, right);
-      }
-    default:
-      throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-  }
-}
-
-template <template <class ResultType,
-                    typename LeftCppType, bool left_nullable,
-                    typename RightCppType, bool right_nullable> class OperatorType,
-          typename LeftType, bool left_nullable>
-UncheckedBinaryOperator* ArithmeticBinaryOperation::makeIntegerBinaryOperatorInnerHelper(
-    const Type &left,
-    const Type &right) const {
-  switch (right.getTypeID()) {
-    case kInt:
-      if (right.isNullable()) {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, IntType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename IntType::cpptype, true>();
-      } else {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, IntType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename IntType::cpptype, false>();
-      }
-    case kLong:
-      if (right.isNullable()) {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, LongType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename LongType::cpptype, true>();
-      } else {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, LongType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename LongType::cpptype, false>();
-      }
-    default:
-      throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-  }
-}
-
-template <template <class ResultType,
-                    typename LeftCppType, bool left_nullable,
-                    typename RightCppType, bool right_nullable> class OperatorType>
-UncheckedBinaryOperator* ArithmeticBinaryOperation::makeNumericBinaryOperatorOuterHelper(
-    const Type &left,
-    const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-      if (left.isNullable()) {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, IntType, true>(
-            left, right);
-      } else {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, IntType, false>(
-            left, right);
-      }
-    case kLong:
-      if (left.isNullable()) {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, LongType, true>(
-            left, right);
-      } else {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, LongType, false>(
-            left, right);
-      }
-    case kFloat:
-      if (left.isNullable()) {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, FloatType, true>(
-            left, right);
-      } else {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, FloatType, false>(
-            left, right);
-      }
-    case kDouble:
-      if (left.isNullable()) {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, DoubleType, true>(
-            left, right);
-      } else {
-        return makeNumericBinaryOperatorInnerHelper<OperatorType, DoubleType, false>(
-            left, right);
-      }
-    default:
-      throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-  }
-}
-
-template <template <class ResultType,
-                    typename LeftCppType, bool left_nullable,
-                    typename RightCppType, bool right_nullable> class OperatorType,
-          typename LeftType, bool left_nullable>
-UncheckedBinaryOperator* ArithmeticBinaryOperation::makeNumericBinaryOperatorInnerHelper(
-    const Type &left,
-    const Type &right) const {
-  switch (right.getTypeID()) {
-    case kInt:
-      if (right.isNullable()) {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, IntType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename IntType::cpptype, true>();
-      } else {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, IntType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename IntType::cpptype, false>();
-      }
-    case kLong:
-      if (right.isNullable()) {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, LongType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename LongType::cpptype, true>();
-      } else {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, LongType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename LongType::cpptype, false>();
-      }
-    case kFloat:
-      if (right.isNullable()) {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, FloatType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename FloatType::cpptype, true>();
-      } else {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, FloatType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename FloatType::cpptype, false>();
-      }
-    case kDouble:
-      if (right.isNullable()) {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, DoubleType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename DoubleType::cpptype, true>();
-      } else {
-        return new OperatorType<typename NumericTypeUnifier<LeftType, DoubleType>::type,
-                                typename LeftType::cpptype, left_nullable,
-                                typename DoubleType::cpptype, false>();
-      }
-    default:
-      throw OperationInapplicableToType(getName(), 2, left.getName().c_str(), right.getName().c_str());
-  }
-}
-
-template <template <typename ResultType,
-                    typename LeftCppType, bool left_nullable,
-                    typename RightCppType, bool right_nullable> class OperatorType,
-          typename ResultType,
-          typename LeftCppType,
-          typename RightCppType>
-UncheckedBinaryOperator* ArithmeticBinaryOperation::makeDateBinaryOperatorOuterHelper(
-    const Type &left,
-    const Type &right) const {
-  if (left.isNullable()) {
-    return makeDateBinaryOperatorInnerHelper<OperatorType, ResultType, LeftCppType, true, RightCppType>(right);
-  } else {
-    return makeDateBinaryOperatorInnerHelper<OperatorType, ResultType, LeftCppType, false, RightCppType>(right);
-  }
-}
-
-template <template <typename ResultType,
-                    typename LeftCppType, bool left_nullable,
-                    typename RightCppType, bool right_nullable> class OperatorType,
-          typename ResultType,
-          typename LeftCppType,
-          bool left_nullable,
-          typename RightCppType>
-UncheckedBinaryOperator* ArithmeticBinaryOperation::makeDateBinaryOperatorInnerHelper(
-    const Type &right) const {
-  if (right.isNullable()) {
-    return new OperatorType<ResultType, LeftCppType, left_nullable, RightCppType, true>();
-  } else {
-    return new OperatorType<ResultType, LeftCppType, left_nullable, RightCppType, false>();
-  }
-}
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/583724ea/types/operations/binary_operations/ArithmeticBinaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryOperations.hpp b/types/operations/binary_operations/ArithmeticBinaryOperations.hpp
new file mode 100644
index 0000000..fa4d926
--- /dev/null
+++ b/types/operations/binary_operations/ArithmeticBinaryOperations.hpp
@@ -0,0 +1,182 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATIONS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATIONS_HPP_
+
+#include <string>
+#include <tuple>
+
+#include "types/DateType.hpp"
+#include "types/DatetimeIntervalType.hpp"
+#include "types/DatetimeLit.hpp"
+#include "types/DatetimeType.hpp"
+#include "types/IntervalLit.hpp"
+#include "types/NumericTypeUnifier.hpp"
+#include "types/Type.hpp"
+#include "types/TypeErrors.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/YearMonthIntervalType.hpp"
+#include "types/operations/binary_operations/ArithmeticBinaryFunctorOverloads.hpp"
+#include "types/operations/binary_operations/BinaryOperationWrapper.hpp"
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename LeftT, typename RightT, typename ResultT,
+          template <typename LeftCppType,
+                    typename RightCppType,
+                    typename EnableT = void> class FunctorOverloadsT,
+          typename FunctorNameT>
+struct ArithmeticBinaryFunctor : public BinaryFunctor<LeftT, RightT, ResultT> {
+  ArithmeticBinaryFunctor() : spec() {}
+  inline typename ResultT::cpptype apply(const typename LeftT::cpptype &left,
+                                         const typename RightT::cpptype &right) const {
+    return spec(left, right);
+  }
+  inline static std::string GetName() {
+    return FunctorNameT::ToString();
+  }
+  const FunctorOverloadsT<typename LeftT::cpptype,
+                          typename RightT::cpptype> spec;
+};
+
+template <typename LeftT, typename RightT, typename ResultT>
+using AddFunctor = ArithmeticBinaryFunctor<LeftT, RightT, ResultT,
+                                           AddFunctorOverloads,
+                                           meta::StringLiteral<'+'>>;
+
+template <typename LeftT, typename RightT, typename ResultT>
+using SubtractFunctor = ArithmeticBinaryFunctor<LeftT, RightT, ResultT,
+                                                SubtractFunctorOverloads,
+                                                meta::StringLiteral<'-'>>;
+
+template <typename LeftT, typename RightT, typename ResultT>
+using MultiplyFunctor = ArithmeticBinaryFunctor<LeftT, RightT, ResultT,
+                                                MultiplyFunctorOverloads,
+                                                meta::StringLiteral<'*'>>;
+
+template <typename LeftT, typename RightT, typename ResultT>
+using DivideFunctor = ArithmeticBinaryFunctor<LeftT, RightT, ResultT,
+                                              DivideFunctorOverloads,
+                                              meta::StringLiteral<'/'>>;
+
+template <typename LeftT, typename RightT, typename ResultT>
+using ModuloFunctor = ArithmeticBinaryFunctor<LeftT, RightT, ResultT,
+                                              ModuloFunctorOverloads,
+                                              meta::StringLiteral<'%'>>;
+
+// ----------------------------------------------------------------------------
+// Packs of functors:
+
+using AddBinaryFunctorPack = FunctorPack<
+// Numeric
+    BinaryFunctorCrossProductPack<
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        AddFunctor, NumericTypeUnifier>,
+// Date
+    AddFunctor<DateType, YearMonthIntervalType, DateType>,
+    AddFunctor<YearMonthIntervalType, DateType, DateType>,
+// Datetime
+    AddFunctor<DatetimeType, DatetimeIntervalType, DatetimeType>,
+    AddFunctor<DatetimeType, YearMonthIntervalType, DatetimeType>,
+    AddFunctor<DatetimeIntervalType, DatetimeType, DatetimeType>,
+    AddFunctor<YearMonthIntervalType, DatetimeType, DatetimeType>,
+// DatetimeInterval
+    AddFunctor<DatetimeIntervalType, DatetimeIntervalType, DatetimeIntervalType>,
+// YearMonthInterval
+    AddFunctor<YearMonthIntervalType, YearMonthIntervalType, YearMonthIntervalType>
+>;
+
+using SubtractBinaryFunctorPack = FunctorPack<
+// Numeric
+    BinaryFunctorCrossProductPack<
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        SubtractFunctor, NumericTypeUnifier>,
+// Date
+    SubtractFunctor<DateType, YearMonthIntervalType, DateType>,
+    // TODO(quickstep-team):
+    // Implement SubtractFunctor<DateType, DateType, YearMonthIntervalType>,
+// Datetime
+    SubtractFunctor<DatetimeType, DatetimeIntervalType, DatetimeType>,
+    SubtractFunctor<DatetimeType, YearMonthIntervalType, DatetimeType>,
+    SubtractFunctor<DatetimeType, DatetimeType, DatetimeIntervalType>,
+// DatetimeInterval
+    SubtractFunctor<DatetimeIntervalType, DatetimeIntervalType, DatetimeIntervalType>,
+// YearMonthInterval
+    SubtractFunctor<YearMonthIntervalType, YearMonthIntervalType, YearMonthIntervalType>
+>;
+
+using MultiplyBinaryFunctorPack = FunctorPack<
+// Numeric
+    BinaryFunctorCrossProductPack<
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        MultiplyFunctor, NumericTypeUnifier>,
+// DatetimeInterval and YearMonthInterval
+    BinaryFunctorCrossProductPack<
+        std::tuple<DatetimeIntervalType, YearMonthIntervalType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        MultiplyFunctor, meta::PairSelectorLeft>,
+    BinaryFunctorCrossProductPack<
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        std::tuple<DatetimeIntervalType, YearMonthIntervalType>,
+        MultiplyFunctor, meta::PairSelectorRight>
+>;
+
+using DivideBinaryFunctorPack = FunctorPack<
+// Numeric
+    BinaryFunctorCrossProductPack<
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        DivideFunctor, NumericTypeUnifier>,
+// DatetimeInterval and YearMonthInterval
+    BinaryFunctorCrossProductPack<
+        std::tuple<DatetimeIntervalType, YearMonthIntervalType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        DivideFunctor, meta::PairSelectorLeft>
+>;
+
+using ModuloBinaryFunctorPack = FunctorPack<
+// Numeric
+    BinaryFunctorCrossProductPack<
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        std::tuple<IntType, LongType, FloatType, DoubleType>,
+        ModuloFunctor, NumericTypeUnifier>
+>;
+
+using ArithmeticBinaryFunctorPack = FunctorPack<
+    AddBinaryFunctorPack,
+    SubtractBinaryFunctorPack,
+    MultiplyBinaryFunctorPack,
+    DivideBinaryFunctorPack,
+    ModuloBinaryFunctorPack
+>;
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATIONS_HPP_