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

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/binary_operations/BinaryOperationWrapper.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationWrapper.hpp b/types/operations/binary_operations/BinaryOperationWrapper.hpp
new file mode 100644
index 0000000..98c2e8d
--- /dev/null
+++ b/types/operations/binary_operations/BinaryOperationWrapper.hpp
@@ -0,0 +1,629 @@
+/**
+ * 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_BINARY_OPERATION_WRAPPER_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_WRAPPER_HPP_
+
+#include <cstddef>
+#include <list>
+#include <string>
+#include <tuple>
+#include <type_traits>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/operations/OperationSignature.hpp"
+#include "types/operations/OperationUtil.hpp"
+#include "types/operations/binary_operations/BinaryOperation.hpp"
+#include "utility/Macros.hpp"
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename LeftArgumentT, typename RightArgumentT, typename ResultT>
+struct BinaryFunctor {
+  typedef LeftArgumentT LeftArgumentType;
+  typedef RightArgumentT RightArgumentType;
+  typedef ResultT ResultType;
+
+  static constexpr Operation
+      ::OperationSuperTypeID kOperationSuperTypeID = Operation::kBinaryOperation;
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+class UncheckedBinaryOperatorWrapperCodegen : public UncheckedBinaryOperator {
+ public:
+  template <typename ...ConstructorArgs>
+  UncheckedBinaryOperatorWrapperCodegen(const Type &left_type,
+                                        const Type &right_type,
+                                        const Type &result_type,
+                                        ConstructorArgs &&...args)
+      : functor_(std::forward<ConstructorArgs>(args)...),
+        impl_(functor_, left_type, right_type, result_type) {}
+
+  TypedValue applyToTypedValues(const TypedValue &left,
+                                const TypedValue &right) const override {
+    return impl_.applyToTypedValues(left, right);
+  }
+
+  ColumnVector* applyToColumnVectors(const ColumnVector &left,
+                                     const ColumnVector &right) const override {
+    using LeftCVT = typename LeftGen::ColumnVectorType;
+    DCHECK_EQ(left.isNative(), LeftCVT::kNative);
+    using LeftAccessorT = ColumnVectorValueAccessor<LeftCVT>;
+    LeftAccessorT left_accessor(static_cast<const LeftCVT&>(left));
+
+    using RightCVT = typename RightGen::ColumnVectorType;
+    DCHECK_EQ(right.isNative(), RightCVT::kNative);
+    using RightAccessorT = ColumnVectorValueAccessor<RightCVT>;
+    RightAccessorT right_accessor(static_cast<const RightCVT&>(right));
+
+    const std::size_t num_tuples = left_accessor.getNumTuples();
+    DCHECK_EQ(num_tuples, right_accessor.getNumTuples());
+
+    return impl_.applyToValueAccessors(num_tuples,
+                                       &left_accessor, kInvalidAttributeID,
+                                       &right_accessor, kInvalidAttributeID);
+  }
+
+  ColumnVector* applyToColumnVectorAndStaticValue(
+      const ColumnVector &left,
+      const TypedValue &right) const override {
+    using LeftCVT = typename LeftGen::ColumnVectorType;
+    DCHECK_EQ(left.isNative(), LeftCVT::kNative);
+
+    using LeftAccessorT = ColumnVectorValueAccessor<LeftCVT>;
+    LeftAccessorT accessor(static_cast<const LeftCVT&>(left));
+    return impl_.applyToValueAccessorAndStaticValue(&accessor, 0, right);
+  }
+
+  ColumnVector* applyToStaticValueAndColumnVector(
+      const TypedValue &left,
+      const ColumnVector &right) const override {
+    using RightCVT = typename RightGen::ColumnVectorType;
+    DCHECK_EQ(right.isNative(), RightCVT::kNative);
+
+    using RightAccessorT = ColumnVectorValueAccessor<RightCVT>;
+    RightAccessorT accessor(static_cast<const RightCVT&>(right));
+    return impl_.applyToStaticValueAndValueAccessor(left, &accessor, 0);
+  }
+
+  ColumnVector* applyToSingleValueAccessor(
+      ValueAccessor *accessor,
+      const attribute_id left_id,
+      const attribute_id right_id) const override {
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        accessor,
+        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      return impl_.applyToSingleValueAccessor(accessor, left_id, right_id);
+    });
+  }
+
+  ColumnVector* applyToValueAccessorAndStaticValue(
+      ValueAccessor *left_accessor,
+      const attribute_id left_id,
+      const TypedValue &right) const override {
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        left_accessor,
+        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      return impl_.applyToValueAccessorAndStaticValue(accessor, left_id, right);
+    });
+  }
+
+  ColumnVector* applyToStaticValueAndValueAccessor(
+      const TypedValue &left,
+      ValueAccessor *right_accessor,
+      const attribute_id right_id) const override {
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        right_accessor,
+        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      return impl_.applyToStaticValueAndValueAccessor(left, accessor, right_id);
+    });
+  }
+
+  ColumnVector* applyToColumnVectorAndValueAccessor(
+      const ColumnVector &left,
+      ValueAccessor *right_accessor,
+      const attribute_id right_id) const override {
+    using LeftCVT = typename LeftGen::ColumnVectorType;
+    DCHECK_EQ(left.isNative(), LeftCVT::kNative);
+    using LeftAccessorT = ColumnVectorValueAccessor<LeftCVT>;
+    LeftAccessorT left_accessor(static_cast<const LeftCVT&>(left));
+
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        right_accessor,
+        [&](auto *right_accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+    const std::size_t num_tuples = left_accessor.getNumTuples();
+    DCHECK_EQ(num_tuples, right_accessor->getNumTuples());
+
+    return impl_.applyToValueAccessors(num_tuples,
+                                       &left_accessor, kInvalidAttributeID,
+                                       right_accessor, right_id);
+    });
+  }
+
+  ColumnVector* applyToValueAccessorAndColumnVector(
+      ValueAccessor *left_accessor,
+      const attribute_id left_id,
+      const ColumnVector &right) const override {
+    using RightCVT = typename RightGen::ColumnVectorType;
+    DCHECK_EQ(right.isNative(), RightCVT::kNative);
+    using RightAccessorT = ColumnVectorValueAccessor<RightCVT>;
+    RightAccessorT right_accessor(static_cast<const RightCVT&>(right));
+
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        left_accessor,
+        [&](auto *left_accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      const std::size_t num_tuples = right_accessor.getNumTuples();
+      DCHECK_EQ(num_tuples, left_accessor->getNumTuples());
+
+      return impl_.applyToValueAccessors(num_tuples,
+                                         left_accessor, left_id,
+                                         &right_accessor, kInvalidAttributeID);
+    });
+  }
+
+  TypedValue accumulateColumnVector(
+      const TypedValue &current,
+      const ColumnVector &column_vector,
+      std::size_t *num_tuples_applied) const override {
+    constexpr bool is_supported =
+        LeftType::kStaticTypeID == ResultType::kStaticTypeID &&
+        (LeftType::kLayout == kNativeEmbedded || LeftType::kLayout == kNativeInline) &&
+        std::is_copy_assignable<typename LeftType::cpptype>::value;
+
+    using RightCVT = typename RightGen::ColumnVectorType;
+    DCHECK_EQ(column_vector.isNative(), RightCVT::kNative);
+    using RightAccessorT = ColumnVectorValueAccessor<RightCVT>;
+    RightAccessorT accessor(static_cast<const RightCVT&>(column_vector));
+
+    return impl_.template accumulateValueAccessor<is_supported>(
+        current,
+        &accessor,
+        kInvalidAttributeID,
+        num_tuples_applied);
+  }
+
+  TypedValue accumulateValueAccessor(
+      const TypedValue &current,
+      ValueAccessor *accessor,
+      const attribute_id value_accessor_id,
+      std::size_t *num_tuples_applied) const override {
+    constexpr bool is_supported =
+        LeftType::kStaticTypeID == ResultType::kStaticTypeID &&
+        (LeftType::kLayout == kNativeEmbedded || LeftType::kLayout == kNativeInline) &&
+        std::is_copy_assignable<typename LeftType::cpptype>::value;
+
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        accessor,
+        [&](auto *accessor) -> TypedValue {  // NOLINT(build/c++11)
+      return impl_.template accumulateValueAccessor<is_supported>(
+          current,
+          accessor,
+          value_accessor_id,
+          num_tuples_applied);
+    });
+  }
+
+ private:
+  using LeftType = typename FunctorT::LeftArgumentType;
+  using RightType = typename FunctorT::RightArgumentType;
+  using ResultType = typename FunctorT::ResultType;
+
+  using FuncSpec = typename FunctorSpecializer<FunctorT, SpecArgs...>::type;
+  using LeftGen = Codegen<FuncSpec, LeftType>;
+  using RightGen = Codegen<FuncSpec, RightType>;
+  using ResultGen = Codegen<FuncSpec, ResultType>;
+
+  template <bool left_nullable, bool right_nullable>
+  struct Implementation;
+
+  const FunctorT functor_;
+  const Implementation<false, false> impl_;
+
+  DISALLOW_COPY_AND_ASSIGN(UncheckedBinaryOperatorWrapperCodegen);
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <bool left_nullable, bool right_nullable>
+struct UncheckedBinaryOperatorWrapperCodegen<FunctorT, SpecArgs...>
+    ::Implementation {
+  Implementation(const FunctorT &functor_in,
+                 const Type &left_type_in,
+                 const Type &right_type_in,
+                 const Type &result_type_in)
+      : functor(functor_in),
+        left_type(left_type_in),
+        right_type(right_type_in),
+        result_type(result_type_in) {}
+
+  inline TypedValue applyToTypedValues(const TypedValue &left,
+                                       const TypedValue &right) const {
+    if ((left_nullable && left.isNull()) || (right_nullable && right.isNull())) {
+      return TypedValue(ResultType::kStaticTypeID);
+    }
+
+    return ResultGen::template ApplyBinaryTypedValue<LeftGen, RightGen>(
+        LeftGen::ToNativeValueConst(left),
+        RightGen::ToNativeValueConst(right),
+        result_type,
+        functor);
+  }
+
+  template <typename AccessorT>
+  inline ColumnVector* applyToValueAccessorAndStaticValue(
+      AccessorT *accessor,
+      const attribute_id attr_id,
+      const TypedValue &static_value) const {
+    using AccessorGen = LeftGen;
+    using StaticValueGen = RightGen;
+
+    constexpr bool accessor_nullable = left_nullable;
+    constexpr bool static_value_nullable = right_nullable;
+
+    using ResultCVT = typename ResultGen::ColumnVectorType;
+    ResultCVT *result_cv = new ResultCVT(result_type, accessor->getNumTuples());
+
+    if (static_value_nullable && static_value.isNull()) {
+      result_cv->fillWithNulls();
+      return result_cv;
+    }
+
+    typename StaticValueGen::NativeTypeConst literal =
+        StaticValueGen::ToNativeValueConst(static_value);
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename AccessorGen::NativeTypeConstPtr arg_value =
+          AccessorGen::template GetValuePtr<
+              accessor_nullable, AccessorT>(accessor, attr_id);
+      if (accessor_nullable && AccessorGen::IsNull(arg_value)) {
+        result_cv->appendNullValue();
+      } else {
+        ResultGen::template ApplyBinaryColumnVector<LeftGen, RightGen>(
+            AccessorGen::Dereference(arg_value), literal, functor, result_cv);
+      }
+    }
+    return result_cv;
+  }
+
+  template <typename AccessorT>
+  inline ColumnVector* applyToStaticValueAndValueAccessor(
+      const TypedValue &static_value,
+      AccessorT *accessor,
+      const attribute_id attr_id) const {
+    using AccessorGen = RightGen;
+    using StaticValueGen = LeftGen;
+
+    constexpr bool accessor_nullable = right_nullable;
+    constexpr bool static_value_nullable = left_nullable;
+
+    using ResultCVT = typename ResultGen::ColumnVectorType;
+    ResultCVT *result_cv = new ResultCVT(result_type, accessor->getNumTuples());
+
+    if (static_value_nullable && static_value.isNull()) {
+      result_cv->fillWithNulls();
+      return result_cv;
+    }
+
+    typename StaticValueGen::NativeTypeConst literal =
+        StaticValueGen::ToNativeValueConst(static_value);
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename AccessorGen::NativeTypeConstPtr arg_value =
+          AccessorGen::template GetValuePtr<
+              accessor_nullable, AccessorT>(accessor, attr_id);
+      if (accessor_nullable && AccessorGen::IsNull(arg_value)) {
+        result_cv->appendNullValue();
+      } else {
+        ResultGen::template ApplyBinaryColumnVector<LeftGen, RightGen>(
+            literal, AccessorGen::Dereference(arg_value), functor, result_cv);
+      }
+    }
+    return result_cv;
+  }
+
+  template <typename AccessorT>
+  inline ColumnVector* applyToSingleValueAccessor(
+      AccessorT *accessor,
+      const attribute_id left_id,
+      const attribute_id right_id) const {
+    using ResultCVT = typename ResultGen::ColumnVectorType;
+    ResultCVT *result_cv = new ResultCVT(result_type, accessor->getNumTuples());
+
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename LeftGen::NativeTypeConstPtr left_value =
+          LeftGen::template GetValuePtr<
+              left_nullable, AccessorT>(accessor, left_id);
+      if (left_nullable && LeftGen::IsNull(left_value)) {
+        result_cv->appendNullValue();
+        continue;
+      }
+      typename RightGen::NativeTypeConstPtr right_value =
+          RightGen::template GetValuePtr<
+              right_nullable, AccessorT>(accessor, right_id);
+      if (right_nullable && RightGen::IsNull(right_value)) {
+        result_cv->appendNullValue();
+        continue;
+      }
+      ResultGen::template ApplyBinaryColumnVector<LeftGen, RightGen>(
+          LeftGen::Dereference(left_value),
+          RightGen::Dereference(right_value),
+          functor,
+          result_cv);
+    }
+    return result_cv;
+  }
+
+  template <typename LeftAccessorT, typename RightAccessorT>
+  inline ColumnVector* applyToValueAccessors(const std::size_t num_tuples,
+                                             LeftAccessorT *left_accessor,
+                                             const attribute_id left_id,
+                                             RightAccessorT *right_accessor,
+                                             const attribute_id right_id) const {
+    DCHECK_EQ(num_tuples, left_accessor->getNumTuples());
+    DCHECK_EQ(num_tuples, right_accessor->getNumTuples());
+
+    using ResultCVT = typename ResultGen::ColumnVectorType;
+    ResultCVT *result_cv = new ResultCVT(result_type, num_tuples);
+
+    left_accessor->beginIteration();
+    right_accessor->beginIteration();
+    while (left_accessor->next()) {
+      right_accessor->next();
+      typename LeftGen::NativeTypeConstPtr left_value =
+          LeftGen::template GetValuePtr<
+              left_nullable, LeftAccessorT>(left_accessor, left_id);
+      if (left_nullable && LeftGen::IsNull(left_value)) {
+        result_cv->appendNullValue();
+        continue;
+      }
+      typename RightGen::NativeTypeConstPtr right_value =
+          RightGen::template GetValuePtr<
+              right_nullable, RightAccessorT>(right_accessor, right_id);
+      if (right_nullable && RightGen::IsNull(right_value)) {
+        result_cv->appendNullValue();
+        continue;
+      }
+      ResultGen::template ApplyBinaryColumnVector<LeftGen, RightGen>(
+          LeftGen::Dereference(left_value),
+          RightGen::Dereference(right_value),
+          functor,
+          result_cv);
+    }
+    return result_cv;
+  }
+
+  template <bool supported, typename AccessorT>
+  inline TypedValue accumulateValueAccessor(const TypedValue &current,
+                                            AccessorT *accessor,
+                                            const attribute_id attr_id,
+                                            std::size_t *num_tuples_applied,
+                                            std::enable_if_t<supported>* = 0) const {
+    DCHECK(num_tuples_applied);
+
+    *num_tuples_applied = 0;
+    if (left_nullable && current.isNull()) {
+      return result_type.makeNullValue();
+    }
+
+    using LeftCppType = typename LeftType::cpptype;
+    using ResultCppType = typename ResultType::cpptype;
+    ResultCppType accumulated = current.getLiteral<LeftCppType>();
+
+    accessor->beginIteration();
+    while (accessor->next()) {
+      typename RightGen::NativeTypeConstPtr right_value =
+          RightGen::template GetValuePtr<
+              right_nullable, AccessorT>(accessor, attr_id);
+      if (right_nullable && RightGen::IsNull(right_value)) {
+        continue;
+      }
+
+      accumulated =
+          ResultGen::FunctorSpecializer::Invoke(
+              functor, accumulated, RightGen::Dereference(right_value));
+      ++(*num_tuples_applied);
+    }
+
+    return TypedValue(accumulated);
+  }
+
+  template <bool supported, typename AccessorT>
+  inline TypedValue accumulateValueAccessor(const TypedValue &current,
+                                            AccessorT *accessor,
+                                            const attribute_id attr_id,
+                                            std::size_t *num_tuples_applied,
+                                            std::enable_if_t<!supported>* = 0) const {
+    LOG(FATAL) << "Unimplemented method UncheckedBinaryOperatorWrapperCodegen"
+               << "::accumulateValueAccessor() because ResultType and LeftType "
+               << "are not same or not native types.";
+  }
+
+  const FunctorT &functor;
+  const Type &left_type;
+  const Type &right_type;
+  const Type &result_type;
+};
+
+template <typename FunctorT>
+class BinaryOperationWrapper : public BinaryOperation {
+ public:
+  BinaryOperationWrapper()
+      : BinaryOperation(),
+        operation_name_(FunctorT::GetName()) {}
+
+  std::string getName() const override {
+    return operation_name_;
+  }
+
+  std::string getShortName() const override {
+    return getName();
+  }
+
+  std::vector<OperationSignaturePtr> getSignatures() const override {
+    return {
+        OperationSignature::Create(
+            getName(), {LeftType::kStaticTypeID, RightType::kStaticTypeID}, 0)
+    };
+  }
+
+  bool canApplyTo(const Type &left,
+                  const Type &right,
+                  const std::vector<TypedValue> &static_arguments,
+                  std::string *message) const override {
+    DCHECK(left.getTypeID() == LeftType::kStaticTypeID);
+    DCHECK(right.getTypeID() == RightType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return true;
+  }
+
+  const Type* getResultType(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(left.getTypeID() == LeftType::kStaticTypeID);
+    DCHECK(right.getTypeID() == RightType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return getResultTypeImpl<ResultType::kParameterized>(
+        left, right, static_arguments);
+  }
+
+  UncheckedBinaryOperator* makeUncheckedBinaryOperator(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(left.getTypeID() == LeftType::kStaticTypeID);
+    DCHECK(right.getTypeID() == RightType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return makeUncheckedBinaryOperatorImpl<
+        std::is_default_constructible<FunctorT>::value>(
+            left, right, static_arguments);
+  }
+
+ private:
+  using LeftType = typename FunctorT::LeftArgumentType;
+  using RightType = typename FunctorT::RightArgumentType;
+  using ResultType = typename FunctorT::ResultType;
+
+  template <bool functor_use_default_constructor>
+  inline UncheckedBinaryOperator* makeUncheckedBinaryOperatorImpl(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<functor_use_default_constructor>* = 0) const {
+    return new UncheckedBinaryOperatorWrapperCodegen<FunctorT>(
+        left, right, *getResultType(left, right, static_arguments));
+  }
+
+  template <bool functor_use_default_constructor>
+  inline UncheckedBinaryOperator* makeUncheckedBinaryOperatorImpl(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<!functor_use_default_constructor>* = 0) const {
+    return new UncheckedBinaryOperatorWrapperCodegen<FunctorT>(
+        left, right, *getResultType(left, right, static_arguments),
+        static_cast<const LeftType&>(left),
+        static_cast<const RightType&>(right));
+  }
+
+  template <bool result_type_has_parameter>
+  inline const Type* getResultTypeImpl(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<!result_type_has_parameter>* = 0) const {
+    return &TypeFactory::GetType(
+        ResultType::kStaticTypeID,
+        left.isNullable() || right.isNullable());
+  }
+
+  template <bool result_type_has_parameter>
+  inline const Type* getResultTypeImpl(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments,
+      std::enable_if_t<result_type_has_parameter>* = 0) const {
+    return FunctorT::GetResultType(left, right);
+  }
+
+  const std::string operation_name_;
+
+  DISALLOW_COPY_AND_ASSIGN(BinaryOperationWrapper);
+};
+
+template <typename LeftPack, typename RightPack,
+          template <typename LeftT,
+                    typename RightT,
+                    typename ResultT> class FunctorT,
+          template <typename LeftT,
+                    typename RightT> class ResultGenerator>
+struct BinaryFunctorCrossProductPack {
+  template <std::size_t l, std::size_t r>
+  inline static OperationPtr GenerateInner() {
+    using LeftType = std::tuple_element_t<l, LeftPack>;
+    using RightType = std::tuple_element_t<r, RightPack>;
+    using ResultType = typename ResultGenerator<LeftType, RightType>::type;
+
+    return std::make_shared<
+        const BinaryOperationWrapper<
+            FunctorT<LeftType, RightType, ResultType>>>();
+  }
+
+  template <std::size_t l, std::size_t ...Rs>
+  inline static std::list<OperationPtr> GenerateRightHelper() {
+    return { GenerateInner<l, Rs>()... };
+  }
+
+  template <std::size_t ...Ls, std::size_t ...Rs>
+  inline static std::vector<std::list<OperationPtr>> GenerateLeftHelper(
+      meta::IntegerSequence<Ls...> &&l_seq, meta::IntegerSequence<Rs...> &&r_seq) {
+    return { GenerateRightHelper<Ls, Rs...>()... };
+  }
+
+  template <typename Dispatcher>
+  inline static std::list<OperationPtr> GenerateOperations() {
+    std::vector<std::list<OperationPtr>> op_list_groups =
+        GenerateLeftHelper(typename meta::MakeSequence<std::tuple_size<LeftPack>::value>::type(),
+                           typename meta::MakeSequence<std::tuple_size<RightPack>::value>::type());
+
+    std::list<OperationPtr> operations;
+    for (std::list<OperationPtr> &op_list : op_list_groups) {
+      operations.splice(operations.end(), std::move(op_list));
+    }
+    return operations;
+  }
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_WRAPPER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/binary_operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/CMakeLists.txt b/types/operations/binary_operations/CMakeLists.txt
index 4d26b75..09566aa 100644
--- a/types/operations/binary_operations/CMakeLists.txt
+++ b/types/operations/binary_operations/CMakeLists.txt
@@ -16,185 +16,97 @@
 # under the License.
 
 # Declare micro-libs:
-add_library(quickstep_types_operations_binaryoperations_AddBinaryOperation
-            AddBinaryOperation.cpp
-            AddBinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
+add_library(quickstep_types_operations_binaryoperations_ArithmeticBinaryOperations
             ../../../empty_src.cpp
-            ArithmeticBinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
+            ArithmeticBinaryOperations.hpp)
+add_library(quickstep_types_operations_binaryoperations_ArithmeticBinaryFunctorOverloads
             ../../../empty_src.cpp
-            ArithmeticBinaryOperators.hpp)
+            ArithmeticBinaryFunctorOverloads.hpp)
+add_library(quickstep_types_operations_binaryoperations_AsciiStringBinaryOperations
+            ../../../empty_src.cpp
+            AsciiStringBinaryOperations.hpp)
 add_library(quickstep_types_operations_binaryoperations_BinaryOperation
             BinaryOperation.cpp
             BinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_BinaryOperationFactory
-            BinaryOperationFactory.cpp
-            BinaryOperationFactory.hpp)
-add_library(quickstep_types_operations_binaryoperations_BinaryOperationID
-            BinaryOperationID.cpp
-            BinaryOperationID.hpp)
-add_library(quickstep_types_operations_binaryoperations_DivideBinaryOperation
-            DivideBinaryOperation.cpp
-            DivideBinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_ModuloBinaryOperation
-            ModuloBinaryOperation.cpp
-            ModuloBinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_MultiplyBinaryOperation
-            MultiplyBinaryOperation.cpp
-            MultiplyBinaryOperation.hpp)
-add_library(quickstep_types_operations_binaryoperations_SubtractBinaryOperation
-            SubtractBinaryOperation.cpp
-            SubtractBinaryOperation.hpp)
+add_library(quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+            ../../../empty_src.cpp
+            BinaryOperationWrapper.hpp)
+add_library(quickstep_types_operations_binaryoperations_CMathBinaryOperations
+            ../../../empty_src.cpp
+            CMathBinaryOperations.hpp)
 
 # Link dependencies:
-target_link_libraries(quickstep_types_operations_binaryoperations_AddBinaryOperation
-                      glog
+target_link_libraries(quickstep_types_operations_binaryoperations_ArithmeticBinaryFunctorOverloads
                       quickstep_types_DateOperatorOverloads
+                      quickstep_utility_meta_Common)
+target_link_libraries(quickstep_types_operations_binaryoperations_ArithmeticBinaryOperations
                       quickstep_types_DateType
                       quickstep_types_DatetimeIntervalType
                       quickstep_types_DatetimeLit
                       quickstep_types_DatetimeType
                       quickstep_types_IntervalLit
+                      quickstep_types_NumericTypeUnifier
                       quickstep_types_Type
                       quickstep_types_TypeErrors
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_types_YearMonthIntervalType
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_EqualsAnyConstant
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
+                      quickstep_types_operations_binaryoperations_ArithmeticBinaryFunctorOverloads
+                      quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+                      quickstep_utility_meta_Common)
+target_link_libraries(quickstep_types_operations_binaryoperations_AsciiStringBinaryOperations
                       glog
-                      quickstep_types_DoubleType
-                      quickstep_types_FloatType
+                      quickstep_types_CharType
                       quickstep_types_IntType
-                      quickstep_types_LongType
-                      quickstep_types_NumericTypeUnifier
                       quickstep_types_Type
-                      quickstep_types_TypeErrors
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
-                      quickstep_types_TypedValue
-                      quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
-                      glog
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_ValueAccessor
-                      quickstep_storage_ValueAccessorUtil
-                      quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
-                      quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_utility_Macros)
+                      quickstep_types_VarCharType
+                      quickstep_types_operations_OperationUtil
+                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper
+                      quickstep_types_port_strnlen)
 target_link_libraries(quickstep_types_operations_binaryoperations_BinaryOperation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_StorageBlockInfo
                       quickstep_types_TypedValue
                       quickstep_types_operations_Operation
-                      quickstep_types_operations_Operation_proto
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      glog
-                      quickstep_types_operations_Operation_proto
-                      quickstep_types_operations_binaryoperations_AddBinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_types_operations_binaryoperations_DivideBinaryOperation
-                      quickstep_types_operations_binaryoperations_ModuloBinaryOperation
-                      quickstep_types_operations_binaryoperations_MultiplyBinaryOperation
-                      quickstep_types_operations_binaryoperations_SubtractBinaryOperation
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_DivideBinaryOperation
-                      glog
-                      quickstep_types_DateOperatorOverloads
-                      quickstep_types_DatetimeIntervalType
-                      quickstep_types_DoubleType
-                      quickstep_types_FloatType
-                      quickstep_types_IntType
-                      quickstep_types_IntervalLit
-                      quickstep_types_LongType
-                      quickstep_types_Type
-                      quickstep_types_TypeErrors
-                      quickstep_types_TypeFactory
-                      quickstep_types_TypeID
-                      quickstep_types_TypedValue
-                      quickstep_types_YearMonthIntervalType
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_EqualsAnyConstant
+                      quickstep_types_operations_OperationSignature
                       quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_ModuloBinaryOperation
+target_link_libraries(quickstep_types_operations_binaryoperations_BinaryOperationWrapper
                       glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
                       quickstep_types_Type
-                      quickstep_types_TypeErrors
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_EqualsAnyConstant
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_MultiplyBinaryOperation
-                      glog
-                      quickstep_types_DateOperatorOverloads
-                      quickstep_types_DatetimeIntervalType
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_operations_OperationSignature
+                      quickstep_types_operations_OperationUtil
+                      quickstep_types_operations_binaryoperations_BinaryOperation
+                      quickstep_utility_Macros
+                      quickstep_utility_meta_Common)
+target_link_libraries(quickstep_types_operations_binaryoperations_CMathBinaryOperations
                       quickstep_types_DoubleType
                       quickstep_types_FloatType
                       quickstep_types_IntType
-                      quickstep_types_IntervalLit
                       quickstep_types_LongType
-                      quickstep_types_Type
-                      quickstep_types_TypeErrors
-                      quickstep_types_TypeFactory
-                      quickstep_types_TypeID
-                      quickstep_types_TypedValue
-                      quickstep_types_YearMonthIntervalType
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_EqualsAnyConstant
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_binaryoperations_SubtractBinaryOperation
-                      glog
-                      quickstep_types_DateOperatorOverloads
-                      quickstep_types_DateType
-                      quickstep_types_DatetimeIntervalType
-                      quickstep_types_DatetimeLit
-                      quickstep_types_DatetimeType
-                      quickstep_types_IntervalLit
-                      quickstep_types_Type
-                      quickstep_types_TypeErrors
-                      quickstep_types_TypeFactory
-                      quickstep_types_TypeID
-                      quickstep_types_TypedValue
-                      quickstep_types_YearMonthIntervalType
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_utility_EqualsAnyConstant
-                      quickstep_utility_Macros)
+                      quickstep_types_operations_OperationUtil
+                      quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+                      quickstep_utility_meta_Common)
 
 # Module all-in-one library:
 add_library(quickstep_types_operations_binaryoperations ../../../empty_src.cpp)
 target_link_libraries(quickstep_types_operations_binaryoperations
                       quickstep_types_operations_binaryoperations_AddBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperation
-                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperators
+                      quickstep_types_operations_binaryoperations_ArithmeticBinaryOperations
+                      quickstep_types_operations_binaryoperations_ArithmeticBinaryFunctorOverloads
+                      quickstep_types_operations_binaryoperations_AsciiStringBinaryOperations
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_types_operations_binaryoperations_DivideBinaryOperation
-                      quickstep_types_operations_binaryoperations_ModuloBinaryOperation
-                      quickstep_types_operations_binaryoperations_MultiplyBinaryOperation
-                      quickstep_types_operations_binaryoperations_SubtractBinaryOperation)
+                      quickstep_types_operations_binaryoperations_BinaryOperationWrapper
+                      quickstep_types_operations_binaryoperations_CMathBinaryOperations)
 
 # Tests:
 add_library(quickstep_types_operations_binaryoperations_tests_BinaryOperationTestUtil
@@ -228,14 +140,7 @@ target_link_libraries(BinaryOperation_tests
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_types_operations_Operation_proto
-                      quickstep_types_operations_binaryoperations_AddBinaryOperation
                       quickstep_types_operations_binaryoperations_BinaryOperation
-                      quickstep_types_operations_binaryoperations_BinaryOperationFactory
-                      quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_types_operations_binaryoperations_DivideBinaryOperation
-                      quickstep_types_operations_binaryoperations_ModuloBinaryOperation
-                      quickstep_types_operations_binaryoperations_MultiplyBinaryOperation
-                      quickstep_types_operations_binaryoperations_SubtractBinaryOperation
                       quickstep_types_operations_binaryoperations_tests_BinaryOperationTestUtil
                       quickstep_utility_Macros)
 add_test(BinaryOperation_tests BinaryOperation_tests)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/binary_operations/CMathBinaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/CMathBinaryOperations.hpp b/types/operations/binary_operations/CMathBinaryOperations.hpp
new file mode 100644
index 0000000..9a90a12
--- /dev/null
+++ b/types/operations/binary_operations/CMathBinaryOperations.hpp
@@ -0,0 +1,78 @@
+/**
+ * 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_CMATH_BINARY_OPERATIONS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_CMATH_BINARY_OPERATIONS_HPP_
+
+#include <cmath>
+#include <string>
+
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/operations/OperationUtil.hpp"
+#include "types/operations/binary_operations/BinaryOperationWrapper.hpp"
+#include "utility/meta/Common.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename LeftT, typename RightT, typename ResultT,
+          typename ResultT::cpptype f(typename LeftT::cpptype,
+                                      typename RightT::cpptype),
+          typename FunctorNameT>
+struct CMathBinaryFunctorWrapper {
+  struct Implemenation : public BinaryFunctor<LeftT, RightT, ResultT> {
+    inline typename ResultT::cpptype apply(
+        const typename LeftT::cpptype &left,
+        const typename RightT::cpptype &right) const {
+      return f(left, right);
+    }
+    inline static std::string GetName() {
+      return FunctorNameT::ToString();
+    }
+  };
+
+  typedef Implemenation type;
+};
+
+template <typename LeftT, typename RightT, typename ResultT,
+          typename ResultT::cpptype f(typename LeftT::cpptype,
+                                      typename RightT::cpptype),
+          typename FunctorNameT>
+using CMathBinaryFunctor =
+    typename CMathBinaryFunctorWrapper<LeftT, RightT, ResultT, f, FunctorNameT>::type;
+
+using CMathBinaryFunctorPack = FunctorPack<
+// pow
+    CMathBinaryFunctor<FloatType, FloatType, FloatType,
+                       std::pow, meta::StringLiteral<'p','o','w'>>,
+    CMathBinaryFunctor<DoubleType, DoubleType, DoubleType,
+                       std::pow, meta::StringLiteral<'p','o','w'>>
+>;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_CMATH_UNARY_OPERATIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/binary_operations/DivideBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/DivideBinaryOperation.cpp b/types/operations/binary_operations/DivideBinaryOperation.cpp
deleted file mode 100644
index 59dcb3e..0000000
--- a/types/operations/binary_operations/DivideBinaryOperation.cpp
+++ /dev/null
@@ -1,391 +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/DivideBinaryOperation.hpp"
-
-#include <string>
-#include <utility>
-
-#include "types/DateOperatorOverloads.hpp"
-#include "types/DatetimeIntervalType.hpp"
-#include "types/DoubleType.hpp"
-#include "types/FloatType.hpp"
-#include "types/IntType.hpp"
-#include "types/IntervalLit.hpp"
-#include "types/LongType.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 DivideBinaryOperation::canApplyToTypes(const Type &left, const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-    case kLong:
-    case kFloat:
-    case kDouble:
-    case kDatetimeInterval:
-    case kYearMonthInterval: {
-      return (right.getSuperTypeID() == Type::kNumeric);
-    }
-    default:
-      return false;
-  }
-}
-
-const Type* DivideBinaryOperation::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() == kDatetimeInterval && right.getSuperTypeID() == Type::kNumeric) {
-    return &(DatetimeIntervalType::Instance(left.isNullable() || right.isNullable()));
-  } else if (left.getTypeID() == kYearMonthInterval && right.getSuperTypeID() == Type::kNumeric) {
-    return &(YearMonthIntervalType::Instance(left.isNullable() || right.isNullable()));
-  } else {
-    return nullptr;
-  }
-}
-
-const Type* DivideBinaryOperation::resultTypeForPartialArgumentTypes(
-    const Type *left,
-    const Type *right) const {
-  if (left == nullptr) {
-    // Can't determine result type just based on right (divisor) type.
-    return nullptr;
-  } else if (right == nullptr) {
-    switch (left->getTypeID()) {
-      case kDouble:
-        // Double has highest precedence of numeric types.
-        return &TypeFactory::GetType(kDouble, true);
-      case kDatetimeInterval:
-        return &TypeFactory::GetType(kDatetimeInterval, true);
-      case kYearMonthInterval:
-        return &TypeFactory::GetType(kYearMonthInterval, true);
-      default:
-        // Ambiguous or inapplicable.
-        return nullptr;
-    }
-  } else {
-    return resultTypeForArgumentTypes(*left, *right);
-  }
-}
-
-bool DivideBinaryOperation::partialTypeSignatureIsPlausible(
-    const Type *result_type,
-    const Type *left_argument_type,
-    const Type *right_argument_type) const {
-  // Early check: if either argument type is nullable or unknown, result type
-  // must also be nullable.
-  if ((left_argument_type == nullptr)
-      || left_argument_type->isNullable()
-      || (right_argument_type == nullptr)
-      || right_argument_type->isNullable()) {
-    if ((result_type != nullptr) && (!result_type->isNullable())) {
-      return false;
-    }
-  }
-
-  if (left_argument_type == nullptr) {
-    if (right_argument_type == nullptr) {
-      if (result_type == nullptr) {
-        // All types unknown.
-        return true;
-      } else {
-        // Only result type is known, just check that it is one of the types
-        // that can possibly be returned.
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-      }
-    }
-
-    if (result_type == nullptr) {
-      // Right (divisor) argument type is known, left (dividend) argument and
-      // result types are unknown. Just check that it is possible to divide by
-      // the right (divisor) type.
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          right_argument_type->getTypeID(),
-          kInt, kLong, kFloat, kDouble);
-    }
-
-    // Return type and right (divisor) argument type are known, left (dividend)
-    // argument type is unknown. Check that result and divisor are compatible.
-    switch (right_argument_type->getTypeID()) {
-      case kInt:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-      case kLong:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kLong, kDouble, kDatetimeInterval, kYearMonthInterval);
-      case kFloat:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-      case kDouble:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kDouble, kDatetimeInterval, kYearMonthInterval);
-      default:
-        return false;
-    }
-  } else {  // left_argument_type != nullptr
-    if (right_argument_type == nullptr) {
-      if (result_type == nullptr) {
-        // Left (dividend) argument type is known, right (divisor) argument
-        // type and result type are unknown. Just check that the left
-        // (dividend) type can be divided.
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            left_argument_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-      }
-
-      // Result type and left (dividend) argument type are known, but right
-      // (divisor) argument type is unknown. Check that result and dividend are
-      // compatible.
-      switch (left_argument_type->getTypeID()) {
-        case kInt:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kInt, kLong, kFloat, kDouble);
-        case kLong:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kLong, kDouble);
-        case kFloat:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kFloat, kDouble);
-        case kDouble:
-          return (result_type->getTypeID() == kDouble);
-        case kDatetimeInterval:
-          return (result_type->getTypeID() == kDatetimeInterval);
-        case kYearMonthInterval:
-          return (result_type->getTypeID() == kYearMonthInterval);
-        default:
-          return false;
-      }
-    }
-
-    // Left and right (dividend and divisor) argument types are both known.
-    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 {
-      // Check if result type matches.
-      return result_type->equals(*actual_result_type);
-    }
-  }
-}
-
-std::pair<const Type*, const Type*> DivideBinaryOperation::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:
-      return std::pair<const Type*, const Type*>(result_type_hint, result_type_hint);
-    case kDatetimeInterval:
-    case kYearMonthInterval:
-      // Left (dividend) should be the same kind of interval as '*type_hint',
-      // right (divisor) can be any numeric type.
-      return std::pair<const Type*, const Type*>(result_type_hint, nullptr);
-    default:
-      // Inapplicable.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-}
-
-TypedValue DivideBinaryOperation::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: {
-      if (right_type.getSuperTypeID() == Type::kNumeric) {
-        return applyToCheckedNumericHelper<DivideFunctor>(left, left_type,
-                                                          right, right_type);
-      }
-      break;
-    }
-    case kDatetimeInterval: {
-      switch (right_type.getTypeID()) {
-        case kInt:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kDatetimeInterval);
-          }
-
-          return TypedValue(left.getLiteral<DatetimeIntervalLit>() / right.getLiteral<IntType::cpptype>());
-        case kLong:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kDatetimeInterval);
-          }
-
-          return TypedValue(left.getLiteral<DatetimeIntervalLit>() / right.getLiteral<LongType::cpptype>());
-        case kFloat:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kDatetimeInterval);
-          }
-
-          return TypedValue(left.getLiteral<DatetimeIntervalLit>() / right.getLiteral<FloatType::cpptype>());
-        case kDouble:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kDatetimeInterval);
-          }
-
-          return TypedValue(left.getLiteral<DatetimeIntervalLit>() / right.getLiteral<DoubleType::cpptype>());
-        default:
-          break;
-      }
-      break;
-    }
-    case kYearMonthInterval: {
-      switch (right_type.getTypeID()) {
-        case kInt:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kYearMonthInterval);
-          }
-
-          return TypedValue(left.getLiteral<YearMonthIntervalLit>() / right.getLiteral<IntType::cpptype>());
-        case kLong:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kYearMonthInterval);
-          }
-
-          return TypedValue(left.getLiteral<YearMonthIntervalLit>() / right.getLiteral<LongType::cpptype>());
-        case kFloat:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kYearMonthInterval);
-          }
-
-          return TypedValue(left.getLiteral<YearMonthIntervalLit>() / right.getLiteral<FloatType::cpptype>());
-        case kDouble:
-          if (left.isNull() || right.isNull()) {
-            return TypedValue(kYearMonthInterval);
-          }
-
-          return TypedValue(left.getLiteral<YearMonthIntervalLit>() / right.getLiteral<DoubleType::cpptype>());
-        default:
-          break;
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-             << left_type.getName() << " and " << right_type.getName();
-}
-
-UncheckedBinaryOperator* DivideBinaryOperation::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<DivideArithmeticUncheckedBinaryOperator>(left, right);
-      }
-      break;
-    }
-    case kDatetimeInterval: {
-      switch (right.getTypeID()) {
-        case kInt: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   DatetimeIntervalType,
-                                                   DatetimeIntervalLit, IntType::cpptype>(left, right);
-        }
-        case kLong: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   DatetimeIntervalType,
-                                                   DatetimeIntervalLit, LongType::cpptype>(left, right);
-        }
-        case kFloat: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   DatetimeIntervalType,
-                                                   DatetimeIntervalLit, FloatType::cpptype>(left, right);
-        }
-        case kDouble: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   DatetimeIntervalType,
-                                                   DatetimeIntervalLit, DoubleType::cpptype>(left, right);
-        }
-        default:
-          break;
-      }
-      break;
-    }
-    case kYearMonthInterval: {
-      switch (right.getTypeID()) {
-        case kInt: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   YearMonthIntervalType,
-                                                   YearMonthIntervalLit, IntType::cpptype>(left, right);
-        }
-        case kLong: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   YearMonthIntervalType,
-                                                   YearMonthIntervalLit, LongType::cpptype>(left, right);
-        }
-        case kFloat: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   YearMonthIntervalType,
-                                                   YearMonthIntervalLit, FloatType::cpptype>(left, right);
-        }
-        case kDouble: {
-          return makeDateBinaryOperatorOuterHelper<DivideArithmeticUncheckedBinaryOperator,
-                                                   YearMonthIntervalType,
-                                                   YearMonthIntervalLit, DoubleType::cpptype>(left, right);
-        }
-        default:
-          break;
-      }
-      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/da9baf7e/types/operations/binary_operations/DivideBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/DivideBinaryOperation.hpp b/types/operations/binary_operations/DivideBinaryOperation.hpp
deleted file mode 100644
index 3ab7aa1..0000000
--- a/types/operations/binary_operations/DivideBinaryOperation.hpp
+++ /dev/null
@@ -1,93 +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_DIVIDE_BINARY_OPERATION_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_DIVIDE_BINARY_OPERATION_HPP_
-
-#include <utility>
-
-#include "types/TypedValue.hpp"
-#include "types/operations/binary_operations/ArithmeticBinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-class Type;
-class UncheckedBinaryOperator;
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief The BinaryOperation for division.
- *
- * @note DivideBinaryOperation is not commutative: the left argument is the
- *       dividend and the right argument is the divisor.
- **/
-class DivideBinaryOperation : public ArithmeticBinaryOperation {
- public:
-  /**
-   * @brief Get a reference to the singleton instance of this Operation.
-   *
-   * @return A reference to the singleton instance of this Operation.
-   **/
-  static const DivideBinaryOperation& Instance() {
-    static DivideBinaryOperation instance;
-    return instance;
-  }
-
-  bool canApplyToTypes(const Type &left,
-                       const Type &right) const override;
-
-  const Type* resultTypeForArgumentTypes(const Type &left,
-                                         const Type &right) const override;
-
-  const Type* resultTypeForPartialArgumentTypes(const Type *left,
-                                                const Type *right) const override;
-
-  bool partialTypeSignatureIsPlausible(const Type *result_type,
-                                       const Type *left_argument_type,
-                                       const Type *right_argument_type) const override;
-
-  std::pair<const Type*, const Type*> pushDownTypeHint(
-      const Type *result_type_hint) const override;
-
-  TypedValue applyToChecked(const TypedValue &left,
-                            const Type &left_type,
-                            const TypedValue &right,
-                            const Type &right_type) const override;
-
-  UncheckedBinaryOperator* makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                               const Type &right) const override;
-
- private:
-  DivideBinaryOperation()
-      : ArithmeticBinaryOperation(BinaryOperationID::kDivide) {
-  }
-
-  DISALLOW_COPY_AND_ASSIGN(DivideBinaryOperation);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_DIVIDE_BINARY_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/binary_operations/ModuloBinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ModuloBinaryOperation.cpp b/types/operations/binary_operations/ModuloBinaryOperation.cpp
deleted file mode 100644
index 917c6c9..0000000
--- a/types/operations/binary_operations/ModuloBinaryOperation.cpp
+++ /dev/null
@@ -1,259 +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/ModuloBinaryOperation.hpp"
-
-#include <string>
-#include <utility>
-
-#include "types/Type.hpp"
-#include "types/TypeErrors.hpp"
-#include "types/TypeFactory.hpp"
-#include "types/TypeID.hpp"
-#include "types/operations/binary_operations/ArithmeticBinaryOperators.hpp"
-#include "utility/EqualsAnyConstant.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-bool ModuloBinaryOperation::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);
-    }
-    // TODO(jianqiao): Extend modulo operator to be applicable to DatetimeInterval
-    // and YearMonthInterval.
-    default:
-      return false;
-  }
-}
-
-const Type* ModuloBinaryOperation::resultTypeForArgumentTypes(const Type &left, const Type &right) const {
-  if (left.getSuperTypeID() == Type::kNumeric && right.getSuperTypeID() == Type::kNumeric) {
-    return TypeFactory::GetUnifyingType(left, right);
-  } else {
-    return nullptr;
-  }
-}
-
-const Type* ModuloBinaryOperation::resultTypeForPartialArgumentTypes(
-    const Type *left,
-    const Type *right) const {
-  if (left == nullptr) {
-    // Can't determine result type just based on right (divisor) type.
-    return nullptr;
-  } else if (right == nullptr) {
-    switch (left->getTypeID()) {
-      case kDouble:
-        // Double has highest precedence of numeric types.
-        return &TypeFactory::GetType(kDouble, true);
-      default:
-        // Ambiguous or inapplicable.
-        return nullptr;
-    }
-  } else {
-    return resultTypeForArgumentTypes(*left, *right);
-  }
-}
-
-bool ModuloBinaryOperation::partialTypeSignatureIsPlausible(
-    const Type *result_type,
-    const Type *left_argument_type,
-    const Type *right_argument_type) const {
-  // Early check: if either argument type is nullable or unknown, result type
-  // must also be nullable.
-  if ((left_argument_type == nullptr)
-      || left_argument_type->isNullable()
-      || (right_argument_type == nullptr)
-      || right_argument_type->isNullable()) {
-    if ((result_type != nullptr) && (!result_type->isNullable())) {
-      return false;
-    }
-  }
-
-  if (left_argument_type == nullptr) {
-    if (right_argument_type == nullptr) {
-      if (result_type == nullptr) {
-        // All types unknown.
-        return true;
-      } else {
-        // Only result type is known, just check that it is one of the types
-        // that can possibly be returned.
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble);
-      }
-    }
-
-    if (result_type == nullptr) {
-      // Right (divisor) argument type is known, left (dividend) argument and
-      // result types are unknown. Just check that it is possible to divide by
-      // the right (divisor) type.
-      return QUICKSTEP_EQUALS_ANY_CONSTANT(
-          right_argument_type->getTypeID(),
-          kInt, kLong, kFloat, kDouble);
-    }
-
-    // Return type and right (divisor) argument type are known, left (dividend)
-    // argument type is unknown. Check that result and divisor are compatible.
-    switch (right_argument_type->getTypeID()) {
-      case kInt:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble);
-      case kLong:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kLong, kDouble);
-      case kFloat:
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            result_type->getTypeID(),
-            kFloat, kDouble);
-      case kDouble:
-        return (result_type->getTypeID() == kDouble);
-      default:
-        return false;
-    }
-  } else {  // left_argument_type != nullptr
-    if (right_argument_type == nullptr) {
-      if (result_type == nullptr) {
-        // Left (dividend) argument type is known, right (divisor) argument
-        // type and result type are unknown. Just check that the left
-        // (dividend) type can be divided.
-        return QUICKSTEP_EQUALS_ANY_CONSTANT(
-            left_argument_type->getTypeID(),
-            kInt, kLong, kFloat, kDouble);
-      }
-
-      // Result type and left (dividend) argument type are known, but right
-      // (divisor) argument type is unknown. Check that result and dividend are
-      // compatible.
-      switch (left_argument_type->getTypeID()) {
-        case kInt:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kInt, kLong, kFloat, kDouble);
-        case kLong:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kLong, kDouble);
-        case kFloat:
-          return QUICKSTEP_EQUALS_ANY_CONSTANT(
-              result_type->getTypeID(),
-              kFloat, kDouble);
-        case kDouble:
-          return (result_type->getTypeID() == kDouble);
-        default:
-          return false;
-      }
-    }
-
-    // Left and right (dividend and divisor) argument types are both known.
-    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 {
-      // Check if result type matches.
-      return result_type->equals(*actual_result_type);
-    }
-  }
-}
-
-std::pair<const Type*, const Type*> ModuloBinaryOperation::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:
-      return std::pair<const Type*, const Type*>(result_type_hint, result_type_hint);
-    default:
-      // Inapplicable.
-      return std::pair<const Type*, const Type*>(nullptr, nullptr);
-  }
-}
-
-TypedValue ModuloBinaryOperation::applyToChecked(const TypedValue &left,
-                                                 const Type &left_type,
-                                                 const TypedValue &right,
-                                                 const Type &right_type) const {
-  switch (left_type.getTypeID()) {
-    case kInt:
-    case kLong: {
-      if (right_type.getTypeID() == TypeID::kInt
-          || right_type.getTypeID() == TypeID::kLong) {
-        return applyToCheckedIntegerHelper<IntegerModuloFunctor>(left, left_type,
-                                                                 right, right_type);
-      }
-    }  // Fall through
-    case kFloat:
-    case kDouble: {
-      if (right_type.getSuperTypeID() == Type::kNumeric) {
-        return applyToCheckedNumericHelper<FloatModuloFunctor>(left, left_type,
-                                                               right, right_type);
-      }
-      break;
-    }
-    default:
-      break;
-  }
-
-  LOG(FATAL) << "Can not apply " << getName() << " to arguments of types "
-             << left_type.getName() << " and " << right_type.getName();
-}
-
-UncheckedBinaryOperator* ModuloBinaryOperation::makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                                                    const Type &right) const {
-  switch (left.getTypeID()) {
-    case kInt:
-    case kLong: {
-      if (right.getTypeID() == TypeID::kInt
-          || right.getTypeID() == TypeID::kLong) {
-        return makeIntegerBinaryOperatorOuterHelper<IntegerModuloArithmeticUncheckedBinaryOperator>(left, right);
-      }
-    }  // Fall through
-    case kFloat:
-    case kDouble: {
-      if (right.getSuperTypeID() == Type::kNumeric) {
-        return makeNumericBinaryOperatorOuterHelper<FloatModuloArithmeticUncheckedBinaryOperator>(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/da9baf7e/types/operations/binary_operations/ModuloBinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ModuloBinaryOperation.hpp b/types/operations/binary_operations/ModuloBinaryOperation.hpp
deleted file mode 100644
index 1d47a45..0000000
--- a/types/operations/binary_operations/ModuloBinaryOperation.hpp
+++ /dev/null
@@ -1,93 +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_MODULO_BINARY_OPERATION_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_MODULO_BINARY_OPERATION_HPP_
-
-#include <utility>
-
-#include "types/TypedValue.hpp"
-#include "types/operations/binary_operations/ArithmeticBinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
-namespace quickstep {
-
-class Type;
-class UncheckedBinaryOperator;
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief The BinaryOperation for modulo.
- *
- * @note ModuloBinaryOperation is not commutative: the left argument is the
- *       dividend and the right argument is the divisor.
- **/
-class ModuloBinaryOperation : public ArithmeticBinaryOperation {
- public:
-  /**
-   * @brief Get a reference to the singleton instance of this Operation.
-   *
-   * @return A reference to the singleton instance of this Operation.
-   **/
-  static const ModuloBinaryOperation& Instance() {
-    static ModuloBinaryOperation instance;
-    return instance;
-  }
-
-  bool canApplyToTypes(const Type &left,
-                       const Type &right) const override;
-
-  const Type* resultTypeForArgumentTypes(const Type &left,
-                                         const Type &right) const override;
-
-  const Type* resultTypeForPartialArgumentTypes(const Type *left,
-                                                const Type *right) const override;
-
-  bool partialTypeSignatureIsPlausible(const Type *result_type,
-                                       const Type *left_argument_type,
-                                       const Type *right_argument_type) const override;
-
-  std::pair<const Type*, const Type*> pushDownTypeHint(
-      const Type *result_type_hint) const override;
-
-  TypedValue applyToChecked(const TypedValue &left,
-                            const Type &left_type,
-                            const TypedValue &right,
-                            const Type &right_type) const override;
-
-  UncheckedBinaryOperator* makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                               const Type &right) const override;
-
- private:
-  ModuloBinaryOperation()
-      : ArithmeticBinaryOperation(BinaryOperationID::kModulo) {
-  }
-
-  DISALLOW_COPY_AND_ASSIGN(ModuloBinaryOperation);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_MODULO_BINARY_OPERATION_HPP_