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:23 UTC

[20/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/ArithmeticBinaryOperators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryOperators.hpp b/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
deleted file mode 100644
index 7224a0c..0000000
--- a/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
+++ /dev/null
@@ -1,848 +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_OPERATORS_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATORS_HPP_
-
-#include <cmath>
-#include <cstddef>
-#include <cstdint>
-#include <type_traits>
-#include <utility>
-#include <vector>
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/StorageBlockInfo.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
-#include "types/operations/binary_operations/BinaryOperation.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-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 LeftArgument, typename RightArgument> struct AddFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &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 AddFunctor<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 AddFunctor<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 LeftArgument, typename RightArgument> struct SubtractFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &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 SubtractFunctor<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 SubtractFunctor<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 LeftArgument, typename RightArgument> struct MultiplyFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &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 MultiplyFunctor<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 MultiplyFunctor<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 LeftArgument, typename RightArgument> struct DivideFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &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 DivideFunctor<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 DivideFunctor<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 LeftArgument, typename RightArgument> struct IntegerModuloFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &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 LeftArgument, typename RightArgument> struct FloatModuloFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &right) const
-      -> decltype(std::fmod(left, right)) {
-    return std::fmod(left, right);
-  }
-};
-
-template <template <typename LeftCppType, typename RightCppType> class OpFunctor,
-          typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-class ArithmeticUncheckedBinaryOperator : public UncheckedBinaryOperator {
- public:
-  ArithmeticUncheckedBinaryOperator() = default;
-  ArithmeticUncheckedBinaryOperator(const ArithmeticUncheckedBinaryOperator &orig) = default;
-  ~ArithmeticUncheckedBinaryOperator() = default;
-
-  inline TypedValue applyToTypedValues(const TypedValue &left,
-                                       const TypedValue &right) const override {
-    return applyToTypedValuesInl(left, right);
-  }
-
-  // NOTE(chasseur): This inline version does NOT override a virtual in the
-  // base class. g++ (and probably other compilers) will not inline a method
-  // that overrides a virtual, so we use this instead when we cast to a
-  // specific subclass so that we can actually inline the call.
-  inline TypedValue applyToTypedValuesInl(const TypedValue &left,
-                                          const TypedValue &right) const {
-    if ((left_nullable && left.isNull()) || (right_nullable && right.isNull())) {
-      return TypedValue(ResultType::kStaticTypeID);
-    }
-    return TypedValue(op_functor_(left.getLiteral<LeftCppType>(),
-                                  right.getLiteral<RightCppType>()));
-  }
-
-  inline TypedValue applyToDataPtrs(const void *left, const void *right) const override {
-    return applyToDataPtrsInl(left, right);
-  }
-
-  // See above note about inlines.
-  inline TypedValue applyToDataPtrsInl(const void *left, const void *right) const {
-    if ((left_nullable && (left == nullptr)) || (right_nullable && (right == nullptr))) {
-      return TypedValue(ResultType::kStaticTypeID);
-    }
-
-    return TypedValue(op_functor_(*static_cast<const LeftCppType*>(left),
-                                  *static_cast<const RightCppType*>(right)));
-  }
-
-  ColumnVector* applyToColumnVectors(
-      const ColumnVector &left,
-      const ColumnVector &right) const override {
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-    // All arithmetic types (numbers, datetime, and intervals) are usable with
-    // NativeColumnVector, so 'left' and 'right' should always be native.
-    DCHECK(left.isNative());
-    DCHECK(right.isNative());
-
-    const NativeColumnVector &left_native = static_cast<const NativeColumnVector&>(left);
-    const NativeColumnVector &right_native = static_cast<const NativeColumnVector&>(right);
-
-    DCHECK_EQ(left_native.size(), right_native.size());
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(left_nullable || right_nullable),
-        left_native.size());
-    for (std::size_t pos = 0;
-         pos < left_native.size();
-         ++pos) {
-      const LeftCppType *left_value
-          = static_cast<const LeftCppType*>(
-              left_native.getUntypedValue<left_nullable>(pos));
-      if (left_nullable && (left_value == nullptr)) {
-        result->appendNullValue();
-        continue;
-      }
-      const RightCppType *right_value
-          = static_cast<const RightCppType*>(
-              right_native.getUntypedValue<right_nullable>(pos));
-      if (right_nullable && (right_value == nullptr)) {
-        result->appendNullValue();
-        continue;
-      }
-      *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-          = op_functor_(*left_value, *right_value);
-    }
-    return result;
-  }
-
-  ColumnVector* applyToColumnVectorAndStaticValue(
-      const ColumnVector &left,
-      const TypedValue &right) const override {
-    return applyToColumnVectorAndStaticValueHelper<true>(left, right);
-  }
-
-  ColumnVector* applyToStaticValueAndColumnVector(
-      const TypedValue &left,
-      const ColumnVector &right) const override {
-    return applyToColumnVectorAndStaticValueHelper<false>(right, left);
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  ColumnVector* applyToSingleValueAccessor(
-      ValueAccessor *accessor,
-      const attribute_id left_id,
-      const attribute_id right_id) const override {
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        accessor,
-        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-      NativeColumnVector *result = new NativeColumnVector(
-          ResultType::Instance(left_nullable || right_nullable),
-          accessor->getNumTuples());
-      accessor->beginIteration();
-      while (accessor->next()) {
-        const LeftCppType *left_value = static_cast<const LeftCppType*>(
-            accessor->template getUntypedValue<left_nullable>(left_id));
-        if (left_nullable && (left_value == nullptr)) {
-          result->appendNullValue();
-          continue;
-        }
-        const RightCppType *right_value = static_cast<const RightCppType*>(
-            accessor->template getUntypedValue<right_nullable>(right_id));
-        if (right_nullable && (right_value == nullptr)) {
-          result->appendNullValue();
-          continue;
-        }
-        *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-            = op_functor_(*left_value, *right_value);
-      }
-      return result;
-    });
-  }
-
-  ColumnVector* applyToValueAccessorAndStaticValue(
-      ValueAccessor *left_accessor,
-      const attribute_id left_id,
-      const TypedValue &right) const override {
-    return applyToValueAccessorAndStaticValueHelper<true>(left_accessor, left_id, right);
-  }
-
-  ColumnVector* applyToStaticValueAndValueAccessor(
-      const TypedValue &left,
-      ValueAccessor *right_accessor,
-      const attribute_id right_id) const override {
-    return applyToValueAccessorAndStaticValueHelper<false>(right_accessor, right_id, left);
-  }
-
-  ColumnVector* applyToColumnVectorAndValueAccessor(
-      const ColumnVector &left,
-      ValueAccessor *right_accessor,
-      const attribute_id right_id) const override {
-    return applyToColumnVectorAndValueAccessorHelper<true>(left, right_accessor, right_id);
-  }
-
-  ColumnVector* applyToValueAccessorAndColumnVector(
-      ValueAccessor *left_accessor,
-      const attribute_id left_id,
-      const ColumnVector &right) const override {
-    return applyToColumnVectorAndValueAccessorHelper<false>(right, left_accessor, left_id);
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  ColumnVector* applyToValueAccessorAndStaticValueForJoin(
-      ValueAccessor *left_accessor,
-      const bool left_accessor_is_left_relation,
-      const attribute_id left_id,
-      const TypedValue &right,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    return applyToValueAccessorAndStaticValueForJoinHelper<true>(left_accessor,
-                                                                 left_accessor_is_left_relation,
-                                                                 left_id,
-                                                                 right,
-                                                                 joined_tuple_ids);
-  }
-
-  ColumnVector* applyToStaticValueAndValueAccessorForJoin(
-      const TypedValue &left,
-      ValueAccessor *right_accessor,
-      const bool right_accessor_is_left_relation,
-      const attribute_id right_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    return applyToValueAccessorAndStaticValueForJoinHelper<false>(right_accessor,
-                                                                  right_accessor_is_left_relation,
-                                                                  right_id,
-                                                                  left,
-                                                                  joined_tuple_ids);
-  }
-
-  ColumnVector* applyToColumnVectorAndValueAccessorForJoin(
-      const ColumnVector &left,
-      ValueAccessor *right_accessor,
-      const bool right_accessor_is_left_relation,
-      const attribute_id right_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    return applyToColumnVectorAndValueAccessorForJoinHelper<true>(left,
-                                                                  right_accessor,
-                                                                  right_accessor_is_left_relation,
-                                                                  right_id,
-                                                                  joined_tuple_ids);
-  }
-
-  ColumnVector* applyToValueAccessorAndColumnVectorForJoin(
-      ValueAccessor *left_accessor,
-      const bool left_accessor_is_left_relation,
-      const attribute_id left_id,
-      const ColumnVector &right,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    return applyToColumnVectorAndValueAccessorForJoinHelper<false>(right,
-                                                                   left_accessor,
-                                                                   left_accessor_is_left_relation,
-                                                                   left_id,
-                                                                   joined_tuple_ids);
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN_WITH_BINARY_EXPRESSIONS
-  ColumnVector* applyToValueAccessorsForJoin(
-      ValueAccessor *left_accessor,
-      const bool left_accessor_is_left_relation,
-      const attribute_id left_id,
-      ValueAccessor *right_accessor,
-      const bool right_accessor_is_left_relation,
-      const attribute_id right_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(left_nullable || right_nullable),
-        joined_tuple_ids.size());
-    InvokeOnValueAccessorNotAdapter(
-        left_accessor,
-        [&](auto *left_accessor) -> void {  // NOLINT(build/c++11)
-      InvokeOnValueAccessorNotAdapter(
-          right_accessor,
-          [&](auto *right_accessor) -> void {  // NOLINT(build/c++11)
-        for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-          const LeftCppType *left_value = static_cast<const LeftCppType*>(
-              left_accessor->template getUntypedValueAtAbsolutePosition<left_nullable>(
-                  left_id,
-                  left_accessor_is_left_relation ? joined_pair.first
-                                                 : joined_pair.second));
-          if (left_nullable && (left_value == nullptr)) {
-            result->appendNullValue();
-            continue;
-          }
-          const RightCppType *right_value = static_cast<const RightCppType*>(
-              right_accessor->template getUntypedValueAtAbsolutePosition<right_nullable>(
-                  right_id,
-                  right_accessor_is_left_relation ? joined_pair.first
-                                                  : joined_pair.second));
-          if (right_nullable && (right_value == nullptr)) {
-            result->appendNullValue();
-            continue;
-          }
-          *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-              = op_functor_(*left_value, *right_value);
-        }
-      });
-    });
-    return result;
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN_WITH_BINARY_EXPRESSIONS
-
-  TypedValue accumulateColumnVector(
-      const TypedValue &current,
-      const ColumnVector &column_vector,
-      std::size_t *num_tuples_applied) const override {
-    return accumulateColumnVectorHelper(
-        current,
-        column_vector,
-        num_tuples_applied,
-        std::is_same<LeftCppType, typename ResultType::cpptype>());
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  TypedValue accumulateValueAccessor(
-      const TypedValue &current,
-      ValueAccessor *accessor,
-      const attribute_id value_accessor_id,
-      std::size_t *num_tuples_applied) const override {
-    return accumulateValueAccessorHelper(
-        current,
-        accessor,
-        value_accessor_id,
-        num_tuples_applied,
-        std::is_same<LeftCppType, typename ResultType::cpptype>());
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
- private:
-  template <bool column_vector_on_left>
-  ColumnVector* applyToColumnVectorAndStaticValueHelper(
-      const ColumnVector &column_vector,
-      const TypedValue &static_value) const {
-    typedef typename std::conditional<column_vector_on_left,
-                                      RightCppType,
-                                      LeftCppType>::type
-        StaticValueCppType;
-
-    constexpr bool cv_nullable = column_vector_on_left ? left_nullable : right_nullable;
-    constexpr bool static_value_nullable = column_vector_on_left ? right_nullable : left_nullable;
-
-    DCHECK(column_vector.isNative());
-    const NativeColumnVector &native_column_vector = static_cast<const NativeColumnVector&>(column_vector);
-
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(left_nullable || right_nullable),
-        native_column_vector.size());
-    if (static_value_nullable && static_value.isNull()) {
-      result->fillWithNulls();
-      return result;
-    }
-    const StaticValueCppType literal = static_value.getLiteral<StaticValueCppType>();
-    for (std::size_t pos = 0;
-         pos < native_column_vector.size();
-         ++pos) {
-      const void* cv_value = native_column_vector.getUntypedValue<cv_nullable>(pos);
-      if (cv_nullable && (cv_value == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-            = castAndApply<column_vector_on_left>(cv_value, &literal);
-      }
-    }
-    return result;
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  template <bool value_accessor_on_left>
-  ColumnVector* applyToValueAccessorAndStaticValueHelper(
-      ValueAccessor *value_accessor,
-      const attribute_id value_accessor_attr_id,
-      const TypedValue &static_value) const {
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-
-    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        value_accessor,
-        [&](auto *value_accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-      typedef typename std::conditional<value_accessor_on_left,
-                                        RightCppType,
-                                        LeftCppType>::type
-          StaticValueCppType;
-
-      constexpr bool va_nullable = value_accessor_on_left ? left_nullable : right_nullable;
-      constexpr bool static_value_nullable = value_accessor_on_left ? right_nullable : left_nullable;
-
-      NativeColumnVector *result = new NativeColumnVector(
-          ResultType::Instance(left_nullable || right_nullable),
-          value_accessor->getNumTuples());
-      if (static_value_nullable && static_value.isNull()) {
-        result->fillWithNulls();
-        return result;
-      }
-      const StaticValueCppType literal = static_value.getLiteral<StaticValueCppType>();
-      value_accessor->beginIteration();
-      while (value_accessor->next()) {
-        const void* va_value
-            = value_accessor->template getUntypedValue<va_nullable>(value_accessor_attr_id);
-        if (va_nullable && (va_value == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-              = this->castAndApply<value_accessor_on_left>(va_value, &literal);
-        }
-      }
-      return result;
-    });
-  }
-
-  template <bool column_vector_on_left>
-  ColumnVector* applyToColumnVectorAndValueAccessorHelper(
-      const ColumnVector &column_vector,
-      ValueAccessor *value_accessor,
-      const attribute_id value_accessor_attr_id) const {
-    DCHECK(column_vector.isNative());
-    const NativeColumnVector &native_column_vector = static_cast<const NativeColumnVector&>(column_vector);
-
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        value_accessor,
-        [&](auto *value_accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-      constexpr bool cv_nullable = column_vector_on_left ? left_nullable : right_nullable;
-      constexpr bool va_nullable = column_vector_on_left ? right_nullable : left_nullable;
-
-      DCHECK_EQ(native_column_vector.size(),
-                static_cast<std::size_t>(value_accessor->getNumTuples()));
-      NativeColumnVector *result = new NativeColumnVector(
-          ResultType::Instance(left_nullable || right_nullable),
-          native_column_vector.size());
-      std::size_t cv_pos = 0;
-      value_accessor->beginIteration();
-      while (value_accessor->next()) {
-        const void *cv_value = native_column_vector.getUntypedValue<cv_nullable>(cv_pos);
-        if (cv_nullable && (cv_value == nullptr)) {
-          result->appendNullValue();
-          ++cv_pos;
-          continue;
-        }
-        const void *va_value
-            = value_accessor->template getUntypedValue<va_nullable>(value_accessor_attr_id);
-        if (va_nullable && (va_value == nullptr)) {
-          result->appendNullValue();
-          ++cv_pos;
-          continue;
-        }
-        *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-            = this->castAndApply<column_vector_on_left>(cv_value, va_value);
-        ++cv_pos;
-      }
-      return result;
-    });
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  template <bool value_accessor_on_left>
-  ColumnVector* applyToValueAccessorAndStaticValueForJoinHelper(
-      ValueAccessor *value_accessor,
-      const bool accessor_is_left_relation,
-      const attribute_id value_accessor_attr_id,
-      const TypedValue &static_value,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
-    typedef typename std::conditional<value_accessor_on_left,
-                                      RightCppType,
-                                      LeftCppType>::type
-        StaticValueCppType;
-
-    constexpr bool static_value_nullable = value_accessor_on_left ? right_nullable : left_nullable;
-
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(left_nullable || right_nullable),
-        joined_tuple_ids.size());
-    if (static_value_nullable && static_value.isNull()) {
-      result->fillWithNulls();
-      return result;
-    }
-
-    const StaticValueCppType literal = static_value.getLiteral<StaticValueCppType>();
-
-    InvokeOnValueAccessorNotAdapter(
-        value_accessor,
-        [&](auto *value_accessor) -> void {  // NOLINT(build/c++11)
-      constexpr bool va_nullable = value_accessor_on_left ? left_nullable : right_nullable;
-
-      for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-        const void* va_value
-            = value_accessor->template getUntypedValueAtAbsolutePosition<va_nullable>(
-                value_accessor_attr_id,
-                accessor_is_left_relation ? joined_pair.first : joined_pair.second);
-        if (va_nullable && (va_value == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-              = this->castAndApply<value_accessor_on_left>(va_value, &literal);
-        }
-      }
-    });
-    return result;
-  }
-
-  template <bool column_vector_on_left>
-  ColumnVector* applyToColumnVectorAndValueAccessorForJoinHelper(
-      const ColumnVector &column_vector,
-      ValueAccessor *value_accessor,
-      const bool accessor_is_left_relation,
-      const attribute_id value_accessor_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
-    DCHECK(column_vector.isNative());
-    const NativeColumnVector &native_column_vector = static_cast<const NativeColumnVector&>(column_vector);
-    DCHECK_EQ(native_column_vector.size(), joined_tuple_ids.size());
-
-    DCHECK(NativeColumnVector::UsableForType(
-        ResultType::Instance(left_nullable || right_nullable)));
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(left_nullable || right_nullable),
-        native_column_vector.size());
-    InvokeOnValueAccessorNotAdapter(
-        value_accessor,
-        [&](auto *value_accessor) -> void {  // NOLINT(build/c++11)
-      constexpr bool cv_nullable = column_vector_on_left ? left_nullable : right_nullable;
-      constexpr bool va_nullable = column_vector_on_left ? right_nullable : left_nullable;
-
-      for (std::size_t pos = 0; pos < native_column_vector.size(); ++pos) {
-        const void *cv_value = native_column_vector.getUntypedValue<cv_nullable>(pos);
-        if (cv_nullable && (cv_value == nullptr)) {
-          result->appendNullValue();
-          continue;
-        }
-        const void *va_value
-            = value_accessor->template getUntypedValueAtAbsolutePosition<va_nullable>(
-                value_accessor_attr_id,
-                accessor_is_left_relation ? joined_tuple_ids[pos].first
-                                          : joined_tuple_ids[pos].second);
-        if (va_nullable && (va_value == nullptr)) {
-          result->appendNullValue();
-          continue;
-        }
-        *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-            = this->castAndApply<column_vector_on_left>(cv_value, va_value);
-      }
-    });
-    return result;
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-  // Actual implementation of accumulator.
-  TypedValue accumulateColumnVectorHelper(
-      const TypedValue &current,
-      const ColumnVector &column_vector,
-      std::size_t *num_tuples_applied,
-      std::true_type) const {
-    DCHECK(num_tuples_applied);
-
-    *num_tuples_applied = 0;
-    if (left_nullable && current.isNull()) {
-      return ResultType::Instance(left_nullable).makeNullValue();
-    }
-
-    LeftCppType accumulated = current.getLiteral<LeftCppType>();
-
-    DCHECK(column_vector.isNative());
-    const NativeColumnVector &native_column_vector = static_cast<const NativeColumnVector&>(column_vector);
-    for (std::size_t pos = 0;
-         pos < native_column_vector.size();
-         ++pos) {
-      const RightCppType *value = static_cast<const RightCppType *>(
-        native_column_vector.getUntypedValue<right_nullable>(pos));
-      if ((right_nullable && value) || !right_nullable) {
-        accumulated = op_functor_(accumulated, *value);
-        ++(*num_tuples_applied);
-      }
-    }
-
-    // Note ResultType::cpptype and LeftCppType are same here.
-    return TypedValue(accumulated);
-  }
-
-  // Unimplemented version of accumlator for use when the result type and left
-  // operand type are not same.
-  TypedValue accumulateColumnVectorHelper(
-      const TypedValue &current,
-      const ColumnVector &column_vector,
-      std::size_t *num_tuples_applied,
-      std::false_type) const {
-    FATAL_ERROR("Unimplemented method UncheckedBinaryOperator::accumulateColumnVectorHelper() "
-        "because ResultType::cpptype and LeftCppType are not same.");
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  // Actual implementation of accumlator.
-  TypedValue accumulateValueAccessorHelper(
-      const TypedValue &current,
-      ValueAccessor *accessor,
-      const attribute_id value_accessor_id,
-      std::size_t *num_tuples_applied,
-      std::true_type) const {
-    DCHECK(num_tuples_applied);
-
-    *num_tuples_applied = 0;
-    if (left_nullable && current.isNull()) {
-      return ResultType::Instance(left_nullable).makeNullValue();
-    }
-
-    LeftCppType accumulated = current.getLiteral<LeftCppType>();
-    InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        accessor,
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-      accessor->beginIteration();
-      while (accessor->next()) {
-        const RightCppType *value = static_cast<const RightCppType *>(
-          accessor->template getUntypedValue<right_nullable>(value_accessor_id));
-        if ((right_nullable && value) || !right_nullable) {
-          accumulated = op_functor_(accumulated, *value);
-          ++(*num_tuples_applied);
-        }
-      }
-    });
-
-    // Note ResultType::cpptype and LeftCppType are same here.
-    return TypedValue(accumulated);
-  }
-
-  // Unimplemented version of accumlator for use when the result type and left
-  // operand type are not same.
-  TypedValue accumulateValueAccessorHelper(
-      const TypedValue &current,
-      ValueAccessor *accessor,
-      const attribute_id value_accessor_id,
-      std::size_t *num_tuples_applied,
-      std::false_type) const {
-    FATAL_ERROR("Unimplemented method UncheckedBinaryOperator::accumulateValueAccessorHelper() "
-        "because ResultType::cpptype and LeftCppType are not same.");
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-  template <bool arguments_in_order>
-  inline typename ResultType::cpptype castAndApply(const void *left, const void *right) const {
-    return op_functor_(*static_cast<const LeftCppType*>(arguments_in_order ? left : right),
-                       *static_cast<const RightCppType*>(arguments_in_order ? right : left));
-  }
-
-  OpFunctor<LeftCppType, RightCppType> op_functor_;
-};
-
-/**
- * @brief The UncheckedBinaryOperator for addition.
- **/
-template <typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-using AddArithmeticUncheckedBinaryOperator
-    = ArithmeticUncheckedBinaryOperator<AddFunctor,
-                                        ResultType,
-                                        LeftCppType, left_nullable,
-                                        RightCppType, right_nullable>;
-
-/**
- * @brief The UncheckedBinaryOperator for subtraction.
- **/
-template <typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-using SubtractArithmeticUncheckedBinaryOperator
-    = ArithmeticUncheckedBinaryOperator<SubtractFunctor,
-                                        ResultType,
-                                        LeftCppType, left_nullable,
-                                        RightCppType, right_nullable>;
-
-/**
- * @brief The UncheckedBinaryOperator for multiplication.
- **/
-template <typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-using MultiplyArithmeticUncheckedBinaryOperator
-    = ArithmeticUncheckedBinaryOperator<MultiplyFunctor,
-                                        ResultType,
-                                        LeftCppType, left_nullable,
-                                        RightCppType, right_nullable>;
-
-/**
- * @brief The UncheckedBinaryOperator for division.
- **/
-template <typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-using DivideArithmeticUncheckedBinaryOperator
-    = ArithmeticUncheckedBinaryOperator<DivideFunctor,
-                                        ResultType,
-                                        LeftCppType, left_nullable,
-                                        RightCppType, right_nullable>;
-
-/**
- * @brief The UncheckedBinaryOperator for integer modulo.
- **/
-template <typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-using IntegerModuloArithmeticUncheckedBinaryOperator
-    = ArithmeticUncheckedBinaryOperator<IntegerModuloFunctor,
-                                        ResultType,
-                                        LeftCppType, left_nullable,
-                                        RightCppType, right_nullable>;
-
-/**
- * @brief The UncheckedBinaryOperator for real number modulo.
- **/
-template <typename ResultType,
-          typename LeftCppType, bool left_nullable,
-          typename RightCppType, bool right_nullable>
-using FloatModuloArithmeticUncheckedBinaryOperator
-    = ArithmeticUncheckedBinaryOperator<FloatModuloFunctor,
-                                        ResultType,
-                                        LeftCppType, left_nullable,
-                                        RightCppType, right_nullable>;
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_ARITHMETIC_BINARY_OPERATORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/binary_operations/AsciiStringBinaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/AsciiStringBinaryOperations.hpp b/types/operations/binary_operations/AsciiStringBinaryOperations.hpp
new file mode 100644
index 0000000..7181bc6
--- /dev/null
+++ b/types/operations/binary_operations/AsciiStringBinaryOperations.hpp
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ASCII_STRING_BINARY_OPERATIONS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ASCII_STRING_BINARY_OPERATIONS_HPP_
+
+#include <cctype>
+#include <cstring>
+#include <string>
+
+#include "types/CharType.hpp"
+#include "types/IntType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/VarCharType.hpp"
+#include "types/operations/OperationUtil.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
+#include "types/port/strnlen.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename LeftT, typename RightT, typename ResultT>
+struct AsciiStringConcatFunctor : public BinaryFunctor<LeftT, RightT, ResultT> {
+  explicit AsciiStringConcatFunctor(const LeftT &left, const RightT &right)
+      : left_max_(left.getStringLength()),
+        right_max_(right.getStringLength()),
+        result_max_(left_max_ + right_max_) {}
+  inline void concat(const void *left, const std::size_t l_len,
+                     const void *right, const std::size_t r_len,
+                     void *result) const {
+    char *result_str = static_cast<char*>(result);
+    std::memcpy(result_str, left, l_len);
+    std::memcpy(result_str + l_len, right, r_len);
+
+    const std::size_t result_len = l_len + r_len;
+    if (ResultT::kStaticTypeID == kVarChar || result_len < result_max_) {
+      result_str[result_len] = 0;
+    }
+  }
+  inline void apply(const void *left, const void *right, void *result) const {
+    concat(left, strnlen(static_cast<const char*>(left), left_max_),
+           right, strnlen(static_cast<const char*>(right), right_max_),
+           result);
+  }
+  inline TypedValue apply(const void *left, const TypedValue &right) const {
+    const std::size_t l_len =
+        strnlen(static_cast<const char*>(left), left_max_);
+    const std::size_t r_len =
+        std::strlen(static_cast<const char*>(right.getOutOfLineData()));
+    const std::size_t buf_len = l_len + r_len + 1;
+    char *buf = static_cast<char*>(std::malloc(buf_len));
+    concat(left, l_len, right.getOutOfLineData(), r_len, buf);
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, buf_len);
+  }
+  inline TypedValue apply(const TypedValue &left, const void *right) const {
+    const std::size_t l_len =
+        std::strlen(static_cast<const char*>(left.getOutOfLineData()));
+    const std::size_t r_len =
+        strnlen(static_cast<const char*>(right), right_max_);;
+    const std::size_t buf_len = l_len + r_len + 1;
+    char *buf = static_cast<char*>(std::malloc(buf_len));
+    concat(left.getOutOfLineData(), l_len, right, r_len, buf);
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, buf_len);
+  }
+  inline TypedValue apply(const TypedValue &left, const TypedValue &right) const {
+    const std::size_t l_len =
+        std::strlen(static_cast<const char*>(left.getOutOfLineData()));
+    const std::size_t r_len =
+        std::strlen(static_cast<const char*>(right.getOutOfLineData()));
+    const std::size_t buf_len = l_len + r_len + 1;
+    char *buf = static_cast<char*>(std::malloc(buf_len));
+    concat(left.getOutOfLineData(), l_len, right.getOutOfLineData(), r_len, buf);
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, buf_len);
+  }
+  inline static std::string GetName() {
+    return "+";
+  }
+  inline static const Type* GetResultType(const Type &left, const Type &right) {
+    DCHECK(left.getTypeID() == LeftT::kStaticTypeID);
+    DCHECK(right.getTypeID() == RightT::kStaticTypeID);
+    const std::size_t result_len =
+        static_cast<const LeftT&>(left).getStringLength() +
+            static_cast<const RightT&>(right).getStringLength();
+    const bool is_nullable = left.isNullable() || right.isNullable();
+    if (LeftT::kStaticTypeID == kChar && RightT::kStaticTypeID == kChar) {
+      return &TypeFactory::GetType(kChar, result_len, is_nullable);
+    } else {
+      return &TypeFactory::GetType(kVarChar, result_len, is_nullable);
+    }
+  }
+  const std::size_t left_max_;
+  const std::size_t right_max_;
+  const std::size_t result_max_;
+};
+
+
+using AsciiStringBinaryFunctorPack = FunctorPack<
+// concat
+    AsciiStringConcatFunctor<CharType, CharType, CharType>,
+    AsciiStringConcatFunctor<CharType, VarCharType, VarCharType>,
+    AsciiStringConcatFunctor<VarCharType, CharType, VarCharType>,
+    AsciiStringConcatFunctor<VarCharType, VarCharType, VarCharType>
+>;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ASCII_STRING_BINARY_OPERATIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/binary_operations/BinaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperation.cpp b/types/operations/binary_operations/BinaryOperation.cpp
index 436086f..4f97710 100644
--- a/types/operations/binary_operations/BinaryOperation.cpp
+++ b/types/operations/binary_operations/BinaryOperation.cpp
@@ -19,35 +19,6 @@
 
 #include "types/operations/binary_operations/BinaryOperation.hpp"
 
-#include "types/operations/Operation.pb.h"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
-#include "utility/Macros.hpp"
-
 namespace quickstep {
 
-serialization::BinaryOperation BinaryOperation::getProto() const {
-  serialization::BinaryOperation proto;
-  switch (operation_id_) {
-    case BinaryOperationID::kAdd:
-      proto.set_operation_id(serialization::BinaryOperation::ADD);
-      break;
-    case BinaryOperationID::kSubtract:
-      proto.set_operation_id(serialization::BinaryOperation::SUBTRACT);
-      break;
-    case BinaryOperationID::kMultiply:
-      proto.set_operation_id(serialization::BinaryOperation::MULTIPLY);
-      break;
-    case BinaryOperationID::kDivide:
-      proto.set_operation_id(serialization::BinaryOperation::DIVIDE);
-      break;
-    case BinaryOperationID::kModulo:
-      proto.set_operation_id(serialization::BinaryOperation::MODULO);
-      break;
-    default:
-      FATAL_ERROR("Unrecognized BinaryOperationID");
-  }
-
-  return proto;
-}
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/binary_operations/BinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperation.hpp b/types/operations/binary_operations/BinaryOperation.hpp
index 585a1c6..7fb9d49 100644
--- a/types/operations/binary_operations/BinaryOperation.hpp
+++ b/types/operations/binary_operations/BinaryOperation.hpp
@@ -30,8 +30,7 @@
 #include "storage/StorageBlockInfo.hpp"
 #include "types/TypedValue.hpp"
 #include "types/operations/Operation.hpp"
-#include "types/operations/Operation.pb.h"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -44,6 +43,9 @@ class ValueAccessor;
  *  @{
  */
 
+class BinaryOperation;
+typedef std::shared_ptr<const BinaryOperation> BinaryOperationPtr;
+
 /**
  * @brief A binary operator which can be quickly applied to data items WITHOUT
  *        checking their types.
@@ -67,16 +69,6 @@ class UncheckedBinaryOperator {
                                         const TypedValue &right) const = 0;
 
   /**
-   * @brief Apply to two data items via pointers without type-checking.
-   *
-   * @param left The left argument data item to apply to.
-   * @param right The right argument data item to apply to.
-   * @return The literal result of the operation.
-   **/
-  virtual TypedValue applyToDataPtrs(const void *left,
-                                     const void *right) const = 0;
-
-  /**
    * @brief Apply to two equally-sized vectors of values without type-checking.
    *
    * @param left The left argument ColumnVector to apply to.
@@ -182,47 +174,6 @@ class UncheckedBinaryOperator {
       const ColumnVector &right) const = 0;
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  virtual ColumnVector* applyToValueAccessorAndStaticValueForJoin(
-      ValueAccessor *left_accessor,
-      const bool left_accessor_is_left_relation,
-      const attribute_id left_id,
-      const TypedValue &right,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
-
-  virtual ColumnVector* applyToStaticValueAndValueAccessorForJoin(
-      const TypedValue &left,
-      ValueAccessor *right_accessor,
-      const bool right_accessor_is_left_relation,
-      const attribute_id right_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
-
-  virtual ColumnVector* applyToColumnVectorAndValueAccessorForJoin(
-      const ColumnVector &left,
-      ValueAccessor *right_accessor,
-      const bool right_accessor_is_left_relation,
-      const attribute_id right_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
-
-  virtual ColumnVector* applyToValueAccessorAndColumnVectorForJoin(
-      ValueAccessor *left_accessor,
-      const bool left_accessor_is_left_relation,
-      const attribute_id left_id,
-      const ColumnVector &right,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN_WITH_BINARY_EXPRESSIONS
-  virtual ColumnVector* applyToValueAccessorsForJoin(
-      ValueAccessor *left_accessor,
-      const bool left_accessor_is_left_relation,
-      const attribute_id left_id,
-      ValueAccessor *right_accessor,
-      const bool right_accessor_is_left_relation,
-      const attribute_id right_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN_WITH_BINARY_EXPRESSIONS
-
   /**
    * @brief Apply binary operator on column vector accumulating on the current
    * value. This is useful for summing/multiplying the values of the column
@@ -297,228 +248,39 @@ class UncheckedBinaryOperator {
  **/
 class BinaryOperation : public Operation {
  public:
-  /**
-   * @brief Generate a serialized Protocol Buffer representation of
-   *        this BinaryOperation.
-   *
-   * @return The serialized Protocol Buffer representation of
-   *         this BinaryOperation.
-   **/
-  serialization::BinaryOperation getProto() const;
-
-  /**
-   * @brief Determine the ID of this BinaryOperation
-   *
-   * @return The ID of this BinaryOperation
-   **/
-  inline BinaryOperationID getBinaryOperationID() const {
-    return operation_id_;
-  }
-
-  /**
-   * @brief Get a human-readable name for this BinaryOperation.
-   *
-   * @return A human-readable name for this BinaryOperation.
-   **/
   virtual std::string getName() const {
-    return kBinaryOperationNames[static_cast<std::size_t>(operation_id_)];
+    return "NoName";
   }
 
-  /**
-   * @brief Get a human-readable short name (e.g. "+", "-", "*", "/") for this
-   *        BinaryOperation.
-   *
-   * @return A human-readable short name for this BinaryOperation.
-   **/
   virtual std::string getShortName() const {
-    return kBinaryOperationShortNames[static_cast<std::size_t>(operation_id_)];
+    return getName();
   }
 
-  /**
-   * @brief Determine whether this BinaryOperation can apply to the specified
-   *        Types.
-   * @note When the Types that an operator can apply to are changed,
-   *       its definition in the function catalog of the query optimizer
-   *       needs to be modified accordingly.
-   *
-   * @param left The left argument Type to check.
-   * @param right The right argument Type to check.
-   * @return Whether this BinaryOperation can apply to left and right.
-   **/
-  virtual bool canApplyToTypes(const Type &left, const Type &right) const = 0;
-
-  /**
-   * @brief Determine the Type of the result from applying this BinaryOperation
-   *        to arguments of the specified Types.
-   *
-   * @param left The left argument Type to check.
-   * @param right The right argument Type to check.
-   * @return The Type of the result from applying this BinaryOperation to left
-   *         and right (NULL if not applicable).
-   **/
-  virtual const Type* resultTypeForArgumentTypes(const Type &left, const Type &right) const = 0;
-
-  /**
-   * @brief Similar to resultTypeForArgumentTypes(), but allows either or both
-   *        argument types to be NULL to indicate an unknown (but presumed
-   *        nullable) type. If the return type can be unambiguously determined
-   *        based on the incomplete information about the argument types, that
-   *        will be returned. Otherwise, NULL will be returned.
-   * @note This method returns NULL in cases where the return Type is
-   *       ambiguous, and also in cases where this BinaryOperation can not
-   *       possibly be applied to argument(s) of one of the known types left or
-   *       right. See also partialTypeSignatureIsPlausible(), which will return
-   *       true in the former case (ambiguous result type) but false in the
-   *       latter (can not apply to specfied argument types).
-   *
-   * @param left The left argument Type, or NULL if unknown (but presumed
-   *        nullable).
-   * @param right The right argument Type, or NULL if unknown (but presumed
-   *        nullable).
-   * @return The result Type for the given partial argument type information,
-   *         if such a result Type can be unambiguously determined. Otherwise
-   *         NULL.
-   **/
-  virtual const Type* resultTypeForPartialArgumentTypes(const Type *left,
-                                                        const Type *right) const = 0;
-
-  /**
-   * @brief Determine if an incomplete type signature is plausible for this
-   *        BinaryOperation. Specifically, checks if for each unknown Type in
-   *        the partial Type signature, there is some concrete Type that could
-   *        be substituted such that the overall signature becomes valid.
-   *
-   * @param result_type The result Type for this BinaryOperation, or NULL if
-   *        unknown.
-   * @param left_argument_type The Type for the left argument to this
-   *        BinaryOperation, or NULL if unknown (but presumed nullable).
-   * @param right_argument_type The Type for the right argument to this
-   *        BinaryOperation, or NULL if unknown (but presumed nullable).
-   * @return true if the specified partial Type signature is plausible.
-   **/
-  virtual bool partialTypeSignatureIsPlausible(const Type *result_type,
-                                               const Type *left_argument_type,
-                                               const Type *right_argument_type) const = 0;
-
-  // TODO(chasseur): This hinting interface is currently rather rudimentary,
-  // and we would like it to be more flexible to capture the full semantics of
-  // binary operations. Three issues immediately come to mind:
-  //
-  //     1. We should be able to use a category of Types or multiple Types as
-  //        a hint. For example, if we hint that the result of
-  //        DivideBinaryOperation should be DatetimeIntervalType, we know that
-  //        the hint for the left argument should be DatetimeIntervalType, but
-  //        the hint for the right argument can be any numeric type.
-  //     2. We should know something about whether or not a BinaryOperation is
-  //        commutative with respect to its argument types. For example,
-  //        AddBinaryOperation is commutative. If we hint that its result
-  //        should be DatetimeType, then one of its arguments should be
-  //        DatetimeType and the other should be one of the interval types, but
-  //        which is left and which is right does not matter (although it does
-  //        matter that there is one of each).
-  //     3. We may want to use "sideways" hinting, where we tentatively resolve
-  //        the types for arguments and note where we encounter "strong"
-  //        information about what a type should be (i.e. from the catalog or
-  //        an explicitly-typed literal in the parser), then using that to help
-  //        hint "weak" guesses about typing for the other argument.
-  //
-  // These issues can be difficult to solve in general, and it may not be worth
-  // doing in the current optimizer if another more sophisticated optimizer
-  // frontend is to be added in the future.
+  virtual bool canApplyTo(const Type &left,
+                          const Type &right,
+                          const std::vector<TypedValue> &static_arguments,
+                          std::string *message) const = 0;
 
-  /**
-   * @brief Get a pair of "hint" Types for the arguments to this
-   *        BinaryOperation based on a hint for this BinaryOperation's result
-   *        type. If possible, returns pointers to Types that, when given to
-   *        this BinaryOperation as arguments, yield values of the desired type
-   *        (i.e. calling resultTypeForArgumentTypes() on the returned types
-   *        should return the original type_hint).
-   *
-   * This method is designed to help the query optimizer's Resolver determine
-   * the type of literal values that appear in SQL queries. The Resolver
-   * propagates known Type information (e.g. from the Catalog) top-down through
-   * parsed expression trees, eventually using a Type hint at a leaf
-   * ParseLiteralValue node in the tree to concretize that literal value as the
-   * appropriate Type. This is especially important for NULL literals that
-   * appear in SQL queries, because a NULL by itself does not convey any type
-   * information and could be an instance of any nullable type. Hints generated
-   * by this method are to be treated as hints only, not as hard or
-   * authoritative requirements. After resolving literals, the Resolver
-   * propagates known concrete Types bottom-up through the tree and explicitly
-   * checks canApplyToTypes() for correctness.
-   *
-   * @note In some cases, either or both arguments could take on more than one
-   *       possible Type and this BinaryOperation would still yield the
-   *       specified result_type_hint (e.g. for arithmetic operations, if
-   *       result_type_hint is DoubleType, then the arguments could both be
-   *       DoubleType, or DoubleType and a different numeric Type, or even
-   *       LongType and FloatType). In such cases, the most precise suitable
-   *       Type is used as the hint.
-   * @note In some cases, the choice of argument Type hints for a given
-   *       result_type_hint is ambiguous and there is no single precise Type
-   *       that can represent values of any of the possible argument types
-   *       (e.g. an addition returning Datetime could be
-   *       Datetime + DatetimeInterval, DatetimeInterval + Datetime,
-   *       Datetime + YearMonthInterval, or YearMonthInterval + Datetime, and
-   *       none of these Types is coercible to the others or to a common more
-   *       precise type). In such cases, NULL is used to represent an ambiguous
-   *       or unknown hint.
-   *
-   * @param result_type_hint A hint about what Type the result of this
-   *        BinaryOperation should have. May be NULL to indicate no preference.
-   * @return A pair of type hints for the left and right arguments to this
-   *         BinaryOperation based on type_hint, or NULL if no suitable Type
-   *         exists or if the hint is ambiguous.
-   **/
-  virtual std::pair<const Type*, const Type*> pushDownTypeHint(
-      const Type *result_type_hint) const = 0;
+  bool canApplyTo(const Type &left,
+                  const Type &right,
+                  const std::vector<TypedValue> &static_arguments = {}) const {
+    std::string message;
+    return canApplyTo(left, right, static_arguments, &message);
+  }
 
-  /**
-   * @brief Apply this BinaryOperation to two TypedValues.
-   * @warning It is an error to call this method if this BinaryOperation can
-   *          not be applied to arguments of the specified types. If in doubt,
-   *          check canApplyToTypes() first.
-   *
-   * @param left The left argument TypedValue to apply to.
-   * @param left_type The Type that left belongs to.
-   * @param right The right argument TypedValue to apply to.
-   * @param right_type The Type that right belongs to.
-   * @return The literal result of the operation.
-   **/
-  virtual TypedValue applyToChecked(const TypedValue &left,
-                                    const Type &left_type,
-                                    const TypedValue &right,
-                                    const Type &right_type) const = 0;
+  virtual const Type* getResultType(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments = {}) const = 0;
 
-  /**
-   * @brief Create an UncheckedBinaryOperator which can apply to items of the
-   *        specified types.
-   * @warning The resulting UncheckedBinaryOperator performs no type-checking
-   *          whatsoever. Nonetheless, it is useful in situations where many
-   *          data items of the same, known types are to be operated on (for
-   *          example, over many tuples in the same table).
-   *
-   * @param left The left argument Type to compare.
-   * @param right The right argument Type to compare.
-   * @return An UncheckedBinaryOperator which applies this BinaryOperation to
-   *         the specified Types.
-   * @exception OperationInapplicableToType This BinaryOperation is not
-   *            applicable to either left or right.
-   **/
-  virtual UncheckedBinaryOperator* makeUncheckedBinaryOperatorForTypes(const Type &left,
-                                                                       const Type &right) const = 0;
+  virtual UncheckedBinaryOperator* makeUncheckedBinaryOperator(
+      const Type &left,
+      const Type &right,
+      const std::vector<TypedValue> &static_arguments = {}) const = 0;
 
  protected:
-  explicit BinaryOperation(const BinaryOperationID operation_id)
-      : Operation(Operation::kBinaryOperation,
-                  kBinaryOperationNames[
-                      static_cast<typename std::underlying_type<BinaryOperationID>::type>(operation_id)],
-                  kBinaryOperationShortNames[
-                      static_cast<typename std::underlying_type<BinaryOperationID>::type>(operation_id)]),
-        operation_id_(operation_id) {
-  }
-
-  const BinaryOperationID operation_id_;
+  BinaryOperation()
+      : Operation(Operation::kBinaryOperation) {}
 
  private:
   DISALLOW_COPY_AND_ASSIGN(BinaryOperation);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/binary_operations/BinaryOperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationFactory.cpp b/types/operations/binary_operations/BinaryOperationFactory.cpp
deleted file mode 100644
index 51b5a7e..0000000
--- a/types/operations/binary_operations/BinaryOperationFactory.cpp
+++ /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.
- **/
-
-#include "types/operations/binary_operations/BinaryOperationFactory.hpp"
-
-#include <string>
-
-#include "types/operations/Operation.pb.h"
-#include "types/operations/binary_operations/AddBinaryOperation.hpp"
-#include "types/operations/binary_operations/BinaryOperationID.hpp"
-#include "types/operations/binary_operations/DivideBinaryOperation.hpp"
-#include "types/operations/binary_operations/ModuloBinaryOperation.hpp"
-#include "types/operations/binary_operations/MultiplyBinaryOperation.hpp"
-#include "types/operations/binary_operations/SubtractBinaryOperation.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-const BinaryOperation& BinaryOperationFactory::GetBinaryOperation(const BinaryOperationID id) {
-  switch (id) {
-    case BinaryOperationID::kAdd:
-      return AddBinaryOperation::Instance();
-    case BinaryOperationID::kSubtract:
-      return SubtractBinaryOperation::Instance();
-    case BinaryOperationID::kMultiply:
-      return MultiplyBinaryOperation::Instance();
-    case BinaryOperationID::kDivide:
-      return DivideBinaryOperation::Instance();
-    case BinaryOperationID::kModulo:
-      return ModuloBinaryOperation::Instance();
-    default:
-      break;  // Prevent compiler from complaining about unhandled case.
-  }
-  // Should never be reached
-  FATAL_ERROR("Unrecognized BinaryOperationID");
-}
-
-bool BinaryOperationFactory::ProtoIsValid(const serialization::BinaryOperation &proto) {
-  // Check that BinaryOperation is fully initialized.
-  if (!proto.IsInitialized()) {
-    return false;
-  }
-
-  // Check that the operation_id is a valid BinaryOperation.
-  if (!proto.BinaryOperationID_IsValid(proto.operation_id())) {
-    return false;
-  }
-
-  return true;
-}
-
-const BinaryOperation& BinaryOperationFactory::ReconstructFromProto(
-    const serialization::BinaryOperation &proto) {
-  DCHECK(ProtoIsValid(proto))
-      << "Attempted to create BinaryOperation from an invalid proto description:\n"
-      << proto.DebugString();
-
-  switch (proto.operation_id()) {
-    case serialization::BinaryOperation::ADD:
-      return GetBinaryOperation(BinaryOperationID::kAdd);
-    case serialization::BinaryOperation::SUBTRACT:
-      return GetBinaryOperation(BinaryOperationID::kSubtract);
-    case serialization::BinaryOperation::MULTIPLY:
-      return GetBinaryOperation(BinaryOperationID::kMultiply);
-    case serialization::BinaryOperation::DIVIDE:
-      return GetBinaryOperation(BinaryOperationID::kDivide);
-    case serialization::BinaryOperation::MODULO:
-      return GetBinaryOperation(BinaryOperationID::kModulo);
-    default:
-      FATAL_ERROR("Unrecognized BinaryOperationID in "
-                  "BinaryOperationFactory::ReconstructFromProto");
-  }
-}
-
-}  // namespace quickstep

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/binary_operations/BinaryOperationID.cpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationID.cpp b/types/operations/binary_operations/BinaryOperationID.cpp
deleted file mode 100644
index 7ba2e69..0000000
--- a/types/operations/binary_operations/BinaryOperationID.cpp
+++ /dev/null
@@ -1,40 +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/BinaryOperationID.hpp"
-
-namespace quickstep {
-
-const char *kBinaryOperationNames[] = {
-  "Add",
-  "Subtract",
-  "Multiply",
-  "Divide",
-  "Modulo"
-};
-
-const char *kBinaryOperationShortNames[] = {
-  "+",
-  "-",
-  "*",
-  "/",
-  "%"
-};
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/da9baf7e/types/operations/binary_operations/BinaryOperationID.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperationID.hpp b/types/operations/binary_operations/BinaryOperationID.hpp
deleted file mode 100644
index 2cf20d1..0000000
--- a/types/operations/binary_operations/BinaryOperationID.hpp
+++ /dev/null
@@ -1,62 +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_BINARY_OPERATION_ID_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_ID_HPP_
-
-#include <type_traits>
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief Concrete BinaryOperations.
- **/
-enum class BinaryOperationID {
-  kAdd = 0,
-  kSubtract,
-  kMultiply,
-  kDivide,
-  kModulo,
-  kNumBinaryOperationIDs  // Not a real BinaryOperationID, exists for counting purposes.
-};
-
-/**
- * @brief Names of comparisons in the same order as BinaryOperationID.
- **/
-extern const char *kBinaryOperationNames[
-    static_cast<typename std::underlying_type<BinaryOperationID>::type>(
-        BinaryOperationID::kNumBinaryOperationIDs)];
-
-/**
- * @brief Short names (i.e. mathematical symbols) of comparisons in the same
- *        order as BinaryOperationID.
- **/
-extern const char *kBinaryOperationShortNames[
-    static_cast<typename std::underlying_type<BinaryOperationID>::type>(
-        BinaryOperationID::kNumBinaryOperationIDs)];
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_BINARY_OPERATIONS_BINARY_OPERATION_ID_HPP_