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/03/09 21:29:11 UTC

[1/8] incubator-quickstep git commit: Initial commit

Repository: incubator-quickstep
Updated Branches:
  refs/heads/new-op [created] 27c416253


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/UnaryOperationWrapper.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationWrapper.hpp b/types/operations/unary_operations/UnaryOperationWrapper.hpp
new file mode 100644
index 0000000..417c752
--- /dev/null
+++ b/types/operations/unary_operations/UnaryOperationWrapper.hpp
@@ -0,0 +1,441 @@
+/**
+ * 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_UNARY_OPERATION_WRAPPER_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_WRAPPER_HPP_
+
+#include <cstddef>
+#include <string>
+#include <type_traits>
+
+#include "catalog/CatalogTypedefs.hpp"
+#include "storage/ValueAccessor.hpp"
+#include "storage/ValueAccessorUtil.hpp"
+#include "types/CharType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/VarCharType.hpp"
+#include "types/containers/ColumnVector.hpp"
+#include "types/operations/Operation.hpp"
+#include "types/operations/OperationSignature.hpp"
+#include "types/operations/Operation.pb.h"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+
+class ColumnVector;
+class Type;
+class ValueAccessor;
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename ArgumentT, typename ResultT>
+struct UnaryFunctor {
+  typedef ArgumentT ArgumentType;
+  typedef ResultT ResultType;
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+class UncheckedUnaryOperatorWrapperCodegen : public UncheckedUnaryOperator {
+ public:
+  template <typename ...ConstructorArgs>
+  UncheckedUnaryOperatorWrapperCodegen(const Type &argument_type,
+                                       const Type &result_type,
+                                       ConstructorArgs &&...args)
+      : functor_(std::forward<ConstructorArgs>(args)...),
+        impl_(argument_type, result_type) {}
+
+  TypedValue applyToTypedValue(const TypedValue &argument) const override {
+    return impl_.applyToTypedValue(argument, functor_);
+  }
+
+  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override {
+    return impl_.applyToColumnVector(argument, functor_);
+  }
+
+  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
+                                     const attribute_id argument_attr_id) const override {
+    return impl_.applyToValueAccessor(accessor, argument_attr_id, functor_);
+  }
+
+ private:
+  using ArgumentType = typename FunctorT::ArgumentType;
+  using ResultType = typename FunctorT::ResultType;
+
+  template <bool specialize, typename EnableT = void>
+  struct FunctorSpecializer;
+
+  template <typename T, typename EnableT = void>
+  struct Codegen;
+
+  template <bool argument_nullable,
+            typename ArgumentGen, typename ResultGen>
+  struct Implementation;
+
+  using FuncSpec = FunctorSpecializer<sizeof...(SpecArgs) != 0>;
+  using Impl = Implementation<false,
+                              Codegen<ArgumentType>,
+                              Codegen<ResultType>>;
+
+  const FunctorT functor_;
+  const Impl impl_;
+};
+
+template <typename FunctorT>
+class UnaryOperationWrapper : public UnaryOperation {
+ public:
+  UnaryOperationWrapper()
+      : UnaryOperation(),
+        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(), {ArgumentType::kStaticTypeID}, 0)
+    };
+  }
+
+  bool canApplyTo(const Type &argument_type,
+                  const std::vector<TypedValue> &static_arguments,
+                  std::string *message) const override {
+    DCHECK(argument_type.getTypeID() == ArgumentType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return true;
+  }
+
+  const Type* getResultType(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(argument_type.getTypeID() == ArgumentType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return &TypeFactory::GetType(ResultType::kStaticTypeID);
+  }
+
+  UncheckedUnaryOperator* makeUncheckedUnaryOperator(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(argument_type.getTypeID() == ArgumentType::kStaticTypeID);
+    DCHECK(static_arguments.empty());
+    return new UncheckedUnaryOperatorWrapperCodegen<FunctorT>(
+        argument_type, *getResultType(argument_type, static_arguments));
+  }
+
+ private:
+  using ArgumentType = typename FunctorT::ArgumentType;
+  using ResultType = typename FunctorT::ResultType;
+
+  // TODO
+  static constexpr bool kCanAutoDerive = !ResultType::kParameterized;
+
+  const std::string operation_name_;
+};
+
+
+template <typename FunctorT, typename ...SpecArgs>
+template <bool specialize>
+struct UncheckedUnaryOperatorWrapperCodegen<FunctorT, SpecArgs...>
+    ::FunctorSpecializer<specialize, std::enable_if_t<specialize>> {
+  template <typename ...FuncArgs>
+  inline static auto Invoke(const FunctorT &functor, FuncArgs &&...args) {
+    return functor.template apply<SpecArgs...>(std::forward<FuncArgs>(args)...);
+  }
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <bool specialize>
+struct UncheckedUnaryOperatorWrapperCodegen<FunctorT, SpecArgs...>
+    ::FunctorSpecializer<specialize, std::enable_if_t<!specialize>> {
+  template <typename ...FuncArgs>
+  inline static auto Invoke(const FunctorT &functor, FuncArgs &&...args) {
+    return functor.apply(std::forward<FuncArgs>(args)...);
+  }
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <typename T>
+struct UncheckedUnaryOperatorWrapperCodegen<FunctorT, SpecArgs...>
+    ::Codegen<T, std::enable_if_t<T::kLayout == kNativeEmbedded>> {
+  using NativeType = typename T::cpptype;
+  using NativeTypeConstRef = const NativeType&;
+  using NativeTypeConstPtr = const NativeType*;
+  using ColumnVectorType = NativeColumnVector;
+
+  template <typename ArgumentGen>
+  inline static TypedValue ApplyGenericTypedValue(
+      typename ArgumentGen::NativeTypeConstRef argument,
+      const Type &result_type,
+      const FunctorT &functor) {
+    return TypedValue(FuncSpec::Invoke(functor, argument));
+  }
+
+  template <typename ArgumentGen>
+  inline static void ApplyGenericColumnVector(
+      const typename ArgumentGen::NativeTypeConstPtr &argument,
+      const FunctorT &functor,
+      ColumnVectorType *cv) {
+    *static_cast<NativeType *>(cv->getPtrForDirectWrite()) =
+        FuncSpec::Invoke(functor, ArgumentGen::Dereference(argument));
+  }
+
+  template <bool nullable>
+  inline static NativeTypeConstPtr GetValuePtrColumnVector(
+      const ColumnVectorType &cv,
+      const std::size_t pos) {
+    return static_cast<NativeTypeConstPtr>(
+        cv.template getUntypedValue<nullable>(pos));
+  }
+
+  template <bool nullable, typename ValueAccessorT>
+  inline static NativeTypeConstPtr GetValuePtrValueAccessor(
+      ValueAccessorT *va,
+      const attribute_id argument_attr_id) {
+    return static_cast<NativeTypeConstPtr>(
+        va->template getUntypedValue<nullable>(argument_attr_id));
+  }
+
+  // Dereference: NativeTypeConstPtr& -> bool
+  inline static bool IsNull(const NativeType *value) {
+    return value == nullptr;
+  }
+
+  // Dereference: NativeTypeConstPtr& -> const NativeType&
+  inline static const NativeType& Dereference(const NativeType *value) {
+    return *value;
+  }
+
+  inline static const NativeType ToNativeValue(const TypedValue &value) {
+    return value.getLiteral<NativeType>();
+  }
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <typename T>
+struct UncheckedUnaryOperatorWrapperCodegen<FunctorT, SpecArgs...>
+    ::Codegen<T, std::enable_if_t<T::kLayout == kNonNativeInline>> {
+  using NativeType = void*;
+  using NativeTypeConstRef = const void*;
+  using NativeTypeConstPtr = const void*;
+  using ColumnVectorType = NativeColumnVector;
+
+  template <typename ArgumentGen>
+  inline static TypedValue ApplyGenericTypedValue(
+      typename ArgumentGen::NativeTypeConstRef argument,
+      const Type &result_type,
+      const FunctorT &functor) {
+    void *result = std::malloc(result_type.maximumByteLength());
+    FuncSpec::Invoke(functor, argument, result);
+    return TypedValue::CreateWithOwnedData(T::kStaticTypeID,
+                                           result,
+                                           result_type.maximumByteLength());
+  }
+
+  template <typename ArgumentGen>
+  inline static void ApplyGenericColumnVector(
+      const typename ArgumentGen::NativeTypeConstPtr &argument,
+      const FunctorT &functor,
+      ColumnVectorType *cv) {
+    FuncSpec::Invoke(functor,
+                     ArgumentGen::Dereference(argument),
+                     cv->getPtrForDirectWrite());
+  }
+
+  template <bool nullable>
+  inline static NativeTypeConstPtr GetValuePtrColumnVector(
+      const ColumnVectorType &cv,
+      const std::size_t pos) {
+    return cv.template getUntypedValue<nullable>(pos);
+  }
+
+  template <bool nullable, typename ValueAccessorT>
+  inline static NativeTypeConstPtr GetValuePtrValueAccessor(
+      ValueAccessorT *va,
+      const attribute_id argument_attr_id) {
+    return va->template getUntypedValue<nullable>(argument_attr_id);
+  }
+
+  inline static bool IsNull(const void *value) {
+    return value == nullptr;
+  }
+
+  // Dereference: NativeTypeConstPtr& -> const NativeType&
+  inline static const void* Dereference(const void *value) {
+    return value;
+  }
+
+  inline static const void* ToNativeValue(const TypedValue &value) {
+    return value.getDataPtr();
+  }
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <typename T>
+struct UncheckedUnaryOperatorWrapperCodegen<FunctorT, SpecArgs...>
+    ::Codegen<T, std::enable_if_t<T::kLayout == kOutOfLine>> {
+  using NativeType = TypedValue;
+  using NativeTypeConstRef = const TypedValue&;
+  using NativeTypeConstPtr = const TypedValue;
+  using ColumnVectorType = IndirectColumnVector;
+
+  template <typename ArgumentGen>
+  inline static TypedValue ApplyGenericTypedValue(
+      typename ArgumentGen::NativeTypeConstRef argument,
+      const Type &result_type,
+      const FunctorT &functor) {
+    return FuncSpec::Invoke(functor, argument);
+  }
+
+  template <typename ArgumentGen>
+  inline static void ApplyGenericColumnVector(
+      const typename ArgumentGen::NativeTypeConstPtr &argument,
+      const FunctorT &functor,
+      ColumnVectorType *cv) {
+    cv->appendTypedValue(
+        FuncSpec::Invoke(functor, ArgumentGen::Dereference(argument)));
+  }
+
+  template <bool nullable>
+  inline static NativeTypeConstPtr GetValuePtrColumnVector(
+      const ColumnVectorType &cv,
+      const std::size_t pos) {
+    return cv.getTypedValue(pos);
+  }
+
+  template <bool nullable, typename ValueAccessorT>
+  inline static NativeTypeConstPtr GetValuePtrValueAccessor(
+      ValueAccessorT *va,
+      const attribute_id argument_attr_id) {
+    return va->getTypedValue(argument_attr_id);
+  }
+
+  inline static bool IsNull(NativeTypeConstPtr &value) {
+    return value.isNull();
+  }
+
+  // Dereference: NativeTypeConstPtr& -> const NativeType&
+  inline static const NativeType& Dereference(NativeTypeConstPtr &value) {
+    return value;
+  }
+
+  inline static const NativeType& ToNativeValue(const TypedValue &value) {
+    return value;
+  }
+};
+
+template <typename FunctorT, typename ...SpecArgs>
+template <bool argument_nullable,
+          typename ArgumentGen, typename ResultGen>
+struct UncheckedUnaryOperatorWrapperCodegen<FunctorT, SpecArgs...>
+    ::Implementation {
+  Implementation(const Type &argument_type_in, const Type &result_type_in)
+      : argument_type(argument_type_in),
+        result_type(result_type_in) {}
+
+  inline TypedValue applyToTypedValue(const TypedValue &argument,
+                                      const FunctorT &functor) const {
+    if (argument_nullable && argument.isNull()) {
+      return TypedValue(result_type.getTypeID());
+    }
+
+    return ResultGen::template ApplyGenericTypedValue<ArgumentGen>(
+        ArgumentGen::ToNativeValue(argument),
+        result_type,
+        functor);
+  }
+
+  inline ColumnVector* applyToColumnVector(const ColumnVector &argument,
+                                           const FunctorT &functor) const {
+    using ArgumentCVT = typename ArgumentGen::ColumnVectorType;
+    using ResultCVT = typename ResultGen::ColumnVectorType;
+
+    const ArgumentCVT &argument_cv = static_cast<const ArgumentCVT&>(argument);
+    ResultCVT *result_cv = new ResultCVT(result_type, argument_cv.size());
+
+    for (std::size_t pos = 0; pos < argument_cv.size(); ++pos) {
+      typename ArgumentGen::NativeTypeConstPtr scalar_arg =
+          ArgumentGen::template GetValuePtrColumnVector<argument_nullable>(
+              argument_cv, pos);
+
+      if (argument_nullable && ArgumentGen::IsNull(scalar_arg)) {
+        result_cv->appendNullValue();
+      } else {
+        ResultGen::template ApplyGenericColumnVector<ArgumentGen>(
+            scalar_arg, functor, result_cv);
+      }
+    }
+    return result_cv;
+  }
+
+  inline ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
+                                            const attribute_id argument_attr_id,
+                                            const FunctorT &functor) const {
+    using ResultCVT = typename ResultGen::ColumnVectorType;
+
+    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
+        accessor,
+        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
+      ResultCVT *result_cv = new ResultCVT(result_type, accessor->getNumTuples());
+
+      accessor->beginIteration();
+      while (accessor->next()) {
+        typename ArgumentGen::NativeTypeConstPtr scalar_arg =
+            ArgumentGen::template GetValuePtrValueAccessor<argument_nullable>(
+                accessor, argument_attr_id);
+
+        if (argument_nullable && ArgumentGen::IsNull(scalar_arg)) {
+          result_cv->appendNullValue();
+        } else {
+          ResultGen::template ApplyGenericColumnVector<ArgumentGen>(
+              scalar_arg, functor, result_cv);
+        }
+      }
+      return result_cv;
+    });
+  }
+
+  const Type &argument_type;
+  const Type &result_type;
+};
+
+template <typename ...FunctorTypes>
+struct UnaryFunctorPack {
+  static std::vector<UnaryOperationPtr> GenerateAll() {
+    return {
+        UnaryOperationPtr(new UnaryOperationWrapper<FunctorTypes>())...
+    };
+  }
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_WRAPPER_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/utility/StringUtil.cpp
----------------------------------------------------------------------
diff --git a/utility/StringUtil.cpp b/utility/StringUtil.cpp
index 2745457..9fca695 100644
--- a/utility/StringUtil.cpp
+++ b/utility/StringUtil.cpp
@@ -52,6 +52,12 @@ std::string ToLower(const std::string& str) {
   return lower_str;
 }
 
+std::string ToUpper(const std::string& str) {
+  std::string upper_str(str.size(), ' ');
+  std::transform(str.begin(), str.end(), upper_str.begin(), toupper);
+  return upper_str;
+}
+
 std::string EscapeSpecialChars(const std::string& text) {
   std::string new_text;
   for (const char& c : text) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/utility/StringUtil.hpp
----------------------------------------------------------------------
diff --git a/utility/StringUtil.hpp b/utility/StringUtil.hpp
index abda8f3..e928225 100644
--- a/utility/StringUtil.hpp
+++ b/utility/StringUtil.hpp
@@ -35,11 +35,21 @@ namespace quickstep {
  * @brief Convert a string \p str to lower case.
  *
  * @param str The string to be converted.
- * @return The converted string with all lower case characters bing converted to upper case characters.
+ * @return The converted string with all lower case characters being converted
+ *         to upper case characters.
  */
 extern std::string ToLower(const std::string &str);
 
 /**
+ * @brief Convert a string \p str to upper case.
+ *
+ * @param str The string to be converted.
+ * @return The converted string with all upper case characters being converted
+ *         to lower case characters.
+ */
+extern std::string ToUpper(const std::string &str);
+
+/**
  * @brief Converts special characters to escape characters.
  *
  * @param text The string to be unescaped.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/utility/TemplateUtil.hpp
----------------------------------------------------------------------
diff --git a/utility/TemplateUtil.hpp b/utility/TemplateUtil.hpp
index dfae8e4..f6baec2 100644
--- a/utility/TemplateUtil.hpp
+++ b/utility/TemplateUtil.hpp
@@ -30,8 +30,6 @@ namespace quickstep {
  *  @{
  */
 
-namespace template_util_inner {
-
 /**
  * @brief Represents a compile-time sequence of integers.
  *
@@ -60,6 +58,20 @@ struct MakeSequence<0, S...> {
   typedef Sequence<S...> type;
 };
 
+template<class...> struct Disjunction : std::false_type {};
+template<class B1> struct Disjunction<B1> : B1 {};
+template<class B1, class... Bn>
+struct Disjunction<B1, Bn...>
+    : std::conditional_t<bool(B1::value), B1, Disjunction<Bn...>>  {};
+
+template <typename check, typename ...cases>
+struct EqualsAny {
+  static constexpr bool value =
+      Disjunction<std::is_same<check, cases>...>::value;
+};
+
+namespace template_util_inner {
+
 /**
  * @brief Final step of CreateBoolInstantiatedInstance. Now all bool_values are
  *        ready. Instantiate the template and create (i.e. new) an instance.
@@ -125,7 +137,7 @@ inline ReturnT* CreateBoolInstantiatedInstance(Tuple &&args) {
   return template_util_inner::CreateBoolInstantiatedInstanceInner<
       T, ReturnT, bool_values...>(
           std::forward<Tuple>(args),
-          typename template_util_inner::MakeSequence<n_args>::type());
+          typename MakeSequence<n_args>::type());
 }
 
 /**
@@ -225,9 +237,17 @@ inline auto InvokeOnBools(ArgTypes ...args) {
   constexpr std::size_t last = sizeof...(args) - 1;
   return template_util_inner::InvokeOnBoolsInner<last>(
       std::forward_as_tuple(args...),
-      typename template_util_inner::MakeSequence<last>::type());
+      typename MakeSequence<last>::type());
 }
 
+
+template <char ...c>
+struct StringLiteral {
+  inline static std::string ToString() {
+    return std::string({c...});
+  }
+};
+
 /** @} */
 
 }  // namespace quickstep


[8/8] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
Initial commit


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

Branch: refs/heads/new-op
Commit: 27c4162537df7829a7e5c2374431ea713dc36d34
Parents: 132fed6
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Sat Mar 4 12:11:13 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Mar 9 15:28:58 2017 -0600

----------------------------------------------------------------------
 expressions/CMakeLists.txt                      |    1 +
 expressions/ExpressionFactories.cpp             |   28 +-
 expressions/Expressions.proto                   |    5 +-
 expressions/scalar/CMakeLists.txt               |    3 +-
 expressions/scalar/ScalarUnaryExpression.cpp    |   82 +-
 expressions/scalar/ScalarUnaryExpression.hpp    |   26 +-
 parser/CMakeLists.txt                           |    3 -
 parser/ParseBasicExpressions.cpp                |   76 -
 parser/ParseBasicExpressions.hpp                |  205 --
 parser/ParseExpression.hpp                      |    3 -
 parser/SqlLexer.lpp                             |    1 +
 parser/SqlParser.ypp                            |   59 +-
 parser/preprocessed/SqlLexer_gen.cpp            | 1457 ++++----
 parser/preprocessed/SqlLexer_gen.hpp            |  192 +-
 parser/preprocessed/SqlParser_gen.cpp           | 3290 +++++++++---------
 parser/preprocessed/SqlParser_gen.hpp           |  201 +-
 query_optimizer/LogicalGenerator.cpp            |    3 +-
 query_optimizer/expressions/CMakeLists.txt      |    5 +-
 query_optimizer/expressions/Cast.cpp            |   17 +-
 query_optimizer/expressions/UnaryExpression.cpp |   32 +-
 query_optimizer/expressions/UnaryExpression.hpp |   44 +-
 query_optimizer/resolver/CMakeLists.txt         |    5 +-
 query_optimizer/resolver/Resolver.cpp           |  304 +-
 query_optimizer/resolver/Resolver.hpp           |    9 +-
 types/CMakeLists.txt                            |   24 +
 types/CharType.cpp                              |    3 +-
 types/CharType.hpp                              |    4 +-
 types/DateType.hpp                              |   10 +-
 types/DatetimeIntervalType.hpp                  |   11 +-
 types/DatetimeLit.hpp                           |    4 +
 types/DatetimeType.hpp                          |   10 +-
 types/DoubleType.cpp                            |    2 -
 types/DoubleType.hpp                            |    6 +-
 types/FloatType.cpp                             |    2 -
 types/FloatType.hpp                             |    6 +-
 types/IntType.cpp                               |    2 -
 types/IntType.hpp                               |    6 +-
 types/LongType.cpp                              |    2 -
 types/LongType.hpp                              |    6 +-
 types/NullCoercibilityCheckMacro.hpp            |    2 +-
 types/NullType.hpp                              |    8 +-
 types/NumericSuperType.hpp                      |   13 +-
 types/Type.cpp                                  |   41 +-
 types/Type.hpp                                  |   59 +-
 types/Type.proto                                |   18 +-
 types/TypeConcept.hpp                           |   33 +
 types/TypeFactory.cpp                           |   50 +-
 types/TypeID.hpp                                |   33 +-
 types/TypeUtil.hpp                              |  181 +
 types/TypedValue.cpp                            |   37 +-
 types/TypedValue.hpp                            |    9 +-
 types/TypedValue.proto                          |    2 +-
 types/VarCharType.cpp                           |    3 +-
 types/VarCharType.hpp                           |    4 +-
 types/YearMonthIntervalType.hpp                 |   11 +-
 types/containers/ColumnVector.hpp               |   12 +-
 types/operations/CMakeLists.txt                 |    9 +
 types/operations/Operation.hpp                  |   18 +-
 types/operations/Operation.proto                |   47 +-
 types/operations/OperationSignature.cpp         |   91 +
 types/operations/OperationSignature.hpp         |  192 +
 .../ArithmeticBinaryOperators.hpp               |   13 +-
 .../binary_operations/BinaryOperation.hpp       |    6 +-
 types/operations/comparisons/CMakeLists.txt     |    2 +
 types/operations/comparisons/Comparison.hpp     |    6 +-
 types/operations/comparisons/ComparisonUtil.hpp |   30 +-
 .../ArithmeticUnaryOperations.cpp               |  145 -
 .../ArithmeticUnaryOperations.hpp               |   80 +-
 .../ArithmeticUnaryOperators.hpp                |  169 -
 .../operations/unary_operations/CMakeLists.txt  |  109 +-
 .../unary_operations/CMathUnaryOperations.hpp   |  108 +
 .../unary_operations/CastOperation.cpp          |  282 ++
 .../unary_operations/CastOperation.hpp          |  149 +
 .../unary_operations/DateExtractOperation.cpp   |  597 +---
 .../unary_operations/DateExtractOperation.hpp   |  169 +-
 .../unary_operations/NumericCastOperation.hpp   |  313 --
 .../unary_operations/SubstringOperation.cpp     |   46 +-
 .../unary_operations/SubstringOperation.hpp     |  187 +-
 .../unary_operations/UnaryOperation.cpp         |   20 -
 .../unary_operations/UnaryOperation.hpp         |  181 +-
 .../unary_operations/UnaryOperationFactory.cpp  |  112 +-
 .../unary_operations/UnaryOperationFactory.hpp  |   82 +-
 .../unary_operations/UnaryOperationID.cpp       |   32 -
 .../unary_operations/UnaryOperationID.hpp       |   63 -
 .../unary_operations/UnaryOperationWrapper.hpp  |  441 +++
 utility/StringUtil.cpp                          |    6 +
 utility/StringUtil.hpp                          |   12 +-
 utility/TemplateUtil.hpp                        |   28 +-
 88 files changed, 5161 insertions(+), 5249 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/CMakeLists.txt b/expressions/CMakeLists.txt
index b1f1fb1..3becc71 100644
--- a/expressions/CMakeLists.txt
+++ b/expressions/CMakeLists.txt
@@ -51,6 +51,7 @@ target_link_libraries(quickstep_expressions_ExpressionFactories
                       quickstep_expressions_scalar_ScalarUnaryExpression
                       quickstep_types_TypeFactory
                       quickstep_types_TypedValue
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_binaryoperations_BinaryOperationFactory
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_unaryoperations_UnaryOperationFactory

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/expressions/ExpressionFactories.cpp
----------------------------------------------------------------------
diff --git a/expressions/ExpressionFactories.cpp b/expressions/ExpressionFactories.cpp
index 01d22a0..dd7532e 100644
--- a/expressions/ExpressionFactories.cpp
+++ b/expressions/ExpressionFactories.cpp
@@ -42,6 +42,7 @@
 #include "expressions/scalar/ScalarUnaryExpression.hpp"
 #include "types/TypeFactory.hpp"
 #include "types/TypedValue.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "types/operations/binary_operations/BinaryOperationFactory.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/unary_operations/UnaryOperationFactory.hpp"
@@ -167,10 +168,24 @@ Scalar* ScalarFactory::ReconstructFromProto(const serialization::Scalar &proto,
           proto.GetExtension(serialization::ScalarAttribute::attribute_id)));
     }
     case serialization::Scalar::UNARY_EXPRESSION: {
+      std::vector<TypedValue> static_arguments;
+      const int num_static_args =
+          proto.ExtensionSize(serialization::ScalarUnaryExpression::static_arguments);
+      for (int i = 0; i < num_static_args; ++i) {
+        static_arguments.emplace_back(
+            TypedValue::ReconstructFromProto(
+                proto.GetExtension(serialization::ScalarUnaryExpression::static_arguments, i)));
+      }
+
+      const OperationSignaturePtr op_signature =
+          OperationSignature::ReconstructFromProto(
+              proto.GetExtension(serialization::ScalarUnaryExpression::op_signature));
+
       return new ScalarUnaryExpression(
-          UnaryOperationFactory::ReconstructFromProto(
-              proto.GetExtension(serialization::ScalarUnaryExpression::operation)),
-          ReconstructFromProto(proto.GetExtension(serialization::ScalarUnaryExpression::operand), database));
+          op_signature,
+          UnaryOperationFactory::Instance().getUnaryOperation(op_signature),
+          ReconstructFromProto(proto.GetExtension(serialization::ScalarUnaryExpression::operand), database),
+          std::make_shared<std::vector<TypedValue>>(std::move(static_arguments)));
     }
     case serialization::Scalar::BINARY_EXPRESSION: {
       return new ScalarBinaryExpression(
@@ -242,11 +257,8 @@ bool ScalarFactory::ProtoIsValid(const serialization::Scalar &proto,
       break;
     }
     case serialization::Scalar::UNARY_EXPRESSION: {
-      if (proto.HasExtension(serialization::ScalarUnaryExpression::operation)
-          && proto.HasExtension(serialization::ScalarUnaryExpression::operand)) {
-        return UnaryOperationFactory::ProtoIsValid(proto.GetExtension(serialization::ScalarUnaryExpression::operation))
-               && ProtoIsValid(proto.GetExtension(serialization::ScalarUnaryExpression::operand), database);
-      }
+      // TODO
+      return true;
       break;
     }
     case serialization::Scalar::BINARY_EXPRESSION: {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/expressions/Expressions.proto
----------------------------------------------------------------------
diff --git a/expressions/Expressions.proto b/expressions/Expressions.proto
index 8d923c5..ea6228a 100644
--- a/expressions/Expressions.proto
+++ b/expressions/Expressions.proto
@@ -102,8 +102,9 @@ message ScalarAttribute {
 
 message ScalarUnaryExpression {
   extend Scalar {
-    optional UnaryOperation operation = 96;
-    optional Scalar operand = 97;
+    optional OperationSignature op_signature = 97;
+    optional Scalar operand = 98;
+    repeated TypedValue static_arguments = 99;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/expressions/scalar/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/expressions/scalar/CMakeLists.txt b/expressions/scalar/CMakeLists.txt
index 8f509da..79d1a96 100644
--- a/expressions/scalar/CMakeLists.txt
+++ b/expressions/scalar/CMakeLists.txt
@@ -102,6 +102,7 @@ target_link_libraries(quickstep_expressions_scalar_ScalarUnaryExpression
                       quickstep_types_TypeErrors
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_utility_Macros)
@@ -173,7 +174,7 @@ target_link_libraries(Scalar_unittest
                       quickstep_types_operations_binaryoperations_BinaryOperation
                       quickstep_types_operations_binaryoperations_BinaryOperationFactory
                       quickstep_types_operations_binaryoperations_BinaryOperationID
-                      quickstep_types_operations_unaryoperations_NumericCastOperation
+                      quickstep_types_operations_unaryoperations_CastOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_types_operations_unaryoperations_UnaryOperationFactory
                       quickstep_types_operations_unaryoperations_UnaryOperationID

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/expressions/scalar/ScalarUnaryExpression.cpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarUnaryExpression.cpp b/expressions/scalar/ScalarUnaryExpression.cpp
index 72fdbe1..61405af 100644
--- a/expressions/scalar/ScalarUnaryExpression.cpp
+++ b/expressions/scalar/ScalarUnaryExpression.cpp
@@ -42,31 +42,57 @@ namespace quickstep {
 
 struct SubBlocksReference;
 
-ScalarUnaryExpression::ScalarUnaryExpression(const UnaryOperation &operation,
-                                             Scalar *operand)
-    : Scalar(*operation.resultTypeForArgumentType(operand->getType())),
+ScalarUnaryExpression::ScalarUnaryExpression(
+    const OperationSignaturePtr &op_signature,
+    const UnaryOperationPtr &operation,
+    Scalar *operand,
+    const std::shared_ptr<const std::vector<TypedValue>> &static_arguments)
+    : Scalar(*operation->getResultType(operand->getType(), *static_arguments)),
+      op_signature_(op_signature),
       operation_(operation),
-      operand_(operand) {
-  initHelper(false);
+      operand_(operand),
+      static_arguments_(static_arguments) {
+  if (operation_->canApplyTo(operand_->getType(), *static_arguments_)) {
+    fast_operator_.reset(
+        operation_->makeUncheckedUnaryOperator(operand_->getType(),
+                                               *static_arguments_));
+    if (operand_->hasStaticValue()) {
+      static_value_.reset(new TypedValue(
+          fast_operator_->applyToTypedValue(operand_->getStaticValue())));
+    }
+  } else {
+    const Type &operand_type = operand_->getType();
+    throw OperationInapplicableToType(operation_->getName(), 1, operand_type.getName().c_str());
+  }
 }
 
 serialization::Scalar ScalarUnaryExpression::getProto() const {
   serialization::Scalar proto;
   proto.set_data_source(serialization::Scalar::UNARY_EXPRESSION);
-  proto.MutableExtension(serialization::ScalarUnaryExpression::operation)->CopyFrom(operation_.getProto());
+  proto.MutableExtension(
+      serialization::ScalarUnaryExpression::op_signature)->CopyFrom(
+          op_signature_->getProto());
   proto.MutableExtension(serialization::ScalarUnaryExpression::operand)->CopyFrom(operand_->getProto());
+  for (const TypedValue &value : *static_arguments_) {
+    proto.AddExtension(
+        serialization::ScalarUnaryExpression::static_arguments)->CopyFrom(
+            value.getProto());
+  }
 
   return proto;
 }
 
 Scalar* ScalarUnaryExpression::clone() const {
-  return new ScalarUnaryExpression(operation_, operand_->clone());
+  return new ScalarUnaryExpression(op_signature_,
+                                   operation_,
+                                   operand_->clone(),
+                                   static_arguments_);
 }
 
 TypedValue ScalarUnaryExpression::getValueForSingleTuple(const ValueAccessor &accessor,
                                                          const tuple_id tuple) const {
   if (fast_operator_.get() == nullptr) {
-    return static_value_.makeReferenceToThis();
+    return static_value_->makeReferenceToThis();
   } else {
     return fast_operator_->applyToTypedValue(operand_->getValueForSingleTuple(accessor, tuple));
   }
@@ -80,7 +106,7 @@ TypedValue ScalarUnaryExpression::getValueForJoinedTuples(
     const relation_id right_relation_id,
     const tuple_id right_tuple_id) const {
   if (fast_operator_.get() == nullptr) {
-    return static_value_.makeReferenceToThis();
+    return static_value_->makeReferenceToThis();
   } else {
     return fast_operator_->applyToTypedValue(operand_->getValueForJoinedTuples(left_accessor,
                                                                                left_relation_id,
@@ -96,7 +122,7 @@ ColumnVector* ScalarUnaryExpression::getAllValues(
     const SubBlocksReference *sub_blocks_ref) const {
   if (fast_operator_.get() == nullptr) {
     return ColumnVector::MakeVectorOfValue(getType(),
-                                           static_value_,
+                                           *static_value_,
                                            accessor->getNumTuplesVirtual());
   } else {
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
@@ -119,26 +145,9 @@ ColumnVector* ScalarUnaryExpression::getAllValuesForJoin(
     const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
   if (fast_operator_.get() == nullptr) {
     return ColumnVector::MakeVectorOfValue(getType(),
-                                           static_value_,
+                                           *static_value_,
                                            joined_tuple_ids.size());
   } else {
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-    const attribute_id operand_attr_id = operand_->getAttributeIdForValueAccessor();
-    if (operand_attr_id != -1) {
-      const relation_id operand_relation_id = operand_->getRelationIdForValueAccessor();
-      DCHECK_NE(operand_relation_id, -1);
-      DCHECK((operand_relation_id == left_relation_id)
-             || (operand_relation_id == right_relation_id));
-      const bool using_left_relation = (operand_relation_id == left_relation_id);
-      ValueAccessor *operand_accessor = using_left_relation ? left_accessor
-                                                            : right_accessor;
-      return fast_operator_->applyToValueAccessorForJoin(operand_accessor,
-                                                         using_left_relation,
-                                                         operand_attr_id,
-                                                         joined_tuple_ids);
-    }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
     std::unique_ptr<ColumnVector> operand_result(
         operand_->getAllValuesForJoin(left_relation_id,
                                       left_accessor,
@@ -149,21 +158,4 @@ ColumnVector* ScalarUnaryExpression::getAllValuesForJoin(
   }
 }
 
-void ScalarUnaryExpression::initHelper(bool own_children) {
-  if (operation_.canApplyToType(operand_->getType())) {
-    if (operand_->hasStaticValue()) {
-      static_value_ = operation_.applyToChecked(operand_->getStaticValue(),
-                                                operand_->getType());
-    } else {
-      fast_operator_.reset(operation_.makeUncheckedUnaryOperatorForType(operand_->getType()));
-    }
-  } else {
-    const Type &operand_type = operand_->getType();
-    if (!own_children) {
-      operand_.release();
-    }
-    throw OperationInapplicableToType(operation_.getName(), 1, operand_type.getName().c_str());
-  }
-}
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/expressions/scalar/ScalarUnaryExpression.hpp
----------------------------------------------------------------------
diff --git a/expressions/scalar/ScalarUnaryExpression.hpp b/expressions/scalar/ScalarUnaryExpression.hpp
index 608a842..c84e2bb 100644
--- a/expressions/scalar/ScalarUnaryExpression.hpp
+++ b/expressions/scalar/ScalarUnaryExpression.hpp
@@ -29,6 +29,7 @@
 #include "expressions/scalar/Scalar.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "types/TypedValue.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
 #include "utility/Macros.hpp"
 
@@ -50,14 +51,10 @@ struct SubBlocksReference;
  **/
 class ScalarUnaryExpression : public Scalar {
  public:
-  /**
-   * @brief Constructor.
-   *
-   * @param operation The unary operation to be performed.
-   * @param operand The argument of the operation, which this
-   *        ScalarUnaryExpression takes ownership of.
-   **/
-  ScalarUnaryExpression(const UnaryOperation &operation, Scalar *operand);
+  ScalarUnaryExpression(const OperationSignaturePtr &op_signature,
+                        const UnaryOperationPtr &operation,
+                        Scalar *operand,
+                        const std::shared_ptr<const std::vector<TypedValue>> &static_arguments);
 
   /**
    * @brief Destructor.
@@ -85,12 +82,12 @@ class ScalarUnaryExpression : public Scalar {
       const tuple_id right_tuple_id) const override;
 
   bool hasStaticValue() const override {
-    return fast_operator_.get() == nullptr;
+    return static_value_ != nullptr;
   }
 
   const TypedValue& getStaticValue() const override {
     DCHECK(hasStaticValue());
-    return static_value_;
+    return *static_value_;
   }
 
   ColumnVector* getAllValues(ValueAccessor *accessor,
@@ -106,10 +103,13 @@ class ScalarUnaryExpression : public Scalar {
  private:
   void initHelper(bool own_children);
 
-  const UnaryOperation &operation_;
+  const OperationSignaturePtr op_signature_;
+  const UnaryOperationPtr operation_;
+
+  const std::unique_ptr<Scalar> operand_;
+  const std::shared_ptr<const std::vector<TypedValue>> static_arguments_;
 
-  std::unique_ptr<Scalar> operand_;
-  TypedValue static_value_;
+  std::unique_ptr<TypedValue> static_value_;
   std::unique_ptr<UncheckedUnaryOperator> fast_operator_;
 
   friend class PredicateTest;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index b1bd870..7a8e04e 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -365,9 +365,6 @@ target_link_libraries(quickstep_parser_SqlParser
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
-                      quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationFactory
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_PtrList
                       quickstep_utility_PtrVector)
 target_link_libraries(quickstep_parser_SqlParserWrapper

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/parser/ParseBasicExpressions.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseBasicExpressions.cpp b/parser/ParseBasicExpressions.cpp
index b0b1247..8212286 100644
--- a/parser/ParseBasicExpressions.cpp
+++ b/parser/ParseBasicExpressions.cpp
@@ -68,27 +68,6 @@ void ParseAttribute::getFieldStringItems(
   }
 }
 
-std::string ParseUnaryExpression::getName() const {
-  return op_.getName();
-}
-
-string ParseUnaryExpression::generateName() const {
-  string name(op_.getShortName());
-  name.append(operand_->generateName());
-  return name;
-}
-
-void ParseUnaryExpression::getFieldStringItems(
-    std::vector<std::string> *inline_field_names,
-    std::vector<std::string> *inline_field_values,
-    std::vector<std::string> *non_container_child_field_names,
-    std::vector<const ParseTreeNode*> *non_container_child_fields,
-    std::vector<std::string> *container_child_field_names,
-    std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const {
-  non_container_child_field_names->push_back("");
-  non_container_child_fields->push_back(operand_.get());
-}
-
 std::string ParseBinaryExpression::getName() const {
   return op_.getName();
 }
@@ -175,59 +154,4 @@ void ParseFunctionCall::getFieldStringItems(
   }
 }
 
-std::string ParseExtractFunction::generateName() const {
-  std::string name;
-  name.append("EXTRACT(");
-  name.append(extract_field_->value());
-  name.append(" FROM ");
-  name.append(date_expression_->generateName());
-  name.push_back(')');
-  return name;
-}
-
-void ParseExtractFunction::getFieldStringItems(
-    std::vector<std::string> *inline_field_names,
-    std::vector<std::string> *inline_field_values,
-    std::vector<std::string> *non_container_child_field_names,
-    std::vector<const ParseTreeNode*> *non_container_child_fields,
-    std::vector<std::string> *container_child_field_names,
-    std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const {
-  inline_field_names->push_back("unit");
-  inline_field_values->push_back(extract_field_->value());
-
-  non_container_child_field_names->push_back("date_expression");
-  non_container_child_fields->push_back(date_expression_.get());
-}
-
-std::string ParseSubstringFunction::generateName() const {
-  std::string name;
-  name.append("SUBSTRING(");
-  name.append(operand_->generateName());
-  name.append(" FROM ");
-  name.append(std::to_string(start_position_));
-  if (length_ != kDefaultLength) {
-    name.append(" FOR ");
-    name.append(std::to_string(length_));
-  }
-  name.push_back(')');
-  return name;
-}
-
-void ParseSubstringFunction::getFieldStringItems(
-    std::vector<std::string> *inline_field_names,
-    std::vector<std::string> *inline_field_values,
-    std::vector<std::string> *non_container_child_field_names,
-    std::vector<const ParseTreeNode*> *non_container_child_fields,
-    std::vector<std::string> *container_child_field_names,
-    std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const {
-  inline_field_names->push_back("start_position");
-  inline_field_values->push_back(std::to_string(start_position_));
-
-  inline_field_names->push_back("length");
-  inline_field_values->push_back(std::to_string(length_));
-
-  non_container_child_field_names->push_back("operand");
-  non_container_child_fields->push_back(operand_.get());
-}
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/parser/ParseBasicExpressions.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseBasicExpressions.hpp b/parser/ParseBasicExpressions.hpp
index d8de669..edf5aab 100644
--- a/parser/ParseBasicExpressions.hpp
+++ b/parser/ParseBasicExpressions.hpp
@@ -173,73 +173,6 @@ class ParseAttribute : public ParseExpression {
 
 
 /**
- * @brief The parsed representation of an unary operation applied to an expression.
- **/
-class ParseUnaryExpression : public ParseExpression {
- public:
-  /**
-   * @brief Constructor.
-   *
-   * @param line_number Line number of the first token of this node in the SQL statement.
-   * @param column_number Column number of the first token of this node in the SQL statement.
-   * @param op The UnaryOperation from the quickstep type system to apply.
-   * @param operand The parsed scalar representation of the unary operation's
-   *        argument, which becomes owned by this ParseScalarUnaryExpression.
-   **/
-  ParseUnaryExpression(const int line_number,
-                       const int column_number,
-                       const UnaryOperation &op,
-                       ParseExpression *operand)
-      : ParseExpression(line_number, column_number),
-        op_(op),
-        operand_(operand) {
-  }
-
-  /**
-   * @brief Destructor.
-   */
-  ~ParseUnaryExpression() override {
-  }
-
-  ExpressionType getExpressionType() const override {
-    return kUnaryExpression;
-  }
-
-  std::string getName() const override;
-
-  /**
-   * @return The unary operation.
-   */
-  const UnaryOperation& op() const {
-    return op_;
-  }
-
-  /**
-   * @return The operand expression.
-   */
-  const ParseExpression* operand() const {
-    return operand_.get();
-  }
-
-  std::string generateName() const override;
-
- protected:
-  void getFieldStringItems(
-      std::vector<std::string> *inline_field_names,
-      std::vector<std::string> *inline_field_values,
-      std::vector<std::string> *non_container_child_field_names,
-      std::vector<const ParseTreeNode*> *non_container_child_fields,
-      std::vector<std::string> *container_child_field_names,
-      std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override;
-
- private:
-  const UnaryOperation &op_;
-  std::unique_ptr<ParseExpression> operand_;
-
-  DISALLOW_COPY_AND_ASSIGN(ParseUnaryExpression);
-};
-
-/**
  * @brief The parsed representation of a binary operation applied to two
  *        expressions.
  **/
@@ -495,144 +428,6 @@ class ParseFunctionCall : public ParseExpression {
   DISALLOW_COPY_AND_ASSIGN(ParseFunctionCall);
 };
 
-
-/**
- * @brief Parsed representation of EXTRACT(unit FROM date).
- */
-class ParseExtractFunction : public ParseExpression {
- public:
-  /**
-   * @brief Constructor.
-   *
-   * @param line_number The line number of the token "extract" in the statement.
-   * @param column_number The column number of the token "extract in the statement.
-   * @param extract_field The field to extract.
-   * @param source_expression The expression to extract a field from.
-   */
-  ParseExtractFunction(const int line_number,
-                       const int column_number,
-                       ParseString *extract_field,
-                       ParseExpression *date_expression)
-      : ParseExpression(line_number, column_number),
-        extract_field_(extract_field),
-        date_expression_(date_expression) {
-  }
-
-  ExpressionType getExpressionType() const override {
-    return kExtract;
-  }
-
-  std::string getName() const override {
-    return "Extract";
-  }
-
-  /**
-   * @return The field to extract.
-   */
-  const ParseString* extract_field() const {
-    return extract_field_.get();
-  }
-
-  /**
-   * @return The expression to extract a field from.
-   */
-  const ParseExpression* date_expression() const {
-    return date_expression_.get();
-  }
-
-  std::string generateName() const override;
-
- protected:
-  void getFieldStringItems(
-      std::vector<std::string> *inline_field_names,
-      std::vector<std::string> *inline_field_values,
-      std::vector<std::string> *non_container_child_field_names,
-      std::vector<const ParseTreeNode*> *non_container_child_fields,
-      std::vector<std::string> *container_child_field_names,
-      std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override;
-
- private:
-  std::unique_ptr<ParseString> extract_field_;
-  std::unique_ptr<ParseExpression> date_expression_;
-
-  DISALLOW_COPY_AND_ASSIGN(ParseExtractFunction);
-};
-
-
-/**
- * @brief Parsed representation of the substring function.
- */
-class ParseSubstringFunction : public ParseExpression {
- public:
-  static constexpr std::size_t kDefaultLength = std::numeric_limits<std::size_t>::max();
-
-  /**
-   * @brief Constructor.
-   *
-   * @param line_number The line number of the first token of the function call.
-   * @param column_number The column number of the first token of the function call.
-   * @param operand The operand of the substring.
-   * @param start_position The 1-based starting position of the substring.
-   * @param length Optional substring length.
-   */
-  ParseSubstringFunction(const int line_number,
-                         const int column_number,
-                         ParseExpression *operand,
-                         const std::size_t start_position,
-                         const std::size_t length = kDefaultLength)
-      : ParseExpression(line_number, column_number),
-        operand_(operand),
-        start_position_(start_position),
-        length_(length) {}
-
-  ExpressionType getExpressionType() const override {
-    return kSubstring;
-  }
-
-  std::string getName() const override {
-    return "Substring";
-  }
-
-  /**
-   * @return The operand of the substring.
-   */
-  const ParseExpression* operand() const {
-    return operand_.get();
-  }
-
-  /**
-   * @return The 1-based starting position of the substring.
-   */
-  std::size_t start_position() const {
-    return start_position_;
-  }
-
-  /**
-   * @return Then substring length.
-   */
-  std::size_t length() const {
-    return length_;
-  }
-
-  std::string generateName() const override;
-
- protected:
-  void getFieldStringItems(
-      std::vector<std::string> *inline_field_names,
-      std::vector<std::string> *inline_field_values,
-      std::vector<std::string> *non_container_child_field_names,
-      std::vector<const ParseTreeNode*> *non_container_child_fields,
-      std::vector<std::string> *container_child_field_names,
-      std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override;
-
- private:
-  std::unique_ptr<ParseExpression> operand_;
-  const std::size_t start_position_;
-  const std::size_t length_;
-
-  DISALLOW_COPY_AND_ASSIGN(ParseSubstringFunction);
-};
-
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/parser/ParseExpression.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseExpression.hpp b/parser/ParseExpression.hpp
index 1b9ade4..94b4487 100644
--- a/parser/ParseExpression.hpp
+++ b/parser/ParseExpression.hpp
@@ -39,14 +39,11 @@ class ParseExpression : public ParseTreeNode {
   enum ExpressionType {
     kAttribute,
     kBinaryExpression,
-    kExtract,
     kFunctionCall,
     kScalarLiteral,
     kSearchedCaseExpression,
     kSimpleCaseExpression,
     kSubqueryExpression,
-    kSubstring,
-    kUnaryExpression,
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/parser/SqlLexer.lpp
----------------------------------------------------------------------
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index 0953dc6..ac88515 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -183,6 +183,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "blocksample"      return TOKEN_BLOCKSAMPLE;
   "bloomfilter"      return TOKEN_BLOOM_FILTER;
   "case"             return TOKEN_CASE;
+  "cast"             return TOKEN_CAST;
   "csbtree"          return TOKEN_CSB_TREE;
   "by"               return TOKEN_BY;
   "char"             return TOKEN_CHARACTER;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index 29b69d7..fd13aa7 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -107,9 +107,6 @@ typedef struct YYLTYPE {
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
-#include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationFactory.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
 #include "utility/PtrList.hpp"
 #include "utility/PtrVector.hpp"
 
@@ -187,7 +184,7 @@ typedef void* yyscan_t;
   quickstep::ParseStatementQuit *quit_statement_;
 
   const quickstep::Comparison *comparison_;
-  const quickstep::UnaryOperation *unary_operation_;
+  quickstep::ParseString *unary_operation_;
   const quickstep::BinaryOperation *binary_operation_;
 
   quickstep::ParseFunctionCall *function_call_;
@@ -258,6 +255,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_CSB_TREE;
 %token TOKEN_BY;
 %token TOKEN_CASE;
+%token TOKEN_CAST;
 %token TOKEN_CHARACTER;
 %token TOKEN_CHECK;
 %token TOKEN_COLUMN;
@@ -386,6 +384,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
   add_expression
   case_expression
   opt_else_clause
+  cast_function
   extract_function
   substr_function
 
@@ -613,7 +612,6 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 
 %destructor { } <boolean_value_>
 %destructor { } <comparison_>
-%destructor { } <unary_operation_>
 %destructor { } <binary_operation_>
 %destructor { } <join_type_>
 
@@ -1632,7 +1630,10 @@ multiply_expression:
 
 unary_expression:
   unary_operation expression_base {
-    $$ = new quickstep::ParseUnaryExpression(@1.first_line, @1.first_column, *$1, $2);
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($2);
+    $$ = new quickstep::ParseFunctionCall(
+        @1.first_line, @1.first_column, false, $1, arguments);
   }
   | expression_base {
     $$ = $1;
@@ -1656,6 +1657,9 @@ expression_base:
     $1->setWindow($4);
     $$ = $1;
   }
+  | cast_function {
+    $$ = $1;
+  }
   | extract_function {
     $$ = $1;
   }
@@ -1688,19 +1692,50 @@ function_call:
     $$ = new quickstep::ParseFunctionCall(@1.first_line, @1.first_column, true, $1, $4);
   };
 
+cast_function:
+  TOKEN_CAST '(' add_expression TOKEN_AS  data_type ')' {
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($3);
+    arguments->push_back(new quickstep::ParseScalarLiteral(
+        new quickstep::StringParseLiteralValue(
+            new quickstep::ParseString(@5.first_line,
+                                       @5.first_column,
+                                       $5->getType().getName()),
+            nullptr)));
+    delete $5;
+    auto *name = new quickstep::ParseString(@1.first_line, @1.first_column, "cast");
+    $$ = new quickstep::ParseFunctionCall(
+        @1.first_line, @1.first_column, false, name, arguments);
+  };
+
 extract_function:
   TOKEN_EXTRACT '(' datetime_unit TOKEN_FROM add_expression ')' {
-    $$ = new quickstep::ParseExtractFunction(@1.first_line, @1.first_column, $3, $5);
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($5);
+    arguments->push_back(new quickstep::ParseScalarLiteral(
+        new quickstep::StringParseLiteralValue($3, nullptr)));
+    auto *name = new quickstep::ParseString(@1.first_line, @1.first_column, "extract");
+    $$ = new quickstep::ParseFunctionCall(
+        @1.first_line, @1.first_column, false, name, arguments);
   };
 
 substr_function:
   TOKEN_SUBSTRING '(' add_expression TOKEN_FROM TOKEN_UNSIGNED_NUMVAL ')' {
-    $$ = new quickstep::ParseSubstringFunction(
-        @1.first_line, @1.first_column, $3, $5->long_value());
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($3);
+    arguments->push_back(new quickstep::ParseScalarLiteral($5));
+    auto *name = new quickstep::ParseString(@1.first_line, @1.first_column, "substring");
+    $$ = new quickstep::ParseFunctionCall(
+        @1.first_line, @1.first_column, false, name, arguments);
   }
   | TOKEN_SUBSTRING '(' add_expression TOKEN_FROM TOKEN_UNSIGNED_NUMVAL TOKEN_FOR TOKEN_UNSIGNED_NUMVAL ')' {
-    $$ = new quickstep::ParseSubstringFunction(
-        @1.first_line, @1.first_column, $3, $5->long_value(), $7->long_value());
+    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
+    arguments->push_back($3);
+    arguments->push_back(new quickstep::ParseScalarLiteral($5));
+    arguments->push_back(new quickstep::ParseScalarLiteral($7));
+    auto *name = new quickstep::ParseString(@1.first_line, @1.first_column, "substring");
+    $$ = new quickstep::ParseFunctionCall(
+        @1.first_line, @1.first_column, false, name, arguments);
   };
 
 case_expression:
@@ -1920,7 +1955,7 @@ unary_operation:
      * to shift rather than reduce, the case in 'literal_value' has precedence
      * over this one.
      **/
-    $$ = &quickstep::UnaryOperationFactory::GetUnaryOperation(quickstep::UnaryOperationID::kNegate);
+    $$ = new quickstep::ParseString(@1.first_line, @1.first_column, std::string("-"));
   };
 
 add_operation:



[6/8] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/parser/preprocessed/SqlLexer_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.hpp b/parser/preprocessed/SqlLexer_gen.hpp
index 1997e75..4c6cce4 100644
--- a/parser/preprocessed/SqlLexer_gen.hpp
+++ b/parser/preprocessed/SqlLexer_gen.hpp
@@ -2,9 +2,9 @@
 #define quickstep_yyHEADER_H 1
 #define quickstep_yyIN_HEADER 1
 
-#line 5 "SqlLexer_gen.hpp"
+#line 6 "SqlLexer_gen.hpp"
 
-#line 7 "SqlLexer_gen.hpp"
+#line 8 "SqlLexer_gen.hpp"
 
 #define  YY_INT_ALIGNED short int
 
@@ -13,89 +13,11 @@
 #define FLEX_SCANNER
 #define YY_FLEX_MAJOR_VERSION 2
 #define YY_FLEX_MINOR_VERSION 6
-#define YY_FLEX_SUBMINOR_VERSION 3
+#define YY_FLEX_SUBMINOR_VERSION 0
 #if YY_FLEX_SUBMINOR_VERSION > 0
 #define FLEX_BETA
 #endif
 
-    #define yy_create_buffer quickstep_yy_create_buffer
-
-    #define yy_delete_buffer quickstep_yy_delete_buffer
-
-    #define yy_scan_buffer quickstep_yy_scan_buffer
-
-    #define yy_scan_string quickstep_yy_scan_string
-
-    #define yy_scan_bytes quickstep_yy_scan_bytes
-
-    #define yy_init_buffer quickstep_yy_init_buffer
-
-    #define yy_flush_buffer quickstep_yy_flush_buffer
-
-    #define yy_load_buffer_state quickstep_yy_load_buffer_state
-
-    #define yy_switch_to_buffer quickstep_yy_switch_to_buffer
-
-    #define yypush_buffer_state quickstep_yypush_buffer_state
-
-    #define yypop_buffer_state quickstep_yypop_buffer_state
-
-    #define yyensure_buffer_stack quickstep_yyensure_buffer_stack
-
-    #define yylex quickstep_yylex
-
-    #define yyrestart quickstep_yyrestart
-
-    #define yylex_init quickstep_yylex_init
-
-    #define yylex_init_extra quickstep_yylex_init_extra
-
-    #define yylex_destroy quickstep_yylex_destroy
-
-    #define yyget_debug quickstep_yyget_debug
-
-    #define yyset_debug quickstep_yyset_debug
-
-    #define yyget_extra quickstep_yyget_extra
-
-    #define yyset_extra quickstep_yyset_extra
-
-    #define yyget_in quickstep_yyget_in
-
-    #define yyset_in quickstep_yyset_in
-
-    #define yyget_out quickstep_yyget_out
-
-    #define yyset_out quickstep_yyset_out
-
-    #define yyget_leng quickstep_yyget_leng
-
-    #define yyget_text quickstep_yyget_text
-
-    #define yyget_lineno quickstep_yyget_lineno
-
-    #define yyset_lineno quickstep_yyset_lineno
-
-        #define yyget_column quickstep_yyget_column
-
-        #define yyset_column quickstep_yyset_column
-
-    #define yywrap quickstep_yywrap
-
-    #define yyget_lval quickstep_yyget_lval
-
-    #define yyset_lval quickstep_yyset_lval
-
-    #define yyget_lloc quickstep_yyget_lloc
-
-    #define yyset_lloc quickstep_yyset_lloc
-
-    #define yyalloc quickstep_yyalloc
-
-    #define yyrealloc quickstep_yyrealloc
-
-    #define yyfree quickstep_yyfree
-
 /* First, we deal with  platform-specific or compiler-specific issues. */
 
 /* begin standard C headers. */
@@ -170,13 +92,25 @@ typedef unsigned int flex_uint32_t;
 
 #endif /* ! FLEXINT_H */
 
-/* TODO: this is always defined, so inline it */
-#define yyconst const
+#ifdef __cplusplus
+
+/* The "const" storage-class-modifier is valid. */
+#define YY_USE_CONST
+
+#else	/* ! __cplusplus */
 
-#if defined(__GNUC__) && __GNUC__ >= 3
-#define yynoreturn __attribute__((__noreturn__))
+/* C99 requires __STDC__ to be defined as 1. */
+#if defined (__STDC__)
+
+#define YY_USE_CONST
+
+#endif	/* defined (__STDC__) */
+#endif	/* ! __cplusplus */
+
+#ifdef YY_USE_CONST
+#define yyconst const
 #else
-#define yynoreturn
+#define yyconst
 #endif
 
 /* An opaque pointer. */
@@ -231,12 +165,12 @@ struct yy_buffer_state
 	/* Size of input buffer in bytes, not including room for EOB
 	 * characters.
 	 */
-	int yy_buf_size;
+	yy_size_t yy_buf_size;
 
 	/* Number of characters read into yy_ch_buf, not including EOB
 	 * characters.
 	 */
-	int yy_n_chars;
+	yy_size_t yy_n_chars;
 
 	/* Whether we "own" the buffer - i.e., we know we created it,
 	 * and can realloc() it to grow it, and should free() it to
@@ -259,7 +193,7 @@ struct yy_buffer_state
 
     int yy_bs_lineno; /**< The line count. */
     int yy_bs_column; /**< The column count. */
-
+    
 	/* Whether to try to fill the input buffer when we reach the
 	 * end of it.
 	 */
@@ -270,21 +204,21 @@ struct yy_buffer_state
 	};
 #endif /* !YY_STRUCT_YY_BUFFER_STATE */
 
-void quickstep_yyrestart ( FILE *input_file , yyscan_t yyscanner );
-void quickstep_yy_switch_to_buffer ( YY_BUFFER_STATE new_buffer , yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_create_buffer ( FILE *file, int size , yyscan_t yyscanner );
-void quickstep_yy_delete_buffer ( YY_BUFFER_STATE b , yyscan_t yyscanner );
-void quickstep_yy_flush_buffer ( YY_BUFFER_STATE b , yyscan_t yyscanner );
-void quickstep_yypush_buffer_state ( YY_BUFFER_STATE new_buffer , yyscan_t yyscanner );
-void quickstep_yypop_buffer_state ( yyscan_t yyscanner );
+void quickstep_yyrestart (FILE *input_file ,yyscan_t yyscanner );
+void quickstep_yy_switch_to_buffer (YY_BUFFER_STATE new_buffer ,yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_create_buffer (FILE *file,int size ,yyscan_t yyscanner );
+void quickstep_yy_delete_buffer (YY_BUFFER_STATE b ,yyscan_t yyscanner );
+void quickstep_yy_flush_buffer (YY_BUFFER_STATE b ,yyscan_t yyscanner );
+void quickstep_yypush_buffer_state (YY_BUFFER_STATE new_buffer ,yyscan_t yyscanner );
+void quickstep_yypop_buffer_state (yyscan_t yyscanner );
 
-YY_BUFFER_STATE quickstep_yy_scan_buffer ( char *base, yy_size_t size , yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_scan_string ( const char *yy_str , yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_scan_bytes ( const char *bytes, int len , yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_buffer (char *base,yy_size_t size ,yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_string (yyconst char *yy_str ,yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_bytes (yyconst char *bytes,yy_size_t len ,yyscan_t yyscanner );
 
-void *quickstep_yyalloc ( yy_size_t , yyscan_t yyscanner );
-void *quickstep_yyrealloc ( void *, yy_size_t , yyscan_t yyscanner );
-void quickstep_yyfree ( void * , yyscan_t yyscanner );
+void *quickstep_yyalloc (yy_size_t ,yyscan_t yyscanner );
+void *quickstep_yyrealloc (void *,yy_size_t ,yyscan_t yyscanner );
+void quickstep_yyfree (void * ,yyscan_t yyscanner );
 
 /* Begin user sect3 */
 
@@ -317,48 +251,48 @@ void quickstep_yyfree ( void * , yyscan_t yyscanner );
 
 int quickstep_yylex_init (yyscan_t* scanner);
 
-int quickstep_yylex_init_extra ( YY_EXTRA_TYPE user_defined, yyscan_t* scanner);
+int quickstep_yylex_init_extra (YY_EXTRA_TYPE user_defined,yyscan_t* scanner);
 
 /* Accessor methods to globals.
    These are made visible to non-reentrant scanners for convenience. */
 
-int quickstep_yylex_destroy ( yyscan_t yyscanner );
+int quickstep_yylex_destroy (yyscan_t yyscanner );
 
-int quickstep_yyget_debug ( yyscan_t yyscanner );
+int quickstep_yyget_debug (yyscan_t yyscanner );
 
-void quickstep_yyset_debug ( int debug_flag , yyscan_t yyscanner );
+void quickstep_yyset_debug (int debug_flag ,yyscan_t yyscanner );
 
-YY_EXTRA_TYPE quickstep_yyget_extra ( yyscan_t yyscanner );
+YY_EXTRA_TYPE quickstep_yyget_extra (yyscan_t yyscanner );
 
-void quickstep_yyset_extra ( YY_EXTRA_TYPE user_defined , yyscan_t yyscanner );
+void quickstep_yyset_extra (YY_EXTRA_TYPE user_defined ,yyscan_t yyscanner );
 
-FILE *quickstep_yyget_in ( yyscan_t yyscanner );
+FILE *quickstep_yyget_in (yyscan_t yyscanner );
 
-void quickstep_yyset_in  ( FILE * _in_str , yyscan_t yyscanner );
+void quickstep_yyset_in  (FILE * _in_str ,yyscan_t yyscanner );
 
-FILE *quickstep_yyget_out ( yyscan_t yyscanner );
+FILE *quickstep_yyget_out (yyscan_t yyscanner );
 
-void quickstep_yyset_out  ( FILE * _out_str , yyscan_t yyscanner );
+void quickstep_yyset_out  (FILE * _out_str ,yyscan_t yyscanner );
 
-			int quickstep_yyget_leng ( yyscan_t yyscanner );
+yy_size_t quickstep_yyget_leng (yyscan_t yyscanner );
 
-char *quickstep_yyget_text ( yyscan_t yyscanner );
+char *quickstep_yyget_text (yyscan_t yyscanner );
 
-int quickstep_yyget_lineno ( yyscan_t yyscanner );
+int quickstep_yyget_lineno (yyscan_t yyscanner );
 
-void quickstep_yyset_lineno ( int _line_number , yyscan_t yyscanner );
+void quickstep_yyset_lineno (int _line_number ,yyscan_t yyscanner );
 
-int quickstep_yyget_column  ( yyscan_t yyscanner );
+int quickstep_yyget_column  (yyscan_t yyscanner );
 
-void quickstep_yyset_column ( int _column_no , yyscan_t yyscanner );
+void quickstep_yyset_column (int _column_no ,yyscan_t yyscanner );
 
-YYSTYPE * quickstep_yyget_lval ( yyscan_t yyscanner );
+YYSTYPE * quickstep_yyget_lval (yyscan_t yyscanner );
 
-void quickstep_yyset_lval ( YYSTYPE * yylval_param , yyscan_t yyscanner );
+void quickstep_yyset_lval (YYSTYPE * yylval_param ,yyscan_t yyscanner );
 
-       YYLTYPE *quickstep_yyget_lloc ( yyscan_t yyscanner );
+       YYLTYPE *quickstep_yyget_lloc (yyscan_t yyscanner );
     
-        void quickstep_yyset_lloc ( YYLTYPE * yylloc_param , yyscan_t yyscanner );
+        void quickstep_yyset_lloc (YYLTYPE * yylloc_param ,yyscan_t yyscanner );
     
 /* Macros after this point can all be overridden by user definitions in
  * section 1.
@@ -366,18 +300,18 @@ void quickstep_yyset_lval ( YYSTYPE * yylval_param , yyscan_t yyscanner );
 
 #ifndef YY_SKIP_YYWRAP
 #ifdef __cplusplus
-extern "C" int quickstep_yywrap ( yyscan_t yyscanner );
+extern "C" int quickstep_yywrap (yyscan_t yyscanner );
 #else
-extern int quickstep_yywrap ( yyscan_t yyscanner );
+extern int quickstep_yywrap (yyscan_t yyscanner );
 #endif
 #endif
 
 #ifndef yytext_ptr
-static void yy_flex_strncpy ( char *, const char *, int , yyscan_t yyscanner);
+static void yy_flex_strncpy (char *,yyconst char *,int ,yyscan_t yyscanner);
 #endif
 
 #ifdef YY_NEED_STRLEN
-static int yy_flex_strlen ( const char * , yyscan_t yyscanner);
+static int yy_flex_strlen (yyconst char * ,yyscan_t yyscanner);
 #endif
 
 #ifndef YY_NO_INPUT
@@ -406,7 +340,7 @@ static int yy_flex_strlen ( const char * , yyscan_t yyscanner);
 #define YY_DECL_IS_OURS 1
 
 extern int quickstep_yylex \
-               (YYSTYPE * yylval_param, YYLTYPE * yylloc_param , yyscan_t yyscanner);
+               (YYSTYPE * yylval_param,YYLTYPE * yylloc_param ,yyscan_t yyscanner);
 
 #define YY_DECL int quickstep_yylex \
                (YYSTYPE * yylval_param, YYLTYPE * yylloc_param , yyscan_t yyscanner)
@@ -426,9 +360,9 @@ extern int quickstep_yylex \
 #undef YY_DECL
 #endif
 
-#line 466 "../SqlLexer.lpp"
+#line 467 "../SqlLexer.lpp"
 
 
-#line 432 "SqlLexer_gen.hpp"
+#line 367 "SqlLexer_gen.hpp"
 #undef quickstep_yyIN_HEADER
 #endif /* quickstep_yyHEADER_H */


[7/8] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/parser/preprocessed/SqlLexer_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlLexer_gen.cpp b/parser/preprocessed/SqlLexer_gen.cpp
index 1cb0ac8..428f029 100644
--- a/parser/preprocessed/SqlLexer_gen.cpp
+++ b/parser/preprocessed/SqlLexer_gen.cpp
@@ -1,6 +1,6 @@
-#line 1 "SqlLexer_gen.cpp"
+#line 2 "SqlLexer_gen.cpp"
 
-#line 3 "SqlLexer_gen.cpp"
+#line 4 "SqlLexer_gen.cpp"
 
 #define  YY_INT_ALIGNED short int
 
@@ -9,89 +9,11 @@
 #define FLEX_SCANNER
 #define YY_FLEX_MAJOR_VERSION 2
 #define YY_FLEX_MINOR_VERSION 6
-#define YY_FLEX_SUBMINOR_VERSION 3
+#define YY_FLEX_SUBMINOR_VERSION 0
 #if YY_FLEX_SUBMINOR_VERSION > 0
 #define FLEX_BETA
 #endif
 
-    #define yy_create_buffer quickstep_yy_create_buffer
-
-    #define yy_delete_buffer quickstep_yy_delete_buffer
-
-    #define yy_scan_buffer quickstep_yy_scan_buffer
-
-    #define yy_scan_string quickstep_yy_scan_string
-
-    #define yy_scan_bytes quickstep_yy_scan_bytes
-
-    #define yy_init_buffer quickstep_yy_init_buffer
-
-    #define yy_flush_buffer quickstep_yy_flush_buffer
-
-    #define yy_load_buffer_state quickstep_yy_load_buffer_state
-
-    #define yy_switch_to_buffer quickstep_yy_switch_to_buffer
-
-    #define yypush_buffer_state quickstep_yypush_buffer_state
-
-    #define yypop_buffer_state quickstep_yypop_buffer_state
-
-    #define yyensure_buffer_stack quickstep_yyensure_buffer_stack
-
-    #define yylex quickstep_yylex
-
-    #define yyrestart quickstep_yyrestart
-
-    #define yylex_init quickstep_yylex_init
-
-    #define yylex_init_extra quickstep_yylex_init_extra
-
-    #define yylex_destroy quickstep_yylex_destroy
-
-    #define yyget_debug quickstep_yyget_debug
-
-    #define yyset_debug quickstep_yyset_debug
-
-    #define yyget_extra quickstep_yyget_extra
-
-    #define yyset_extra quickstep_yyset_extra
-
-    #define yyget_in quickstep_yyget_in
-
-    #define yyset_in quickstep_yyset_in
-
-    #define yyget_out quickstep_yyget_out
-
-    #define yyset_out quickstep_yyset_out
-
-    #define yyget_leng quickstep_yyget_leng
-
-    #define yyget_text quickstep_yyget_text
-
-    #define yyget_lineno quickstep_yyget_lineno
-
-    #define yyset_lineno quickstep_yyset_lineno
-
-        #define yyget_column quickstep_yyget_column
-
-        #define yyset_column quickstep_yyset_column
-
-    #define yywrap quickstep_yywrap
-
-    #define yyget_lval quickstep_yyget_lval
-
-    #define yyset_lval quickstep_yyset_lval
-
-    #define yyget_lloc quickstep_yyget_lloc
-
-    #define yyset_lloc quickstep_yyset_lloc
-
-    #define yyalloc quickstep_yyalloc
-
-    #define yyrealloc quickstep_yyrealloc
-
-    #define yyfree quickstep_yyfree
-
 /* First, we deal with  platform-specific or compiler-specific issues. */
 
 /* begin standard C headers. */
@@ -166,22 +88,36 @@ typedef unsigned int flex_uint32_t;
 
 #endif /* ! FLEXINT_H */
 
-/* TODO: this is always defined, so inline it */
-#define yyconst const
+#ifdef __cplusplus
 
-#if defined(__GNUC__) && __GNUC__ >= 3
-#define yynoreturn __attribute__((__noreturn__))
+/* The "const" storage-class-modifier is valid. */
+#define YY_USE_CONST
+
+#else	/* ! __cplusplus */
+
+/* C99 requires __STDC__ to be defined as 1. */
+#if defined (__STDC__)
+
+#define YY_USE_CONST
+
+#endif	/* defined (__STDC__) */
+#endif	/* ! __cplusplus */
+
+#ifdef YY_USE_CONST
+#define yyconst const
 #else
-#define yynoreturn
+#define yyconst
 #endif
 
 /* Returned upon end-of-file. */
 #define YY_NULL 0
 
-/* Promotes a possibly negative, possibly signed char to an
- *   integer in range [0..255] for use as an array index.
+/* Promotes a possibly negative, possibly signed char to an unsigned
+ * integer for use as an array index.  If the signed char is negative,
+ * we want to instead treat it as an 8-bit unsigned char, hence the
+ * double cast.
  */
-#define YY_SC_TO_UI(c) ((YY_CHAR) (c))
+#define YY_SC_TO_UI(c) ((unsigned int) (unsigned char) c)
 
 /* An opaque pointer. */
 #ifndef YY_TYPEDEF_YY_SCANNER_T
@@ -205,16 +141,20 @@ typedef void* yyscan_t;
  * definition of BEGIN.
  */
 #define BEGIN yyg->yy_start = 1 + 2 *
+
 /* Translate the current start state into a value that can be later handed
  * to BEGIN to return to the state.  The YYSTATE alias is for lex
  * compatibility.
  */
 #define YY_START ((yyg->yy_start - 1) / 2)
 #define YYSTATE YY_START
+
 /* Action number for EOF rule of a given start state. */
 #define YY_STATE_EOF(state) (YY_END_OF_BUFFER + state + 1)
+
 /* Special action meaning "start processing a new file". */
 #define YY_NEW_FILE quickstep_yyrestart(yyin ,yyscanner )
+
 #define YY_END_OF_BUFFER_CHAR 0
 
 /* Size of default input buffer. */
@@ -247,10 +187,10 @@ typedef size_t yy_size_t;
 #define EOB_ACT_CONTINUE_SCAN 0
 #define EOB_ACT_END_OF_FILE 1
 #define EOB_ACT_LAST_MATCH 2
-    
+
     /* Note: We specifically omit the test for yy_rule_can_match_eol because it requires
      *       access to the local variable yy_act. Since yyless() is a macro, it would break
-     *       existing scanners that call yyless() from OUTSIDE quickstep_yylex.
+     *       existing scanners that call yyless() from OUTSIDE quickstep_yylex. 
      *       One obvious solution it to make yy_act a global. I tried that, and saw
      *       a 5% performance hit in a non-yylineno scanner, because yy_act is
      *       normally declared as a register variable-- so it is not worth it.
@@ -283,6 +223,7 @@ typedef size_t yy_size_t;
 		YY_DO_BEFORE_ACTION; /* set up yytext again */ \
 		} \
 	while ( 0 )
+
 #define unput(c) yyunput( c, yyg->yytext_ptr , yyscanner )
 
 #ifndef YY_STRUCT_YY_BUFFER_STATE
@@ -297,12 +238,12 @@ struct yy_buffer_state
 	/* Size of input buffer in bytes, not including room for EOB
 	 * characters.
 	 */
-	int yy_buf_size;
+	yy_size_t yy_buf_size;
 
 	/* Number of characters read into yy_ch_buf, not including EOB
 	 * characters.
 	 */
-	int yy_n_chars;
+	yy_size_t yy_n_chars;
 
 	/* Whether we "own" the buffer - i.e., we know we created it,
 	 * and can realloc() it to grow it, and should free() it to
@@ -325,7 +266,7 @@ struct yy_buffer_state
 
     int yy_bs_lineno; /**< The line count. */
     int yy_bs_column; /**< The column count. */
-
+    
 	/* Whether to try to fill the input buffer when we reach the
 	 * end of it.
 	 */
@@ -359,33 +300,36 @@ struct yy_buffer_state
 #define YY_CURRENT_BUFFER ( yyg->yy_buffer_stack \
                           ? yyg->yy_buffer_stack[yyg->yy_buffer_stack_top] \
                           : NULL)
+
 /* Same as previous macro, but useful when we know that the buffer stack is not
  * NULL or when we need an lvalue. For internal use only.
  */
 #define YY_CURRENT_BUFFER_LVALUE yyg->yy_buffer_stack[yyg->yy_buffer_stack_top]
 
-void quickstep_yyrestart ( FILE *input_file , yyscan_t yyscanner );
-void quickstep_yy_switch_to_buffer ( YY_BUFFER_STATE new_buffer , yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_create_buffer ( FILE *file, int size , yyscan_t yyscanner );
-void quickstep_yy_delete_buffer ( YY_BUFFER_STATE b , yyscan_t yyscanner );
-void quickstep_yy_flush_buffer ( YY_BUFFER_STATE b , yyscan_t yyscanner );
-void quickstep_yypush_buffer_state ( YY_BUFFER_STATE new_buffer , yyscan_t yyscanner );
-void quickstep_yypop_buffer_state ( yyscan_t yyscanner );
-
-static void quickstep_yyensure_buffer_stack ( yyscan_t yyscanner );
-static void quickstep_yy_load_buffer_state ( yyscan_t yyscanner );
-static void quickstep_yy_init_buffer ( YY_BUFFER_STATE b, FILE *file , yyscan_t yyscanner );
+void quickstep_yyrestart (FILE *input_file ,yyscan_t yyscanner );
+void quickstep_yy_switch_to_buffer (YY_BUFFER_STATE new_buffer ,yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_create_buffer (FILE *file,int size ,yyscan_t yyscanner );
+void quickstep_yy_delete_buffer (YY_BUFFER_STATE b ,yyscan_t yyscanner );
+void quickstep_yy_flush_buffer (YY_BUFFER_STATE b ,yyscan_t yyscanner );
+void quickstep_yypush_buffer_state (YY_BUFFER_STATE new_buffer ,yyscan_t yyscanner );
+void quickstep_yypop_buffer_state (yyscan_t yyscanner );
+
+static void quickstep_yyensure_buffer_stack (yyscan_t yyscanner );
+static void quickstep_yy_load_buffer_state (yyscan_t yyscanner );
+static void quickstep_yy_init_buffer (YY_BUFFER_STATE b,FILE *file ,yyscan_t yyscanner );
+
 #define YY_FLUSH_BUFFER quickstep_yy_flush_buffer(YY_CURRENT_BUFFER ,yyscanner)
 
-YY_BUFFER_STATE quickstep_yy_scan_buffer ( char *base, yy_size_t size , yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_scan_string ( const char *yy_str , yyscan_t yyscanner );
-YY_BUFFER_STATE quickstep_yy_scan_bytes ( const char *bytes, int len , yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_buffer (char *base,yy_size_t size ,yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_string (yyconst char *yy_str ,yyscan_t yyscanner );
+YY_BUFFER_STATE quickstep_yy_scan_bytes (yyconst char *bytes,yy_size_t len ,yyscan_t yyscanner );
 
-void *quickstep_yyalloc ( yy_size_t , yyscan_t yyscanner );
-void *quickstep_yyrealloc ( void *, yy_size_t , yyscan_t yyscanner );
-void quickstep_yyfree ( void * , yyscan_t yyscanner );
+void *quickstep_yyalloc (yy_size_t ,yyscan_t yyscanner );
+void *quickstep_yyrealloc (void *,yy_size_t ,yyscan_t yyscanner );
+void quickstep_yyfree (void * ,yyscan_t yyscanner );
 
 #define yy_new_buffer quickstep_yy_create_buffer
+
 #define yy_set_interactive(is_interactive) \
 	{ \
 	if ( ! YY_CURRENT_BUFFER ){ \
@@ -395,6 +339,7 @@ void quickstep_yyfree ( void * , yyscan_t yyscanner );
 	} \
 	YY_CURRENT_BUFFER_LVALUE->yy_is_interactive = is_interactive; \
 	}
+
 #define yy_set_bol(at_bol) \
 	{ \
 	if ( ! YY_CURRENT_BUFFER ){\
@@ -404,34 +349,40 @@ void quickstep_yyfree ( void * , yyscan_t yyscanner );
 	} \
 	YY_CURRENT_BUFFER_LVALUE->yy_at_bol = at_bol; \
 	}
+
 #define YY_AT_BOL() (YY_CURRENT_BUFFER_LVALUE->yy_at_bol)
 
 /* Begin user sect3 */
 
 #define quickstep_yywrap(yyscanner) (/*CONSTCOND*/1)
 #define YY_SKIP_YYWRAP
-typedef flex_uint8_t YY_CHAR;
+
+typedef unsigned char YY_CHAR;
 
 typedef int yy_state_type;
 
 #define yytext_ptr yytext_r
 
-static yy_state_type yy_get_previous_state ( yyscan_t yyscanner );
-static yy_state_type yy_try_NUL_trans ( yy_state_type current_state  , yyscan_t yyscanner);
-static int yy_get_next_buffer ( yyscan_t yyscanner );
-static void yynoreturn yy_fatal_error ( const char* msg , yyscan_t yyscanner );
+static yy_state_type yy_get_previous_state (yyscan_t yyscanner );
+static yy_state_type yy_try_NUL_trans (yy_state_type current_state  ,yyscan_t yyscanner);
+static int yy_get_next_buffer (yyscan_t yyscanner );
+#if defined(__GNUC__) && __GNUC__ >= 3
+__attribute__((__noreturn__))
+#endif
+static void yy_fatal_error (yyconst char msg[] ,yyscan_t yyscanner );
 
 /* Done after the current pattern has been matched and before the
  * corresponding action - sets up yytext.
  */
 #define YY_DO_BEFORE_ACTION \
 	yyg->yytext_ptr = yy_bp; \
-	yyleng = (int) (yy_cp - yy_bp); \
+	yyleng = (size_t) (yy_cp - yy_bp); \
 	yyg->yy_hold_char = *yy_cp; \
 	*yy_cp = '\0'; \
 	yyg->yy_c_buf_p = yy_cp;
-#define YY_NUM_RULES 161
-#define YY_END_OF_BUFFER 162
+
+#define YY_NUM_RULES 162
+#define YY_END_OF_BUFFER 163
 /* This struct is not used in this scanner,
    but its presence is necessary. */
 struct yy_trans_info
@@ -439,77 +390,77 @@ struct yy_trans_info
 	flex_int32_t yy_verify;
 	flex_int32_t yy_nxt;
 	};
-static const flex_int16_t yy_accept[593] =
+static yyconst flex_int16_t yy_accept[594] =
     {   0,
         0,    0,    0,    0,    0,    0,    0,    0,    0,    0,
-        0,    0,  162,    2,    2,  160,  160,  159,  158,  160,
-      137,  133,  136,  133,  133,  156,  129,  126,  130,  155,
-      155,  155,  155,  155,  155,  155,  155,  155,  155,  155,
-      155,  155,  155,  155,  155,  155,  155,  155,  155,  155,
-      155,  155,  155,  155,  134,    4,    5,    5,    3,  152,
-      152,  149,  153,  153,  147,  154,  154,  151,    1,  159,
-      127,  157,  156,  156,  156,    0,  131,  128,  132,  155,
-      155,  155,  155,   10,  155,  155,  155,   22,  155,  155,
-      155,  155,  155,  155,  155,  155,  155,  155,  155,  135,
-
-      155,  155,  155,  155,  155,  155,  155,  155,  155,  155,
-      155,  155,  155,   60,   68,  155,  155,  155,  155,  155,
-      155,  155,  155,  155,  155,  155,   82,   83,  155,  155,
-      155,  155,  155,  155,  155,  155,  155,  155,  155,  155,
-      155,  155,  155,  155,  155,  155,  155,  155,  155,  155,
-      155,  155,  155,    4,    5,    3,  152,  148,  153,  146,
-      146,  138,  140,  141,  142,  143,  144,  145,  146,  154,
-      150,  157,  156,    0,  156,    6,    7,  155,    9,   11,
-      155,  155,   15,  155,  155,  155,  155,  155,  155,  155,
-      155,  155,  155,  155,   33,  155,  155,  155,  155,  155,
-
-      155,  155,  155,   44,  155,  155,  155,  155,  155,  155,
-      155,   52,  155,  155,  155,  155,  155,  155,  155,  155,
-      155,   64,  155,   70,  155,  155,  155,  155,  155,  155,
-      155,   78,  155,   81,  155,  155,  155,  155,  155,  155,
-      155,  155,  155,  155,  155,  155,  155,   99,  155,  155,
-      104,  105,  155,  155,  155,  155,  155,  155,  155,  155,
-      155,  155,  155,  155,  155,  155,  155,  155,  138,  140,
-      139,  155,  155,  155,  155,  155,  155,  155,   20,   23,
-      155,  155,  155,   28,  155,  155,  155,   31,  155,  155,
-      155,  155,   38,  155,  155,   42,   43,  155,  155,  155,
-
-      155,  155,  155,  155,  155,   54,   55,  155,   57,  155,
-       59,  155,  155,  155,  155,   67,   69,   71,   72,   73,
-      155,   75,  155,  155,   79,  155,  155,   86,  155,  155,
-      155,  155,  155,   93,  155,   95,  155,  155,  155,  101,
-      155,  155,  155,  155,  155,  155,  109,  110,  112,  155,
-      155,  155,  155,  155,  155,  155,  120,  155,  155,  123,
-      124,  138,  139,    8,  155,  155,  155,  155,  155,  155,
-      155,   25,  155,  155,  155,  155,  155,  155,  155,  155,
-      155,  155,  155,  155,  155,  155,  155,  155,   48,   49,
-       50,  155,  155,   56,  155,   61,   62,  155,  155,  155,
-
-       74,  155,   77,   80,   84,   85,  155,  155,  155,  155,
-      155,   94,  155,  155,   98,  155,  155,  155,  155,  155,
-      108,  155,  155,  155,  155,  155,  117,  155,  155,  121,
-      155,  155,  155,  155,   14,  155,  155,  155,  155,  155,
-       26,  155,   29,  155,  155,  155,  155,  155,   36,  155,
-      155,  155,   41,  155,   46,  155,  155,  155,   58,   63,
-      155,  155,   76,  155,  155,  155,  155,  155,  155,   97,
-      155,  102,  103,  155,  155,  155,  155,  155,  115,  116,
-      118,  155,  122,  155,  155,   13,  155,  155,  155,  155,
-      155,  155,   21,   30,  155,   34,   35,  155,  155,  155,
-
-      155,   47,  155,   53,   65,  155,  155,   89,  155,   91,
-      155,  155,  155,  155,  155,  155,  155,  155,  119,  155,
-      155,  155,  155,  155,  155,  155,  155,   32,  155,  155,
-       40,  155,  155,   66,  155,  155,   92,  155,  155,  106,
-      155,  155,  155,  155,  155,   12,  155,  155,  155,  155,
-       24,  155,   37,  155,  155,   51,   87,   90,  155,  155,
-      107,  111,  155,  114,  125,   16,  155,  155,  155,   27,
-       39,  155,   88,   96,  155,  155,  155,   18,   19,  155,
-      155,  113,  155,  155,  155,  155,  155,  100,  155,   45,
-       17,    0
+        0,    0,  163,    2,    2,  161,  161,  160,  159,  161,
+      138,  134,  137,  134,  134,  157,  130,  127,  131,  156,
+      156,  156,  156,  156,  156,  156,  156,  156,  156,  156,
+      156,  156,  156,  156,  156,  156,  156,  156,  156,  156,
+      156,  156,  156,  156,  135,    4,    5,    5,    3,  153,
+      153,  150,  154,  154,  148,  155,  155,  152,    1,  160,
+      128,  158,  157,  157,  157,    0,  132,  129,  133,  156,
+      156,  156,  156,   10,  156,  156,  156,   23,  156,  156,
+      156,  156,  156,  156,  156,  156,  156,  156,  156,  136,
+
+      156,  156,  156,  156,  156,  156,  156,  156,  156,  156,
+      156,  156,  156,   61,   69,  156,  156,  156,  156,  156,
+      156,  156,  156,  156,  156,  156,   83,   84,  156,  156,
+      156,  156,  156,  156,  156,  156,  156,  156,  156,  156,
+      156,  156,  156,  156,  156,  156,  156,  156,  156,  156,
+      156,  156,  156,    4,    5,    3,  153,  149,  154,  147,
+      147,  139,  141,  142,  143,  144,  145,  146,  147,  155,
+      151,  158,  157,    0,  157,    6,    7,  156,    9,   11,
+      156,  156,   15,  156,  156,  156,  156,  156,  156,  156,
+      156,  156,  156,  156,   34,  156,  156,  156,  156,  156,
+
+      156,  156,  156,   45,  156,  156,  156,  156,  156,  156,
+      156,   53,  156,  156,  156,  156,  156,  156,  156,  156,
+      156,   65,  156,   71,  156,  156,  156,  156,  156,  156,
+      156,   79,  156,   82,  156,  156,  156,  156,  156,  156,
+      156,  156,  156,  156,  156,  156,  156,  100,  156,  156,
+      105,  106,  156,  156,  156,  156,  156,  156,  156,  156,
+      156,  156,  156,  156,  156,  156,  156,  156,  139,  141,
+      140,  156,  156,  156,  156,  156,  156,  156,   20,   21,
+       24,  156,  156,  156,   29,  156,  156,  156,   32,  156,
+      156,  156,  156,   39,  156,  156,   43,   44,  156,  156,
+
+      156,  156,  156,  156,  156,  156,   55,   56,  156,   58,
+      156,   60,  156,  156,  156,  156,   68,   70,   72,   73,
+       74,  156,   76,  156,  156,   80,  156,  156,   87,  156,
+      156,  156,  156,  156,   94,  156,   96,  156,  156,  156,
+      102,  156,  156,  156,  156,  156,  156,  110,  111,  113,
+      156,  156,  156,  156,  156,  156,  156,  121,  156,  156,
+      124,  125,  139,  140,    8,  156,  156,  156,  156,  156,
+      156,  156,   26,  156,  156,  156,  156,  156,  156,  156,
+      156,  156,  156,  156,  156,  156,  156,  156,  156,   49,
+       50,   51,  156,  156,   57,  156,   62,   63,  156,  156,
+
+      156,   75,  156,   78,   81,   85,   86,  156,  156,  156,
+      156,  156,   95,  156,  156,   99,  156,  156,  156,  156,
+      156,  109,  156,  156,  156,  156,  156,  118,  156,  156,
+      122,  156,  156,  156,  156,   14,  156,  156,  156,  156,
+      156,   27,  156,   30,  156,  156,  156,  156,  156,   37,
+      156,  156,  156,   42,  156,   47,  156,  156,  156,   59,
+       64,  156,  156,   77,  156,  156,  156,  156,  156,  156,
+       98,  156,  103,  104,  156,  156,  156,  156,  156,  116,
+      117,  119,  156,  123,  156,  156,   13,  156,  156,  156,
+      156,  156,  156,   22,   31,  156,   35,   36,  156,  156,
+
+      156,  156,   48,  156,   54,   66,  156,  156,   90,  156,
+       92,  156,  156,  156,  156,  156,  156,  156,  156,  120,
+      156,  156,  156,  156,  156,  156,  156,  156,   33,  156,
+      156,   41,  156,  156,   67,  156,  156,   93,  156,  156,
+      107,  156,  156,  156,  156,  156,   12,  156,  156,  156,
+      156,   25,  156,   38,  156,  156,   52,   88,   91,  156,
+      156,  108,  112,  156,  115,  126,   16,  156,  156,  156,
+       28,   40,  156,   89,   97,  156,  156,  156,   18,   19,
+      156,  156,  114,  156,  156,  156,  156,  156,  101,  156,
+       46,   17,    0
 
     } ;
 
-static const YY_CHAR yy_ec[256] =
+static yyconst YY_CHAR yy_ec[256] =
     {   0,
         1,    1,    1,    1,    1,    1,    1,    1,    2,    3,
         1,    1,    4,    1,    1,    1,    1,    1,    1,    1,
@@ -541,7 +492,7 @@ static const YY_CHAR yy_ec[256] =
         1,    1,    1,    1,    1
     } ;
 
-static const YY_CHAR yy_meta[72] =
+static yyconst YY_CHAR yy_meta[72] =
     {   0,
         1,    1,    2,    1,    1,    3,    1,    4,    1,    5,
         5,    6,    6,    5,    1,    1,    1,    7,    7,    7,
@@ -553,155 +504,155 @@ static const YY_CHAR yy_meta[72] =
         8
     } ;
 
-static const flex_int16_t yy_base[608] =
+static yyconst flex_uint16_t yy_base[609] =
     {   0,
         0,    1,   46,    0,  117,  163,    2,    3,  128,  132,
-        6,   10,  211, 1312, 1312,    0, 1312,   13, 1312,  194,
-     1312, 1312, 1312,  194,    6,  130,    4, 1312,  170,  124,
+        6,   10,  211, 1313, 1313,    0, 1313,   13, 1313,  194,
+     1313, 1313, 1313,  194,    6,  130,    4, 1313,  170,  124,
       161,  215,  170,  207,  265,   92,  167,  162,   96,  107,
       223,  165,  162,  221,  274,   92,  284,  277,  315,  206,
-      128,  232,    0,  125, 1312,  152,    4,   19,    0,    0,
+      128,  232,    0,  125, 1313,  152,    4,   19,    0,    0,
         0,  143,    0,    0,  379,    0,    0,  144,    0,   22,
-     1312,    0,  292,  305,  335,   18, 1312, 1312, 1312,    0,
+     1313,    0,  292,  305,  335,   18, 1313, 1313, 1313,    0,
       174,  264,  180,  186,  209,  271,  229,    0,  277,  333,
-      335,  312,  330,  317,  323,  338,  323,  333,  340, 1312,
+      335,  312,  330,  317,  323,  338,  323,  333,  340, 1313,
 
       339,  355,  357,  381,  370,  373,  377,  382,  380,  384,
       383,  383,  383,  431,    0,  398,  383,  390,  406,  402,
       403,  404,  425,  420,  431,  442,    0,  445,  432,  448,
       436,  437,  451,  448,  444,  460,  452,  439,  489,  464,
       468,  469,  468,  462,  455,  477,  501,  493,  489,  494,
-      502,  494,  510,  142,   29,    0,    0, 1312,    0, 1312,
-     1312,   22,   24, 1312, 1312, 1312, 1312, 1312,    0,    0,
-     1312,    0,  520,   26,   28,    0,    0,  508,    0,  512,
-      495,  510,  497,  530,  518,  506,  525,  508,  512,  509,
-      543,  530,  533,  550,    0,  547,  558,  556,  561,  546,
-
-      565,  552,  564,    0,  569,  552,  554,  556,  557,  576,
-      566,  576,  569,  571,  565,  579,  580,  573,  587,  597,
-      598,  604,  594,    0,  593,  594,  612,  613,  616,  603,
-      605,    0,  614,    0,  622,  623,  611,  610,  630,  631,
-      626,  615,  629,  628,  639,  640,  639,  631,  634,  656,
-        0,  654,  648,  659,  658,  670,  672,  668,  666,  665,
-      682,  670,  664,  683,  673,  684,  681,  674,   30,  125,
-        0,  675,  680,  692,  684,  694,  691,  690,    0,  704,
-      696,  695,  699,    0,  704,  707,  724,  710,  720,  714,
-      718,  726,  735,  732,  730,    0,    0,  727,  724,  744,
-
-      741,  727,  729,  735,  742,    0,    0,  736,    0,  741,
-        0,  732,  739,  742,  759,    0,    0,    0,    0,    0,
-      741,    0,  743,  757,  763,  765,  769,    0,  779,  786,
-      788,  795,  779,    0,  793,    0,  781,  776,  781,    0,
-      798,  789,  801,  793,  787,  803,    0,  790,    0,  805,
-      792,  793,  795,  811,  814,  813,    0,  818,  809,    0,
-      822,  136, 1312,    0,  836,  836,  825,  845,  835,  843,
-      852,    0,  842,  839,  853,  854,  846,  852,  861,  851,
-      860,  857,  853,  854,  866,  867,  854,  873,    0,    0,
-        0,  854,  872,    0,  874,    0,    0,  862,  888,  876,
-
-        0,  894,    0,    0,    0,    0,  883,  890,  903,  891,
-      903,    0,  908,  898,    0,  910,  912,  897,  909,  901,
-        0,  900,  902,  909,  919,  920,    0,  907,  928,    0,
-      907,  916,  925,  921,    0,  915,  921,  939,  942,  936,
-        0,  956,    0,  956,  942,  952,  955,  950,    0,  951,
-      968,  970,    0,   93,    0,  954,  966,  962,    0,    0,
-      959,  977,    0,  970,  961,  973,  958,  964,  973,    0,
-      976,    0,    0,  975,  983,  992,  994,  993,    0,    0,
-        0,  980,    0,  995, 1000,    0, 1006, 1004, 1007, 1011,
-     1020, 1019,    0,    0, 1024,    0,    0, 1025, 1022, 1012,
-
-     1014,    0, 1020,    0,    0, 1023, 1021,    0, 1023,    0,
-     1014, 1037, 1032, 1022, 1031, 1033, 1034, 1045,    0, 1031,
-     1046, 1041, 1040, 1051, 1052, 1055, 1063,    0, 1060, 1067,
-        0, 1063, 1079,    0, 1073, 1081,    0, 1084, 1077,    0,
-     1084, 1076, 1077, 1090, 1087,    0, 1090, 1093, 1087, 1095,
-        0, 1083,    0, 1097, 1087,    0, 1089,    0, 1090, 1102,
-        0,    0, 1101,    0,    0,    0, 1096, 1120, 1112,    0,
-        0, 1122,    0,    0, 1115, 1131, 1119,    0,    0, 1127,
-     1139,    0, 1136, 1139, 1129, 1143, 1130,    0, 1131,    0,
-        0, 1312, 1196, 1206, 1216, 1226, 1236, 1240, 1243, 1249,
-
-     1257, 1267, 1277, 1287, 1297, 1302, 1304
+      502,  494,  510,  142,   29,    0,    0, 1313,    0, 1313,
+     1313,   22,   24, 1313, 1313, 1313, 1313, 1313,    0,    0,
+     1313,    0,  520,   26,   28,    0,    0,  508,    0,  512,
+      495,  510,  497,  530,  524,  505,  521,  507,  512,  509,
+      549,  531,  537,  551,    0,  550,  560,  559,  562,  546,
+
+      565,  553,  565,    0,  570,  553,  557,  557,  558,  577,
+      569,  577,  570,  574,  566,  581,  582,  574,  597,  598,
+      603,  604,  599,    0,  594,  597,  617,  614,  617,  604,
+      606,    0,  615,    0,  623,  624,  612,  613,  631,  632,
+      626,  616,  633,  632,  640,  642,  641,  631,  646,  661,
+        0,  655,  652,  660,  661,  672,  675,  669,  667,  666,
+      683,  671,  665,  684,  674,  685,  683,  675,   30,  125,
+        0,  676,  683,  693,  685,  697,  692,  692,    0,    0,
+      706,  697,  706,  704,    0,  705,  711,  725,  713,  722,
+      717,  719,  727,  736,  733,  731,    0,    0,  728,  725,
+
+      745,  742,  729,  730,  736,  743,    0,    0,  739,    0,
+      742,    0,  733,  742,  743,  759,    0,    0,    0,    0,
+        0,  743,    0,  745,  774,  764,  769,  770,    0,  782,
+      788,  791,  796,  780,    0,  794,    0,  782,  777,  782,
+        0,  799,  790,  802,  795,  788,  804,    0,  791,    0,
+      808,  793,  794,  798,  812,  816,  815,    0,  819,  820,
+        0,  827,  136, 1313,    0,  837,  840,  826,  848,  835,
+      849,  853,    0,  843,  840,  854,  855,  847,  853,  862,
+      853,  861,  858,  854,  857,  867,  868,  857,  874,    0,
+        0,    0,  856,  874,    0,  875,    0,    0,  873,  893,
+
+      877,    0,  898,    0,    0,    0,    0,  884,  893,  905,
+      894,  904,    0,  909,  899,    0,  911,  913,  898,  910,
+      902,    0,  901,  904,  910,  920,  921,    0,  910,  929,
+        0,  908,  919,  926,  923,    0,  917,  922,  950,  947,
+      937,    0,  960,    0,  957,  945,  954,  958,  951,    0,
+      952,  969,  971,    0,   93,    0,  955,  967,  963,    0,
+        0,  960,  978,    0,  972,  962,  974,  959,  967,  974,
+        0,  977,    0,    0,  978,  984,  994,  996,  994,    0,
+        0,    0,  991,    0, 1000, 1001,    0, 1010, 1005, 1010,
+     1013, 1023, 1020,    0,    0, 1025,    0,    0, 1026, 1023,
+
+     1013, 1015,    0, 1021,    0,    0, 1024, 1022,    0, 1025,
+        0, 1015, 1038, 1033, 1025, 1032, 1034, 1037, 1046,    0,
+     1033, 1048, 1042, 1051, 1056, 1053, 1059, 1064,    0, 1063,
+     1069,    0, 1066, 1080,    0, 1074, 1082,    0, 1085, 1078,
+        0, 1085, 1077, 1078, 1091, 1089,    0, 1091, 1094, 1088,
+     1098,    0, 1084,    0, 1098, 1090,    0, 1090,    0, 1092,
+     1104,    0,    0, 1102,    0,    0,    0, 1107, 1125, 1113,
+        0,    0, 1126,    0,    0, 1116, 1134, 1121,    0,    0,
+     1130, 1140,    0, 1137, 1140, 1130, 1144, 1131,    0, 1132,
+        0,    0, 1313, 1197, 1207, 1217, 1227, 1237, 1241, 1244,
+
+     1250, 1258, 1268, 1278, 1288, 1298, 1303, 1305
     } ;
 
-static const flex_int16_t yy_def[608] =
+static yyconst flex_int16_t yy_def[609] =
     {   0,
-      593,  593,  592,    3,  594,  594,  595,  595,  596,  596,
-      597,  597,  592,  592,  592,  598,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  592,  592,  592,  592,  600,  601,
-      601,  592,  602,  602,  603,  604,  604,  592,  598,  592,
-      592,  605,  592,  592,  592,  592,  592,  592,  592,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  592,
-
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  592,  592,  600,  601,  592,  602,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  606,  604,
-      592,  605,  592,  592,  592,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  592,  592,
-      607,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  592,  592,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,  599,  599,  599,  599,  599,  599,  599,  599,  599,
-      599,    0,  592,  592,  592,  592,  592,  592,  592,  592,
-
-      592,  592,  592,  592,  592,  592,  592
+      594,  594,  593,    3,  595,  595,  596,  596,  597,  597,
+      598,  598,  593,  593,  593,  599,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  593,  593,  593,  593,  601,  602,
+      602,  593,  603,  603,  604,  605,  605,  593,  599,  593,
+      593,  606,  593,  593,  593,  593,  593,  593,  593,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  593,
+
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  593,  593,  601,  602,  593,  603,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  607,  605,
+      593,  606,  593,  593,  593,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  593,  593,
+      608,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  593,  593,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,  600,  600,  600,  600,  600,  600,  600,  600,
+      600,  600,    0,  593,  593,  593,  593,  593,  593,  593,
+
+      593,  593,  593,  593,  593,  593,  593,  593
     } ;
 
-static const flex_int16_t yy_nxt[1384] =
+static yyconst flex_uint16_t yy_nxt[1385] =
     {   0,
-      592,  592,   15,   15,   61,   61,  155,  155,   67,   62,
-       62,   68,   67,  592,   70,   68,   70,   73,   73,   77,
-       78,  155,  155,   70,  592,   70,  174,  174,  592,  175,
+      593,  593,   15,   15,   61,   61,  155,  155,   67,   62,
+       62,   68,   67,  593,   70,   68,   70,   73,   73,   77,
+       78,  155,  155,   70,  593,   70,  174,  174,  593,  175,
       175,  155,  155,  269,  270,  270,  270,  175,  175,  175,
-      175,  362,  270,  592,   16,   16,   17,   18,   19,   18,
+      175,  363,  270,  593,   16,   16,   17,   18,   19,   18,
        20,   21,   22,   23,   22,   24,   25,   26,   26,   17,
        27,   28,   29,   30,   31,   32,   33,   34,   35,   36,
        37,   38,   39,   40,   41,   42,   43,   44,   45,   46,
@@ -711,7 +662,7 @@ static const flex_int16_t yy_nxt[1384] =
        38,   39,   40,   41,   42,   43,   44,   45,   46,   47,
        48,   49,   50,   51,   52,   53,   54,   17,   56,   57,
        58,   17,   17,   17,   17,   17,  111,  116,  117,  134,
-       64,   17,   17,   17,   64,   62,  270,  270,  501,   62,
+       64,   17,   17,   17,   64,   62,  270,  270,  502,   62,
        74,   75,   75,  154,   81,  150,  153,  270,  270,  171,
       158,   76,   82,  154,   83,  111,  116,  117,  134,   84,
        17,   17,   17,   17,   56,   57,   58,   17,   17,   17,
@@ -720,81 +671,81 @@ static const flex_int16_t yy_nxt[1384] =
       122,   96,  114,  124,  176,   97,  123,  115,  113,  125,
 
       179,   98,   88,   72,   99,  180,   17,   17,   17,   71,
-      592,   85,  592,  112,  100,   86,   95,  592,   87,  122,
+      593,   85,  593,  112,  100,   86,   95,  593,   87,  122,
        96,  114,  124,  176,   97,  123,  115,  113,  125,  179,
        98,   88,   89,   99,  180,  101,  147,  102,  148,   90,
       118,  149,  103,  126,  119,  181,   91,  104,  120,   92,
-       93,  127,   94,  592,  121,  128,  151,  152,  129,  130,
-      184,   89,  592,  592,  101,  147,  102,  148,   90,  118,
+       93,  127,   94,  593,  121,  128,  151,  152,  129,  130,
+      184,   89,  593,  593,  101,  147,  102,  148,   90,  118,
       149,  103,  126,  119,  181,   91,  104,  120,   92,   93,
       127,   94,  105,  121,  128,  151,  152,  129,  130,  184,
-      106,  131,  177,  107,  182,  132,  108,  592,  139,  109,
+      106,  131,  177,  107,  182,  132,  108,  593,  139,  109,
 
       178,  135,  110,   73,   73,  136,  140,  183,  133,  137,
-      592,  105,  185,   76,  141,  138,  173,  173,  592,  106,
+      593,  105,  185,   76,  141,  138,  173,  173,  593,  106,
       131,  177,  107,  182,  132,  108,   76,  139,  109,  178,
       135,  110,  142,  191,  136,  140,  183,  133,  137,  143,
       144,  185,   76,  141,  138,   74,   75,   75,  192,  145,
-      186,  193,  146,  592,  187,   76,   76,  196,  200,  194,
+      186,  193,  146,  593,  187,   76,   76,  196,  200,  194,
       197,  142,  191,  188,  195,  189,  198,  190,  143,  144,
-      201,  202,  592,  199,  203,  204,  205,  192,  145,  186,
-      193,  146,  161,  187,  592,   76,  196,  200,  194,  197,
+      201,  202,  593,  199,  203,  204,  205,  192,  145,  186,
+      193,  146,  161,  187,  593,   76,  196,  200,  194,  197,
       162,  163,  188,  195,  189,  198,  190,  164,  208,  201,
 
       202,  165,  199,  203,  204,  205,  206,  209,  210,  166,
       211,  213,  214,  167,  215,  168,  212,  207,  216,  169,
-      218,  217,  592,  223,  224,  225,  164,  208,  226,  227,
-      165,  228,  592,  229,  230,  206,  209,  210,  166,  211,
+      218,  217,  593,  223,  224,  225,  164,  208,  226,  227,
+      165,  228,  593,  229,  230,  206,  209,  210,  166,  211,
       213,  214,  167,  215,  168,  212,  207,  216,  169,  218,
       217,  219,  223,  224,  225,  231,  232,  226,  227,  233,
       228,  220,  229,  230,  234,  235,  221,  222,  236,  237,
-      238,  239,  240,  242,  243,  247,  241,  244,  248,  592,
+      238,  239,  240,  242,  243,  247,  241,  244,  248,  593,
       219,  252,  245,  246,  231,  232,  253,  254,  233,  255,
       220,  256,  257,  234,  235,  221,  222,  236,  237,  238,
 
       239,  240,  242,  243,  247,  241,  244,  248,  249,  258,
       252,  245,  246,  261,  262,  253,  254,  250,  255,  259,
       256,  257,  263,  265,  266,  251,  260,  268,  264,  272,
-      267,  173,  173,  273,  274,  275,  276,  249,  258,  279,
-      280,   76,  261,  262,  281,  282,  250,  283,  259,  277,
-      284,  263,  265,  266,  251,  260,  268,  264,  272,  267,
-      285,  278,  273,  274,  275,  276,  286,  287,  279,  280,
-       76,  288,  289,  281,  282,  290,  283,  291,  277,  284,
-      293,  292,  294,  295,  296,  297,  298,  299,  300,  285,
+      267,  173,  173,  273,  274,  275,  276,  249,  258,  281,
+      282,   76,  261,  262,  283,  279,  250,  284,  259,  277,
+      285,  263,  265,  266,  251,  260,  268,  264,  272,  267,
+      280,  278,  273,  274,  275,  276,  286,  287,  281,  282,
+       76,  288,  289,  283,  279,  290,  284,  291,  277,  285,
+      292,  294,  295,  296,  293,  297,  298,  299,  300,  280,
       278,  301,  302,  303,  304,  286,  287,  305,  306,  307,
 
-      288,  289,  308,  309,  290,  310,  291,  311,  312,  293,
-      292,  294,  295,  296,  297,  298,  299,  300,  313,  314,
-      301,  302,  303,  304,  317,  315,  305,  306,  307,  318,
-      319,  308,  309,  320,  310,  316,  311,  312,  321,  322,
+      288,  289,  308,  309,  290,  310,  291,  311,  312,  292,
+      294,  295,  296,  293,  297,  298,  299,  300,  313,  314,
+      301,  302,  303,  304,  315,  316,  305,  306,  307,  318,
+      319,  308,  309,  320,  310,  317,  311,  312,  321,  322,
       323,  324,  325,  326,  327,  328,  329,  313,  314,  330,
-      331,  334,  335,  317,  315,  332,  336,  333,  318,  319,
-      337,  338,  320,  339,  316,  342,  340,  321,  322,  323,
-      324,  325,  326,  327,  328,  329,  341,  343,  330,  331,
-      334,  335,  344,  345,  332,  336,  333,  346,  347,  337,
-      338,  348,  339,  349,  342,  340,  350,  351,  352,  353,
+      331,  332,  335,  315,  316,  333,  336,  334,  318,  319,
+      337,  338,  320,  339,  317,  340,  341,  321,  322,  323,
+      324,  325,  326,  327,  328,  329,  342,  343,  330,  331,
+      332,  335,  344,  345,  333,  336,  334,  346,  347,  337,
+      338,  348,  339,  349,  340,  341,  350,  351,  352,  353,
 
-      354,  355,  356,  357,  359,  360,  343,  358,  361,  364,
+      354,  355,  356,  357,  358,  360,  343,  361,  359,  362,
       365,  344,  345,  366,  367,  368,  346,  347,  369,  370,
       348,  371,  349,  372,  373,  350,  351,  352,  353,  354,
-      355,  356,  357,  359,  360,  374,  358,  361,  364,  365,
+      355,  356,  357,  358,  360,  374,  361,  359,  362,  365,
       375,  376,  366,  367,  368,  377,  378,  369,  370,  379,
       371,  380,  372,  373,  381,  382,  383,  384,  385,  386,
       387,  388,  389,  390,  374,  391,  392,  393,  394,  375,
-      376,  395,  396,  397,  377,  378,  398,  401,  379,  402,
-      380,  403,  399,  381,  382,  383,  384,  385,  386,  387,
-      388,  389,  390,  400,  391,  392,  393,  394,  404,  405,
+      376,  395,  396,  397,  377,  378,  398,  399,  379,  402,
+      380,  403,  400,  381,  382,  383,  384,  385,  386,  387,
+      388,  389,  390,  401,  391,  392,  393,  394,  404,  405,
 
-      395,  396,  397,  406,  407,  398,  401,  408,  402,  409,
-      403,  399,  410,  411,  412,  413,  414,  415,  416,  417,
-      418,  419,  400,  420,  421,  422,  423,  404,  405,  424,
+      395,  396,  397,  406,  407,  398,  399,  408,  402,  409,
+      403,  400,  410,  411,  412,  413,  414,  415,  416,  417,
+      418,  419,  401,  420,  421,  422,  423,  404,  405,  424,
       425,  426,  406,  407,  427,  428,  408,  429,  409,  430,
       431,  410,  411,  412,  413,  414,  415,  416,  417,  418,
       419,  432,  420,  421,  422,  423,  433,  434,  424,  425,
-      426,  435,  436,  427,  428,  439,  429,  437,  430,  431,
-      438,  440,  441,  442,  443,  444,  445,  446,  447,  448,
+      426,  435,  436,  427,  428,  437,  429,  438,  430,  431,
+      439,  440,  441,  442,  443,  444,  445,  446,  447,  448,
       432,  449,  450,  451,  452,  433,  434,  453,  454,  455,
-      435,  436,  456,  457,  439,  458,  437,  459,  460,  438,
+      435,  436,  456,  457,  437,  458,  438,  459,  460,  439,
 
       440,  441,  442,  443,  444,  445,  446,  447,  448,  461,
       449,  450,  451,  452,  462,  463,  453,  454,  455,  464,
@@ -805,10 +756,10 @@ static const flex_int16_t yy_nxt[1384] =
       471,  472,  473,  474,  475,  476,  477,  490,  478,  479,
       480,  481,  491,  492,  482,  483,  484,  493,  494,  485,
       486,  495,  487,  496,  488,  489,  497,  498,  499,  500,
-      502,  503,  504,  505,  506,  507,  490,  508,  509,  510,
+      501,  503,  504,  505,  506,  507,  490,  508,  509,  510,
 
       511,  491,  492,  512,  513,  514,  493,  494,  515,  516,
-      495,  517,  496,  518,  519,  497,  498,  499,  500,  502,
+      495,  517,  496,  518,  519,  497,  498,  499,  500,  501,
       503,  504,  505,  506,  507,  520,  508,  509,  510,  511,
       521,  522,  512,  513,  514,  523,  524,  515,  516,  525,
       517,  526,  518,  519,  527,  528,  529,  530,  531,  532,
@@ -824,34 +775,34 @@ static const flex_int16_t yy_nxt[1384] =
       577,  556,  557,  558,  559,  560,  561,  562,  563,  564,
       565,  578,  566,  567,  568,  569,  579,  580,  570,  571,
       572,  581,  582,  573,  574,  583,  575,  584,  576,  577,
-      585,  586,  587,  588,  589,  590,  591,  592,  592,  592,
-      578,  592,  592,  592,  592,  579,  580,  592,  592,  592,
-      581,  582,  592,  592,  583,  592,  584,  592,  592,  585,
-      586,  587,  588,  589,  590,  591,   14,   14,   14,   14,
-
-       14,   14,   14,   14,   14,   14,   59,   59,   59,   59,
-       59,   59,   59,   59,   59,   59,   60,   60,   60,   60,
-       60,   60,   60,   60,   60,   60,   63,   63,   63,   63,
-       63,   63,   63,   63,   63,   63,   66,   66,   66,   66,
-       66,   66,   66,   66,   66,   66,   69,   69,   80,   80,
-       80,  592,   80,  156,  156,  156,  156,  157,  157,  157,
-      592,  157,  157,  157,  157,  157,  157,  159,  159,  159,
-      592,  159,  159,  159,  159,  592,  159,  160,  160,  160,
-      160,  160,  160,  160,  160,  160,  160,  170,  170,  592,
-      170,  170,  170,  170,  170,  170,  170,  172,  592,  172,
-
-      172,  172,  172,  172,  172,  172,  172,  271,  271,  363,
-      363,   13,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592
+      585,  586,  587,  588,  589,  590,  591,  592,  593,  593,
+      578,  593,  593,  593,  593,  579,  580,  593,  593,  593,
+      581,  582,  593,  593,  583,  593,  584,  593,  593,  585,
+      586,  587,  588,  589,  590,  591,  592,   14,   14,   14,
+
+       14,   14,   14,   14,   14,   14,   14,   59,   59,   59,
+       59,   59,   59,   59,   59,   59,   59,   60,   60,   60,
+       60,   60,   60,   60,   60,   60,   60,   63,   63,   63,
+       63,   63,   63,   63,   63,   63,   63,   66,   66,   66,
+       66,   66,   66,   66,   66,   66,   66,   69,   69,   80,
+       80,   80,  593,   80,  156,  156,  156,  156,  157,  157,
+      157,  593,  157,  157,  157,  157,  157,  157,  159,  159,
+      159,  593,  159,  159,  159,  159,  593,  159,  160,  160,
+      160,  160,  160,  160,  160,  160,  160,  160,  170,  170,
+      593,  170,  170,  170,  170,  170,  170,  170,  172,  593,
+
+      172,  172,  172,  172,  172,  172,  172,  172,  271,  271,
+      364,  364,   13,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593
     } ;
 
-static const flex_int16_t yy_chk[1384] =
+static yyconst flex_int16_t yy_chk[1385] =
     {   0,
         0,    0,    1,    2,    7,    8,   57,   57,   11,    7,
         8,   11,   12,    0,   18,   12,   18,   25,   25,   27,
@@ -867,8 +818,8 @@ static const flex_int16_t yy_chk[1384] =
         3,    3,    3,    3,    3,    3,    3,    3,    3,    3,
         3,    3,    3,    3,    3,    3,    3,    5,    5,    5,
         5,    5,    5,    5,    5,    5,   36,   39,   40,   46,
-        9,    5,    5,    5,   10,    9,  270,  270,  454,   10,
-       26,   26,   26,  154,   30,   51,   54,  362,  362,   68,
+        9,    5,    5,    5,   10,    9,  270,  270,  455,   10,
+       26,   26,   26,  154,   30,   51,   54,  363,  363,   68,
        62,   26,   30,   56,   30,   36,   39,   40,   46,   30,
         5,    5,    5,    6,    6,    6,    6,    6,    6,    6,
         6,    6,    9,   30,   51,   54,   10,    6,    6,    6,
@@ -911,104 +862,104 @@ static const flex_int16_t yy_chk[1384] =
       132,  133,  134,  135,  137,  133,  136,  138,  139,  146,
       140,  136,  136,  148,  149,  141,  142,  139,  143,  147,
       144,  145,  150,  151,  152,  139,  147,  153,  150,  178,
-      152,  173,  173,  180,  181,  182,  183,  139,  146,  185,
-      186,  173,  148,  149,  187,  188,  139,  189,  147,  184,
+      152,  173,  173,  180,  181,  182,  183,  139,  146,  186,
+      187,  173,  148,  149,  188,  185,  139,  189,  147,  184,
       190,  150,  151,  152,  139,  147,  153,  150,  178,  152,
-      191,  184,  180,  181,  182,  183,  192,  193,  185,  186,
-      173,  194,  196,  187,  188,  197,  189,  198,  184,  190,
-      199,  198,  200,  201,  202,  203,  205,  206,  207,  191,
-      184,  208,  209,  210,  211,  192,  193,  212,  213,  214,
-
-      194,  196,  215,  216,  197,  217,  198,  218,  219,  199,
-      198,  200,  201,  202,  203,  205,  206,  207,  220,  221,
-      208,  209,  210,  211,  223,  222,  212,  213,  214,  225,
-      226,  215,  216,  227,  217,  222,  218,  219,  228,  229,
-      230,  231,  233,  235,  236,  237,  238,  220,  221,  239,
-      240,  242,  243,  223,  222,  241,  244,  241,  225,  226,
-      245,  246,  227,  247,  222,  249,  248,  228,  229,  230,
-      231,  233,  235,  236,  237,  238,  248,  250,  239,  240,
-      242,  243,  252,  253,  241,  244,  241,  254,  255,  245,
-      246,  256,  247,  257,  249,  248,  258,  259,  260,  261,
-
-      262,  263,  264,  265,  266,  267,  250,  265,  268,  272,
-      273,  252,  253,  274,  275,  276,  254,  255,  277,  278,
-      256,  280,  257,  281,  282,  258,  259,  260,  261,  262,
-      263,  264,  265,  266,  267,  283,  265,  268,  272,  273,
-      285,  286,  274,  275,  276,  287,  288,  277,  278,  289,
-      280,  290,  281,  282,  291,  292,  293,  294,  295,  298,
-      299,  300,  301,  302,  283,  303,  304,  305,  308,  285,
-      286,  310,  312,  313,  287,  288,  314,  321,  289,  323,
-      290,  324,  315,  291,  292,  293,  294,  295,  298,  299,
-      300,  301,  302,  315,  303,  304,  305,  308,  325,  326,
-
-      310,  312,  313,  327,  329,  314,  321,  330,  323,  331,
-      324,  315,  332,  333,  335,  337,  338,  339,  341,  342,
-      343,  344,  315,  345,  346,  348,  350,  325,  326,  351,
-      352,  353,  327,  329,  354,  355,  330,  356,  331,  358,
-      359,  332,  333,  335,  337,  338,  339,  341,  342,  343,
-      344,  361,  345,  346,  348,  350,  365,  366,  351,  352,
-      353,  367,  368,  354,  355,  370,  356,  369,  358,  359,
-      369,  371,  373,  374,  375,  376,  377,  378,  379,  380,
-      361,  381,  382,  383,  384,  365,  366,  385,  386,  387,
-      367,  368,  388,  392,  370,  393,  369,  395,  398,  369,
-
-      371,  373,  374,  375,  376,  377,  378,  379,  380,  399,
-      381,  382,  383,  384,  400,  402,  385,  386,  387,  407,
-      408,  388,  392,  409,  393,  410,  395,  398,  411,  413,
-      414,  416,  417,  418,  419,  420,  422,  423,  399,  424,
-      425,  426,  428,  400,  402,  429,  431,  432,  407,  408,
-      433,  434,  409,  436,  410,  437,  438,  411,  413,  414,
-      416,  417,  418,  419,  420,  422,  423,  439,  424,  425,
-      426,  428,  440,  442,  429,  431,  432,  444,  445,  433,
-      434,  446,  436,  447,  437,  438,  448,  450,  451,  452,
-      456,  457,  458,  461,  462,  464,  439,  465,  466,  467,
-
-      468,  440,  442,  469,  471,  474,  444,  445,  475,  476,
-      446,  477,  447,  478,  482,  448,  450,  451,  452,  456,
-      457,  458,  461,  462,  464,  484,  465,  466,  467,  468,
-      485,  487,  469,  471,  474,  488,  489,  475,  476,  490,
-      477,  491,  478,  482,  492,  495,  498,  499,  500,  501,
-      503,  506,  507,  509,  484,  511,  512,  513,  514,  485,
-      487,  515,  516,  517,  488,  489,  518,  520,  490,  521,
-      491,  522,  523,  492,  495,  498,  499,  500,  501,  503,
-      506,  507,  509,  524,  511,  512,  513,  514,  525,  526,
-      515,  516,  517,  527,  529,  518,  520,  530,  521,  532,
-
-      522,  523,  533,  535,  536,  538,  539,  541,  542,  543,
-      544,  545,  524,  547,  548,  549,  550,  525,  526,  552,
-      554,  555,  527,  529,  557,  559,  530,  560,  532,  563,
-      567,  533,  535,  536,  538,  539,  541,  542,  543,  544,
-      545,  568,  547,  548,  549,  550,  569,  572,  552,  554,
-      555,  575,  576,  557,  559,  577,  560,  580,  563,  567,
-      581,  583,  584,  585,  586,  587,  589,    0,    0,    0,
-      568,    0,    0,    0,    0,  569,  572,    0,    0,    0,
-      575,  576,    0,    0,  577,    0,  580,    0,    0,  581,
-      583,  584,  585,  586,  587,  589,  593,  593,  593,  593,
-
-      593,  593,  593,  593,  593,  593,  594,  594,  594,  594,
-      594,  594,  594,  594,  594,  594,  595,  595,  595,  595,
-      595,  595,  595,  595,  595,  595,  596,  596,  596,  596,
-      596,  596,  596,  596,  596,  596,  597,  597,  597,  597,
-      597,  597,  597,  597,  597,  597,  598,  598,  599,  599,
-      599,    0,  599,  600,  600,  600,  600,  601,  601,  601,
-        0,  601,  601,  601,  601,  601,  601,  602,  602,  602,
-        0,  602,  602,  602,  602,    0,  602,  603,  603,  603,
-      603,  603,  603,  603,  603,  603,  603,  604,  604,    0,
-      604,  604,  604,  604,  604,  604,  604,  605,    0,  605,
-
-      605,  605,  605,  605,  605,  605,  605,  606,  606,  607,
-      607,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592,  592,  592,  592,  592,  592,  592,  592,
-      592,  592,  592
+      185,  184,  180,  181,  182,  183,  191,  192,  186,  187,
+      173,  193,  194,  188,  185,  196,  189,  197,  184,  190,
+      198,  199,  200,  201,  198,  202,  203,  205,  206,  185,
+      184,  207,  208,  209,  210,  191,  192,  211,  212,  213,
+
+      193,  194,  214,  215,  196,  216,  197,  217,  218,  198,
+      199,  200,  201,  198,  202,  203,  205,  206,  219,  220,
+      207,  208,  209,  210,  221,  222,  211,  212,  213,  223,
+      225,  214,  215,  226,  216,  222,  217,  218,  227,  228,
+      229,  230,  231,  233,  235,  236,  237,  219,  220,  238,
+      239,  240,  242,  221,  222,  241,  243,  241,  223,  225,
+      244,  245,  226,  246,  222,  247,  248,  227,  228,  229,
+      230,  231,  233,  235,  236,  237,  248,  249,  238,  239,
+      240,  242,  250,  252,  241,  243,  241,  253,  254,  244,
+      245,  255,  246,  256,  247,  248,  257,  258,  259,  260,
+
+      261,  262,  263,  264,  265,  266,  249,  267,  265,  268,
+      272,  250,  252,  273,  274,  275,  253,  254,  276,  277,
+      255,  278,  256,  281,  282,  257,  258,  259,  260,  261,
+      262,  263,  264,  265,  266,  283,  267,  265,  268,  272,
+      284,  286,  273,  274,  275,  287,  288,  276,  277,  289,
+      278,  290,  281,  282,  291,  292,  293,  294,  295,  296,
+      299,  300,  301,  302,  283,  303,  304,  305,  306,  284,
+      286,  309,  311,  313,  287,  288,  314,  315,  289,  322,
+      290,  324,  316,  291,  292,  293,  294,  295,  296,  299,
+      300,  301,  302,  316,  303,  304,  305,  306,  325,  326,
+
+      309,  311,  313,  327,  328,  314,  315,  330,  322,  331,
+      324,  316,  332,  333,  334,  336,  338,  339,  340,  342,
+      343,  344,  316,  345,  346,  347,  349,  325,  326,  351,
+      352,  353,  327,  328,  354,  355,  330,  356,  331,  357,
+      359,  332,  333,  334,  336,  338,  339,  340,  342,  343,
+      344,  360,  345,  346,  347,  349,  362,  366,  351,  352,
+      353,  367,  368,  354,  355,  369,  356,  370,  357,  359,
+      370,  371,  372,  374,  375,  376,  377,  378,  379,  380,
+      360,  381,  382,  383,  384,  362,  366,  385,  386,  387,
+      367,  368,  388,  389,  369,  393,  370,  394,  396,  370,
+
+      371,  372,  374,  375,  376,  377,  378,  379,  380,  399,
+      381,  382,  383,  384,  400,  401,  385,  386,  387,  403,
+      408,  388,  389,  409,  393,  410,  394,  396,  411,  412,
+      414,  415,  417,  418,  419,  420,  421,  423,  399,  424,
+      425,  426,  427,  400,  401,  429,  430,  432,  403,  408,
+      433,  434,  409,  435,  410,  437,  438,  411,  412,  414,
+      415,  417,  418,  419,  420,  421,  423,  439,  424,  425,
+      426,  427,  440,  441,  429,  430,  432,  443,  445,  433,
+      434,  446,  435,  447,  437,  438,  448,  449,  451,  452,
+      453,  457,  458,  459,  462,  463,  439,  465,  466,  467,
+
+      468,  440,  441,  469,  470,  472,  443,  445,  475,  476,
+      446,  477,  447,  478,  479,  448,  449,  451,  452,  453,
+      457,  458,  459,  462,  463,  483,  465,  466,  467,  468,
+      485,  486,  469,  470,  472,  488,  489,  475,  476,  490,
+      477,  491,  478,  479,  492,  493,  496,  499,  500,  501,
+      502,  504,  507,  508,  483,  510,  512,  513,  514,  485,
+      486,  515,  516,  517,  488,  489,  518,  519,  490,  521,
+      491,  522,  523,  492,  493,  496,  499,  500,  501,  502,
+      504,  507,  508,  524,  510,  512,  513,  514,  525,  526,
+      515,  516,  517,  527,  528,  518,  519,  530,  521,  531,
+
+      522,  523,  533,  534,  536,  537,  539,  540,  542,  543,
+      544,  545,  524,  546,  548,  549,  550,  525,  526,  551,
+      553,  555,  527,  528,  556,  558,  530,  560,  531,  561,
+      564,  533,  534,  536,  537,  539,  540,  542,  543,  544,
+      545,  568,  546,  548,  549,  550,  569,  570,  551,  553,
+      555,  573,  576,  556,  558,  577,  560,  578,  561,  564,
+      581,  582,  584,  585,  586,  587,  588,  590,    0,    0,
+      568,    0,    0,    0,    0,  569,  570,    0,    0,    0,
+      573,  576,    0,    0,  577,    0,  578,    0,    0,  581,
+      582,  584,  585,  586,  587,  588,  590,  594,  594,  594,
+
+      594,  594,  594,  594,  594,  594,  594,  595,  595,  595,
+      595,  595,  595,  595,  595,  595,  595,  596,  596,  596,
+      596,  596,  596,  596,  596,  596,  596,  597,  597,  597,
+      597,  597,  597,  597,  597,  597,  597,  598,  598,  598,
+      598,  598,  598,  598,  598,  598,  598,  599,  599,  600,
+      600,  600,    0,  600,  601,  601,  601,  601,  602,  602,
+      602,    0,  602,  602,  602,  602,  602,  602,  603,  603,
+      603,    0,  603,  603,  603,  603,    0,  603,  604,  604,
+      604,  604,  604,  604,  604,  604,  604,  604,  605,  605,
+        0,  605,  605,  605,  605,  605,  605,  605,  606,    0,
+
+      606,  606,  606,  606,  606,  606,  606,  606,  607,  607,
+      608,  608,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593,  593,  593,  593,  593,  593,  593,
+      593,  593,  593,  593
     } ;
 
 /* Table of booleans, true if rule could match eol. */
-static const flex_int32_t yy_rule_can_match_eol[162] =
+static yyconst flex_int32_t yy_rule_can_match_eol[163] =
     {   0,
 0, 1, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
@@ -1017,8 +968,8 @@ static const flex_int32_t yy_rule_can_match_eol[162] =
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
     0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 
-    0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 0, 1, 1, 1, 0, 0, 0, 1, 0, 
-    0, 0,     };
+    0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 0, 1, 1, 1, 0, 0, 0, 1, 
+    0, 0, 0,     };
 
 /* The intent behind this definition is that it'll catch
  * any uses of REJECT which flex missed.
@@ -1128,14 +1079,17 @@ class UnaryOperation;
     yycolumn += yyleng;                                   \
   }
 
-#line 1131 "SqlLexer_gen.cpp"
 /* FIXME(chasseur, qzeng): Add support for hexadecimal literals. */
 /**
  * These patterns are based on the SQL-2011 standard for syntax of numeric
  * literals (Part 2, Section 5.3 of the standard).
  **/
 
-#line 1138 "SqlLexer_gen.cpp"
+
+
+
+
+#line 1093 "SqlLexer_gen.cpp"
 
 #define INITIAL 0
 #define CONDITION_SQL 1
@@ -1169,8 +1123,8 @@ struct yyguts_t
     size_t yy_buffer_stack_max; /**< capacity of stack. */
     YY_BUFFER_STATE * yy_buffer_stack; /**< Stack as an array. */
     char yy_hold_char;
-    int yy_n_chars;
-    int yyleng_r;
+    yy_size_t yy_n_chars;
+    yy_size_t yyleng_r;
     char *yy_c_buf_p;
     int yy_init;
     int yy_start;
@@ -1194,7 +1148,7 @@ struct yyguts_t
 
     }; /* end struct yyguts_t */
 
-static int yy_init_globals ( yyscan_t yyscanner );
+static int yy_init_globals (yyscan_t yyscanner );
 
     /* This must go here because YYSTYPE and YYLTYPE are included
      * from bison output in section 1.*/
@@ -1204,48 +1158,48 @@ static int yy_init_globals ( yyscan_t yyscanner );
     
 int quickstep_yylex_init (yyscan_t* scanner);
 
-int quickstep_yylex_init_extra ( YY_EXTRA_TYPE user_defined, yyscan_t* scanner);
+int quickstep_yylex_init_extra (YY_EXTRA_TYPE user_defined,yyscan_t* scanner);
 
 /* Accessor methods to globals.
    These are made visible to non-reentrant scanners for convenience. */
 
-int quickstep_yylex_destroy ( yyscan_t yyscanner );
+int quickstep_yylex_destroy (yyscan_t yyscanner );
 
-int quickstep_yyget_debug ( yyscan_t yyscanner );
+int quickstep_yyget_debug (yyscan_t yyscanner );
 
-void quickstep_yyset_debug ( int debug_flag , yyscan_t yyscanner );
+void quickstep_yyset_debug (int debug_flag ,yyscan_t yyscanner );
 
-YY_EXTRA_TYPE quickstep_yyget_extra ( yyscan_t yyscanner );
+YY_EXTRA_TYPE quickstep_yyget_extra (yyscan_t yyscanner );
 
-void quickstep_yyset_extra ( YY_EXTRA_TYPE user_defined , yyscan_t yyscanner );
+void quickstep_yyset_extra (YY_EXTRA_TYPE user_defined ,yyscan_t yyscanner );
 
-FILE *quickstep_yyget_in ( yyscan_t yyscanner );
+FILE *quickstep_yyget_in (yyscan_t yyscanner );
 
-void quickstep_yyset_in  ( FILE * _in_str , yyscan_t yyscanner );
+void quickstep_yyset_in  (FILE * _in_str ,yyscan_t yyscanner );
 
-FILE *quickstep_yyget_out ( yyscan_t yyscanner );
+FILE *quickstep_yyget_out (yyscan_t yyscanner );
 
-void quickstep_yyset_out  ( FILE * _out_str , yyscan_t yyscanner );
+void quickstep_yyset_out  (FILE * _out_str ,yyscan_t yyscanner );
 
-			int quickstep_yyget_leng ( yyscan_t yyscanner );
+yy_size_t quickstep_yyget_leng (yyscan_t yyscanner );
 
-char *quickstep_yyget_text ( yyscan_t yyscanner );
+char *quickstep_yyget_text (yyscan_t yyscanner );
 
-int quickstep_yyget_lineno ( yyscan_t yyscanner );
+int quickstep_yyget_lineno (yyscan_t yyscanner );
 
-void quickstep_yyset_lineno ( int _line_number , yyscan_t yyscanner );
+void quickstep_yyset_lineno (int _line_number ,yyscan_t yyscanner );
 
-int quickstep_yyget_column  ( yyscan_t yyscanner );
+int quickstep_yyget_column  (yyscan_t yyscanner );
 
-void quickstep_yyset_column ( int _column_no , yyscan_t yyscanner );
+void quickstep_yyset_column (int _column_no ,yyscan_t yyscanner );
 
-YYSTYPE * quickstep_yyget_lval ( yyscan_t yyscanner );
+YYSTYPE * quickstep_yyget_lval (yyscan_t yyscanner );
 
-void quickstep_yyset_lval ( YYSTYPE * yylval_param , yyscan_t yyscanner );
+void quickstep_yyset_lval (YYSTYPE * yylval_param ,yyscan_t yyscanner );
 
-       YYLTYPE *quickstep_yyget_lloc ( yyscan_t yyscanner );
+       YYLTYPE *quickstep_yyget_lloc (yyscan_t yyscanner );
     
-        void quickstep_yyset_lloc ( YYLTYPE * yylloc_param , yyscan_t yyscanner );
+        void quickstep_yyset_lloc (YYLTYPE * yylloc_param ,yyscan_t yyscanner );
     
 /* Macros after this point can all be overridden by user definitions in
  * section 1.
@@ -1253,9 +1207,9 @@ void quickstep_yyset_lval ( YYSTYPE * yylval_param , yyscan_t yyscanner );
 
 #ifndef YY_SKIP_YYWRAP
 #ifdef __cplusplus
-extern "C" int quickstep_yywrap ( yyscan_t yyscanner );
+extern "C" int quickstep_yywrap (yyscan_t yyscanner );
 #else
-extern int quickstep_yywrap ( yyscan_t yyscanner );
+extern int quickstep_yywrap (yyscan_t yyscanner );
 #endif
 #endif
 
@@ -1264,18 +1218,19 @@ extern int quickstep_yywrap ( yyscan_t yyscanner );
 #endif
 
 #ifndef yytext_ptr
-static void yy_flex_strncpy ( char *, const char *, int , yyscan_t yyscanner);
+static void yy_flex_strncpy (char *,yyconst char *,int ,yyscan_t yyscanner);
 #endif
 
 #ifdef YY_NEED_STRLEN
-static int yy_flex_strlen ( const char * , yyscan_t yyscanner);
+static int yy_flex_strlen (yyconst char * ,yyscan_t yyscanner);
 #endif
 
 #ifndef YY_NO_INPUT
+
 #ifdef __cplusplus
-static int yyinput ( yyscan_t yyscanner );
+static int yyinput (yyscan_t yyscanner );
 #else
-static int input ( yyscan_t yyscanner );
+static int input (yyscan_t yyscanner );
 #endif
 
 #endif
@@ -1295,7 +1250,7 @@ static int input ( yyscan_t yyscanner );
 /* This used to be an fputs(), but since the string might contain NUL's,
  * we now use fwrite().
  */
-#define ECHO do { if (fwrite( yytext, (size_t) yyleng, 1, yyout )) {} } while (0)
+#define ECHO do { if (fwrite( yytext, yyleng, 1, yyout )) {} } while (0)
 #endif
 
 /* Gets input and stuffs it into "buf".  number of characters read, or YY_NULL,
@@ -1306,7 +1261,7 @@ static int input ( yyscan_t yyscanner );
 	if ( YY_CURRENT_BUFFER_LVALUE->yy_is_interactive ) \
 		{ \
 		int c = '*'; \
-		int n; \
+		size_t n; \
 		for ( n = 0; n < max_size && \
 			     (c = getc( yyin )) != EOF && c != '\n'; ++n ) \
 			buf[n] = (char) c; \
@@ -1319,7 +1274,7 @@ static int input ( yyscan_t yyscanner );
 	else \
 		{ \
 		errno=0; \
-		while ( (result = (int) fread(buf, 1, (yy_size_t) max_size, yyin)) == 0 && ferror(yyin)) \
+		while ( (result = fread(buf, 1, max_size, yyin))==0 && ferror(yyin)) \
 			{ \
 			if( errno != EINTR) \
 				{ \
@@ -1361,7 +1316,7 @@ static int input ( yyscan_t yyscanner );
 #define YY_DECL_IS_OURS 1
 
 extern int quickstep_yylex \
-               (YYSTYPE * yylval_param, YYLTYPE * yylloc_param , yyscan_t yyscanner);
+               (YYSTYPE * yylval_param,YYLTYPE * yylloc_param ,yyscan_t yyscanner);
 
 #define YY_DECL int quickstep_yylex \
                (YYSTYPE * yylval_param, YYLTYPE * yylloc_param , yyscan_t yyscanner)
@@ -1425,7 +1380,7 @@ YY_DECL
 #line 131 "../SqlLexer.lpp"
 
 
-#line 1428 "SqlLexer_gen.cpp"
+#line 1384 "SqlLexer_gen.cpp"
 
 	while ( /*CONSTCOND*/1 )		/* loops until end-of-file is reached */
 		{
@@ -1452,13 +1407,13 @@ yy_match:
 			while ( yy_chk[yy_base[yy_current_state] + yy_c] != yy_current_state )
 				{
 				yy_current_state = (int) yy_def[yy_current_state];
-				if ( yy_current_state >= 593 )
-					yy_c = yy_meta[yy_c];
+				if ( yy_current_state >= 594 )
+					yy_c = yy_meta[(unsigned int) yy_c];
 				}
-			yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c];
+			yy_current_state = yy_nxt[yy_base[yy_current_state] + (unsigned int) yy_c];
 			++yy_cp;
 			}
-		while ( yy_current_state != 592 );
+		while ( yy_current_state != 593 );
 		yy_cp = yyg->yy_last_accepting_cpos;
 		yy_current_state = yyg->yy_last_accepting_state;
 
@@ -1469,10 +1424,10 @@ yy_find_action:
 
 		if ( yy_act != YY_END_OF_BUFFER && yy_rule_can_match_eol[yy_act] )
 			{
-			int yyl;
+			yy_size_t yyl;
 			for ( yyl = 0; yyl < yyleng; ++yyl )
 				if ( yytext[yyl] == '\n' )
-					
+					   
     do{ yylineno++;
         yycolumn=0;
     }while(0)
@@ -1622,17 +1577,17 @@ return TOKEN_CASE;
 case 21:
 YY_RULE_SETUP
 #line 186 "../SqlLexer.lpp"
-return TOKEN_CSB_TREE;
+return TOKEN_CAST;
 	YY_BREAK
 case 22:
 YY_RULE_SETUP
 #line 187 "../SqlLexer.lpp"
-return TOKEN_BY;
+return TOKEN_CSB_TREE;
 	YY_BREAK
 case 23:
 YY_RULE_SETUP
 #line 188 "../SqlLexer.lpp"
-return TOKEN_CHARACTER;
+return TOKEN_BY;
 	YY_BREAK
 case 24:
 YY_RULE_SETUP
@@ -1642,72 +1597,72 @@ return TOKEN_CHARACTER;
 case 25:
 YY_RULE_SETUP
 #line 190 "../SqlLexer.lpp"
-return TOKEN_CHECK;
+return TOKEN_CHARACTER;
 	YY_BREAK
 case 26:
 YY_RULE_SETUP
 #line 191 "../SqlLexer.lpp"
-return TOKEN_COLUMN;
+return TOKEN_CHECK;
 	YY_BREAK
 case 27:
 YY_RULE_SETUP
 #line 192 "../SqlLexer.lpp"
-return TOKEN_CONSTRAINT;
+return TOKEN_COLUMN;
 	YY_BREAK
 case 28:
 YY_RULE_SETUP
 #line 193 "../SqlLexer.lpp"
-return TOKEN_COPY;
+return TOKEN_CONSTRAINT;
 	YY_BREAK
 case 29:
 YY_RULE_SETUP
 #line 194 "../SqlLexer.lpp"
-return TOKEN_CREATE;
+return TOKEN_COPY;
 	YY_BREAK
 case 30:
 YY_RULE_SETUP
 #line 195 "../SqlLexer.lpp"
-return TOKEN_CURRENT;
+return TOKEN_CREATE;
 	YY_BREAK
 case 31:
 YY_RULE_SETUP
 #line 196 "../SqlLexer.lpp"
-return TOKEN_DATE;
+return TOKEN_CURRENT;
 	YY_BREAK
 case 32:
 YY_RULE_SETUP
 #line 197 "../SqlLexer.lpp"
-return TOKEN_DATETIME;
+return TOKEN_DATE;
 	YY_BREAK
 case 33:
 YY_RULE_SETUP
 #line 198 "../SqlLexer.lpp"
-return TOKEN_DAY;
+return TOKEN_DATETIME;
 	YY_BREAK
 case 34:
 YY_RULE_SETUP
 #line 199 "../SqlLexer.lpp"
-return TOKEN_DECIMAL;
+return TOKEN_DAY;
 	YY_BREAK
 case 35:
 YY_RULE_SETUP
 #line 200 "../SqlLexer.lpp"
-return TOKEN_DEFAULT;
+return TOKEN_DECIMAL;
 	YY_BREAK
 case 36:
 YY_RULE_SETUP
 #line 201 "../SqlLexer.lpp"
-return TOKEN_DELETE;
+return TOKEN_DEFAULT;
 	YY_BREAK
 case 37:
 YY_RULE_SETUP
 #line 202 "../SqlLexer.lpp"
-return TOKEN_DELIMITER;
+return TOKEN_DELETE;
 	YY_BREAK
 case 38:
 YY_RULE_SETUP
 #line 203 "../SqlLexer.lpp"
-return TOKEN_DESC;
+return TOKEN_DELIMITER;
 	YY_BREAK
 case 39:
 YY_RULE_SETUP
@@ -1717,127 +1672,127 @@ return TOKEN_DESC;
 case 40:
 YY_RULE_SETUP
 #line 205 "../SqlLexer.lpp"
-return TOKEN_DISTINCT;
+return TOKEN_DESC;
 	YY_BREAK
 case 41:
 YY_RULE_SETUP
 #line 206 "../SqlLexer.lpp"
-return TOKEN_DOUBLE;
+return TOKEN_DISTINCT;
 	YY_BREAK
 case 42:
 YY_RULE_SETUP
 #line 207 "../SqlLexer.lpp"
-return TOKEN_DROP;
+return TOKEN_DOUBLE;
 	YY_BREAK
 case 43:
 YY_RULE_SETUP
 #line 208 "../SqlLexer.lpp"
-return TOKEN_ELSE;
+return TOKEN_DROP;
 	YY_BREAK
 case 44:
 YY_RULE_SETUP
 #line 209 "../SqlLexer.lpp"
-return TOKEN_END;
+return TOKEN_ELSE;
 	YY_BREAK
 case 45:
 YY_RULE_SETUP
 #line 210 "../SqlLexer.lpp"
-return TOKEN_ESCAPE_STRINGS;
+return TOKEN_END;
 	YY_BREAK
 case 46:
 YY_RULE_SETUP
 #line 211 "../SqlLexer.lpp"
-return TOKEN_EXISTS;
+return TOKEN_ESCAPE_STRINGS;
 	YY_BREAK
 case 47:
 YY_RULE_SETUP
 #line 212 "../SqlLexer.lpp"
-return TOKEN_EXTRACT;
+return TOKEN_EXISTS;
 	YY_BREAK
 case 48:
 YY_RULE_SETUP
 #line 213 "../SqlLexer.lpp"
-return TOKEN_FALSE;
+return TOKEN_EXTRACT;
 	YY_BREAK
 case 49:
 YY_RULE_SETUP
 #line 214 "../SqlLexer.lpp"
-return TOKEN_FIRST;
+return TOKEN_FALSE;
 	YY_BREAK
 case 50:
 YY_RULE_SETUP
 #line 215 "../SqlLexer.lpp"
-return TOKEN_FLOAT;
+return TOKEN_FIRST;
 	YY_BREAK
 case 51:
 YY_RULE_SETUP
 #line 216 "../SqlLexer.lpp"
-return TOKEN_FOLLOWING;
+return TOKEN_FLOAT;
 	YY_BREAK
 case 52:
 YY_RULE_SETUP
 #line 217 "../SqlLexer.lpp"
-return TOKEN_FOR;
+return TOKEN_FOLLOWING;
 	YY_BREAK
 case 53:
 YY_RULE_SETUP
 #line 218 "../SqlLexer.lpp"
-return TOKEN_FOREIGN;
+return TOKEN_FOR;
 	YY_BREAK
 case 54:
 YY_RULE_SETUP
 #line 219 "../SqlLexer.lpp"
-return TOKEN_FROM;
+return TOKEN_FOREIGN;
 	YY_BREAK
 case 55:
 YY_RULE_SETUP
 #line 220 "../SqlLexer.lpp"
-return TOKEN_FULL;
+return TOKEN_FROM;
 	YY_BREAK
 case 56:
 YY_RULE_SETUP
 #line 221 "../SqlLexer.lpp"
-return TOKEN_GROUP;
+return TOKEN_FULL;
 	YY_BREAK
 case 57:
 YY_RULE_SETUP
 #line 222 "../SqlLexer.lpp"
-return TOKEN_HASH;
+return TOKEN_GROUP;
 	YY_BREAK
 case 58:
 YY_RULE_SETUP
 #line 223 "../SqlLexer.lpp"
-return TOKEN_HAVING;
+return TOKEN_HASH;
 	YY_BREAK
 case 59:
 YY_RULE_SETUP
 #line 224 "../SqlLexer.lpp"
-return TOKEN_HOUR;
+return TOKEN_HAVING;
 	YY_BREAK
 case 60:
 YY_RULE_SETUP
 #line 225 "../SqlLexer.lpp"
-return TOKEN_IN;
+return TOKEN_HOUR;
 	YY_BREAK
 case 61:
 YY_RULE_SETUP
 #line 226 "../SqlLexer.lpp"
-return TOKEN_INDEX;
+return TOKEN_IN;
 	YY_BREAK
 case 62:
 YY_RULE_SETUP
 #line 227 "../SqlLexer.lpp"
-return TOKEN_INNER;
+return TOKEN_INDEX;
 	YY_BREAK
 case 63:
 YY_RULE_SETUP
 #line 228 "../SqlLexer.lpp"
-return TOKEN_INSERT;
+return TOKEN_INNER;
 	YY_BREAK
 case 64:
 YY_RULE_SETUP
 #line 229 "../SqlLexer.lpp"
-return TOKEN_INTEGER;
+return TOKEN_INSERT;
 	YY_BREAK
 case 65:
 YY_RULE_SETUP
@@ -1847,312 +1802,312 @@ return TOKEN_INTEGER;
 case 66:
 YY_RULE_SETUP
 #line 231 "../SqlLexer.lpp"
-return TOKEN_INTERVAL;
+return TOKEN_INTEGER;
 	YY_BREAK
 case 67:
 YY_RULE_SETUP
 #line 232 "../SqlLexer.lpp"
-return TOKEN_INTO;
+return TOKEN_INTERVAL;
 	YY_BREAK
 case 68:
 YY_RULE_SETUP
 #line 233 "../SqlLexer.lpp"
-return TOKEN_IS;
+return TOKEN_INTO;
 	YY_BREAK
 case 69:
 YY_RULE_SETUP
 #line 234 "../SqlLexer.lpp"
-return TOKEN_JOIN;
+return TOKEN_IS;
 	YY_BREAK
 case 70:
 YY_RULE_SETUP
 #line 235 "../SqlLexer.lpp"
-return TOKEN_KEY;
+return TOKEN_JOIN;
 	YY_BREAK
 case 71:
 YY_RULE_SETUP
 #line 236 "../SqlLexer.lpp"
-return TOKEN_LAST;
+return TOKEN_KEY;
 	YY_BREAK
 case 72:
 YY_RULE_SETUP
 #line 237 "../SqlLexer.lpp"
-return TOKEN_LEFT;
+return TOKEN_LAST;
 	YY_BREAK
 case 73:
 YY_RULE_SETUP
 #line 238 "../SqlLexer.lpp"
-return TOKEN_LIKE;
+return TOKEN_LEFT;
 	YY_BREAK
 case 74:
 YY_RULE_SETUP
 #line 239 "../SqlLexer.lpp"
-return TOKEN_LIMIT;
+return TOKEN_LIKE;
 	YY_BREAK
 case 75:
 YY_RULE_SETUP
 #line 240 "../SqlLexer.lpp"
-return TOKEN_LONG;
+return TOKEN_LIMIT;
 	YY_BREAK
 case 76:
 YY_RULE_SETUP
 #line 241 "../SqlLexer.lpp"
-return TOKEN_MINUTE;
+return TOKEN_LONG;
 	YY_BREAK
 case 77:
 YY_RULE_SETUP
 #line 242 "../SqlLexer.lpp"
-return TOKEN_MONTH;
+return TOKEN_MINUTE;
 	YY_BREAK
 case 78:
 YY_RULE_SETUP
 #line 243 "../SqlLexer.lpp"
-return TOKEN_NOT;
+return TOKEN_MONTH;
 	YY_BREAK
 case 79:
 YY_RULE_SETUP
 #line 244 "../SqlLexer.lpp"
-return TOKEN_NULL;
+return TOKEN_NOT;
 	YY_BREAK
 case 80:
 YY_RULE_SETUP
 #line 245 "../SqlLexer.lpp"
-return TOKEN_NULLS;
+return TOKEN_NULL;
 	YY_BREAK
 case 81:
 YY_RULE_SETUP
 #line 246 "../SqlLexer.lpp"
-return TOKEN_OFF;
+return TOKEN_NULLS;
 	YY_BREAK
 case 82:
 YY_RULE_SETUP
 #line 247 "../SqlLexer.lpp"
-return TOKEN_ON;
+return TOKEN_OFF;
 	YY_BREAK
 case 83:
 YY_RULE_SETUP
 #line 248 "../SqlLexer.lpp"
-return TOKEN_OR;
+return TOKEN_ON;
 	YY_BREAK
 case 84:
 YY_RULE_SETUP
 #line 249 "../SqlLexer.lpp"
-return TOKEN_ORDER;
+return TOKEN_OR;
 	YY_BREAK
 case 85:
 YY_RULE_SETUP
 #line 250 "../SqlLexer.lpp"
-return TOKEN_OUTER;
+return TOKEN_ORDER;
 	YY_BREAK
 case 86:
 YY_RULE_SETUP
 #line 251 "../SqlLexer.lpp"
-return TOKEN_OVER;
+return TOKEN_OUTER;
 	YY_BREAK
 case 87:
 YY_RULE_SETUP
 #line 252 "../SqlLexer.lpp"
-return TOKEN_PARTITION;
+return TOKEN_OVER;
 	YY_BREAK
 case 88:
 YY_RULE_SETUP
 #line 253 "../SqlLexer.lpp"
-return TOKEN_PARTITIONS;
+return TOKEN_PARTITION;
 	YY_BREAK
 case 89:
 YY_RULE_SETUP
 #line 254 "../SqlLexer.lpp"
-return TOKEN_PERCENT;
+return TOKEN_PARTITIONS;
 	YY_BREAK
 case 90:
 YY_RULE_SETUP
 #line 255 "../SqlLexer.lpp"
-return TOKEN_PRECEDING;
+return TOKEN_PERCENT;
 	YY_BREAK
 case 91:
 YY_RULE_SETUP
 #line 256 "../SqlLexer.lpp"
-return TOKEN_PRIMARY;
+return TOKEN_PRECEDING;
 	YY_BREAK
 case 92:
 YY_RULE_SETUP
 #line 257 "../SqlLexer.lpp"
-return TOKEN_PRIORITY;
+return TOKEN_PRIMARY;
 	YY_BREAK
 case 93:
 YY_RULE_SETUP
 #line 258 "../SqlLexer.lpp"
-return TOKEN_QUIT;
+return TOKEN_PRIORITY;
 	YY_BREAK
 case 94:
 YY_RULE_SETUP
 #line 259 "../SqlLexer.lpp"
-return TOKEN_RANGE;
+return TOKEN_QUIT;
 	YY_BREAK
 case 95:
 YY_RULE_SETUP
 #line 260 "../SqlLexer.lpp"
-return TOKEN_REAL;
+return TOKEN_RANGE;
 	YY_BREAK
 case 96:
 YY_RULE_SETUP
 #line 261 "../SqlLexer.lpp"
-return TOKEN_REFERENCES;
+return TOKEN_REAL;
 	YY_BREAK
 case 97:
 YY_RULE_SETUP
 #line 262 "../SqlLexer.lpp"
-return TOKEN_REGEXP;
+return TOKEN_REFERENCES;
 	YY_BREAK
 case 98:
 YY_RULE_SETUP
 #line 263 "../SqlLexer.lpp"
-return TOKEN_RIGHT;
+return TOKEN_REGEXP;
 	YY_BREAK
 case 99:
 YY_RULE_SETUP
 #line 264 "../SqlLexer.lpp"
-return TOKEN_ROW;
+return TOKEN_RIGHT;
 	YY_BREAK
 case 100:
 YY_RULE_SETUP
 #line 265 "../SqlLexer.lpp"
-return TOKEN_ROW_DELIMITER;
+return TOKEN_ROW;
 	YY_BREAK
 case 101:
 YY_RULE_SETUP
 #line 266 "../SqlLexer.lpp"
-return TOKEN_ROWS;
+return TOKEN_ROW_DELIMITER;
 	YY_BREAK
 case 102:
 YY_RULE_SETUP
 #line 267 "../SqlLexer.lpp"
-return TOKEN_SECOND;
+return TOKEN_ROWS;
 	YY_BREAK
 case 103:
 YY_RULE_SETUP
 #line 268 "../SqlLexer.lpp"
-return TOKEN_SELECT;
+return TOKEN_SECOND;
 	YY_BREAK
 case 104:
 YY_RULE_SETUP
 #line 269 "../SqlLexer.lpp"
-return TOKEN_SET;
+return TOKEN_SELECT;
 	YY_BREAK
 case 105:
 YY_RULE_SETUP
 #line 270 "../SqlLexer.lpp"
-return TOKEN_SMA;
+return TOKEN_SET;
 	YY_BREAK
 case 106:
 YY_RULE_SETUP
 #line 271 "../SqlLexer.lpp"
-return TOKEN_SMALLINT;
+return TOKEN_SMA;
 	YY_BREAK
 case 107:
 YY_RULE_SETUP
 #line 272 "../SqlLexer.lpp"
-return TOKEN_SUBSTRING;
+return TOKEN_SMALLINT;
 	YY_BREAK
 case 108:
 YY_RULE_SETUP
 #line 273 "../SqlLexer.lpp"
-return TOKEN_TABLE;
+return TOKEN_SUBSTRING;
 	YY_BREAK
 case 109:
 YY_RULE_SETUP
 #line 274 "../SqlLexer.lpp"
-return TOKEN_THEN;
+return TOKEN_TABLE;
 	YY_BREAK
 case 110:
 YY_RULE_SETUP
 #line 275 "../SqlLexer.lpp"
-return TOKEN_TIME;
+return TOKEN_THEN;
 	YY_BREAK
 case 111:
 YY_RULE_SETUP
 #line 276 "../SqlLexer.lpp"
-return TOKEN_TIMESTAMP;
+return TOKEN_TIME;
 	YY_BREAK
 case 112:
 YY_RULE_SETUP
 #line 277 "../SqlLexer.lpp"
-return TOKEN_TRUE;
+return TOKEN_TIMESTAMP;
 	YY_BREAK
 case 113:
 YY_RULE_SETUP
 #line 278 "../SqlLexer.lpp"
-return TOKEN_TUPLESAMPLE;
+return TOKEN_TRUE;
 	YY_BREAK
 case 114:
 YY_RULE_SETUP
 #line 279 "../SqlLexer.lpp"
-return TOKEN_UNBOUNDED;
+return TOKEN_TUPLESAMPLE;
 	YY_BREAK
 case 115:
 YY_RULE_SETUP
 #line 280 "../SqlLexer.lpp"
-return TOKEN_UNIQUE;
+return TOKEN_UNBOUNDED;
 	YY_BREAK
 case 116:
 YY_RULE_SETUP
 #line 281 "../SqlLexer.lpp"
-return TOKEN_UPDATE;
+return TOKEN_UNIQUE;
 	YY_BREAK
 case 117:
 YY_RULE_SETUP
 #line 282 "../SqlLexer.lpp"
-return TOKEN_USING;
+return TOKEN_UPDATE;
 	YY_BREAK
 case 118:
 YY_RULE_SETUP
 #line 283 "../SqlLexer.lpp"
-return TOKEN_VALUES;
+return TOKEN_USING;
 	YY_BREAK
 case 119:
 YY_RULE_SETUP
 #line 284 "../SqlLexer.lpp"
-return TOKEN_VARCHAR;
+return TOKEN_VALUES;
 	YY_BREAK
 case 120:
 YY_RULE_SETUP
 #line 285 "../SqlLexer.lpp"
-return TOKEN_WHEN;
+return TOKEN_VARCHAR;
 	YY_BREAK
 case 121:
 YY_RULE_SETUP
 #line 286 "../SqlLexer.lpp"
-return TOKEN_WHERE;
+return TOKEN_WHEN;
 	YY_BREAK
 case 122:
 YY_RULE_SETUP
 #line 287 "../SqlLexer.lpp"
-return TOKEN_WINDOW;
+return TOKEN_WHERE;
 	YY_BREAK
 case 123:
 YY_RULE_SETUP
 #line 288 "../SqlLexer.lpp"
-return TOKEN_WITH;
+return TOKEN_WINDOW;
 	YY_BREAK
 case 124:
 YY_RULE_SETUP
 #line 289 "../SqlLexer.lpp"
-return TOKEN_YEAR;
+return TOKEN_WITH;
 	YY_BREAK
 case 125:
 YY_RULE_SETUP
 #line 290 "../SqlLexer.lpp"
-return TOKEN_YEARMONTH;
+return TOKEN_YEAR;
 	YY_BREAK
 case 126:
 YY_RULE_SETUP
-#line 292 "../SqlLexer.lpp"
-return TOKEN_EQ;
+#line 291 "../SqlLexer.lpp"
+return TOKEN_YEARMONTH;
 	YY_BREAK
 case 127:
 YY_RULE_SETUP
 #line 293 "../SqlLexer.lpp"
-return TOKEN_NEQ;
+return TOKEN_EQ;
 	YY_BREAK
 case 128:
 YY_RULE_SETUP
@@ -2162,56 +2117,61 @@ return TOKEN_NEQ;
 case 129:
 YY_RULE_SETUP
 #line 295 "../SqlLexer.lpp"
-return TOKEN_LT;
+return TOKEN_NEQ;
 	YY_BREAK
 case 130:
 YY_RULE_SETUP
 #line 296 "../SqlLexer.lpp"
-return TOKEN_GT;
+return TOKEN_LT;
 	YY_BREAK
 case 131:
 YY_RULE_SETUP
 #line 297 "../SqlLexer.lpp"
-return TOKEN_LEQ;
+return TOKEN_GT;
 	YY_BREAK
 case 132:
 YY_RULE_SETUP
 #line 298 "../SqlLexer.lpp"
-return TOKEN_GEQ;
+return TOKEN_LEQ;
 	YY_BREAK
 case 133:
 YY_RULE_SETUP
-#line 300 "../SqlLexer.lpp"
-return yytext[0];
+#line 299 "../SqlLexer.lpp"
+return TOKEN_GEQ;
 	YY_BREAK
 case 134:
 YY_RULE_SETUP
 #line 301 "../SqlLexer.lpp"
 return yytext[0];
 	YY_BREAK
+case 135:
+YY_RULE_SETUP
+#line 302 "../SqlLexer.lpp"
+return yytext[0];
+	YY_BREAK
 /**
     * Quoted strings. Prefacing a string with an 'e' or 'E' causes escape
     * sequences to be processed (as in PostgreSQL).
     **/
-case 135:
+case 136:
 YY_RULE_SETUP
-#line 307 "../SqlLexer.lpp"
+#line 308 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED_ESCAPED);
   }
 	YY_BREAK
-case 136:
+case 137:
 YY_RULE_SETUP
-#line 312 "../SqlLexer.lpp"
+#line 313 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_SINGLE_QUOTED);
   }
 	YY_BREAK
-case 137:
+case 138:
 YY_RULE_SETUP
-#line 317 "../SqlLexer.lpp"
+#line 318 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(yylloc->first_line, yylloc->first_column);
     BEGIN(CONDITION_STRING_DOUBLE_QUOTED);
@@ -2223,7 +2183,7 @@ YY_RULE_SETUP
 case YY_STATE_EOF(CONDITION_STRING_SINGLE_QUOTED):
 case YY_STATE_EOF(CONDITION_STRING_SINGLE_QUOTED_ESCAPED):
 case YY_STATE_EOF(CONDITION_STRING_DOUBLE_QUOTED):
-#line 326 "../SqlLexer.lpp"
+#line 327 "../SqlLexer.lpp"
 {
     delete yylval->string_value_;
     BEGIN(INITIAL);
@@ -2234,9 +2194,9 @@ case YY_STATE_EOF(CONDITION_STRING_DOUBLE_QUOTED):
 
 /* Process escape sequences. */
 
-case 138:
+case 139:
 YY_RULE_SETUP
-#line 336 "../SqlLexer.lpp"
+#line 337 "../SqlLexer.lpp"
 {
     /* Octal code */
     unsigned int code;
@@ -2250,9 +2210,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 139:
+case 140:
 YY_RULE_SETUP
-#line 348 "../SqlLexer.lpp"
+#line 349 "../SqlLexer.lpp"
 {
     /* Hexadecimal code */
     unsigned int code;
@@ -2260,9 +2220,9 @@ YY_RULE_SETUP
     yylval->string_value_->push_back(code);
   }
 	YY_BREAK
-case 140:
+case 141:
 YY_RULE_SETUP
-#line 354 "../SqlLexer.lpp"
+#line 355 "../SqlLexer.lpp"
 {
     /* A numeric escape sequence that isn't correctly specified. */
     delete yylval->string_value_;
@@ -2271,58 +2231,58 @@ YY_RULE_SETUP
     return TOKEN_LEX_ERROR;
   }
 	YY_BREAK
-case 141:
+case 142:
 YY_RULE_SETUP
-#line 361 "../SqlLexer.lpp"
+#line 362 "../SqlLexer.lpp"
 {
     /* Backspace */
     yylval->string_value_->push_back('\b');
   }
 	YY_BREAK
-case 142:
+case 143:
 YY_RULE_SETUP
-#line 365 "../SqlLexer.lpp"
+#line 366 "../SqlLexer.lpp"
 {
     /* Form-feed */
     yylval->string_value_->push_back('\f');
   }
 	YY_BREAK
-case 143:
+case 144:
 YY_RULE_SETUP
-#line 369 "../SqlLexer.lpp"
+#line 370 "../SqlLexer.lpp"
 {
     /* Newline */
     yylval->string_value_->push_back('\n');
   }
 	YY_BREAK
-case 144:
+case 145:
 YY_RULE_SETUP
-#line 373 "../SqlLexer.lpp"
+#line 374 "../SqlLexer.lpp"
 {
     /* Carriage-return */
     yylval->string_value_->push_back('\r');
   }
 	YY_BREAK
-case 145:
+case 146:
 YY_RULE_SETUP
-#line 377 "../SqlLexer.lpp"
+#line 378 "../SqlLexer.lpp"
 {
     /* Horizontal Tab */
     yylval->string_value_->push_back('\t');
   }
 	YY_BREAK
-case 146:
-/* rule 146 can match eol */
+case 147:
+/* rule 147 can match eol */
 YY_RULE_SETUP
-#line 381 "../SqlLexer.lpp"
+#line 382 "../SqlLexer.lpp"
 {
     /* Any other character (including actual newline or carriage return) */
     yylval->string_value_->push_back(yytext[1]);
   }
 	YY_BREAK
-case 147:
+case 148:
 YY_RULE_SETUP
-#line 385 "../SqlLexer.lpp"
+#line 386 "../SqlLexer.lpp"
 {
     /* This should only be encountered right before an EOF. */
     delete yylval->string_value_;
@@ -2333,17 +2293,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 148:
+case 149:
 YY_RULE_SETUP
-#line 395 "../SqlLexer.lpp"
+#line 396 "../SqlLexer.lpp"
 {
     /* Two quotes in a row become a single quote (this is specified by the SQL standard). */
     yylval->string_value_->push_back('\'');
   }
 	YY_BREAK
-case 149:
+case 150:
 YY_RULE_SETUP
-#line 399 "../SqlLexer.lpp"
+#line 400 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2352,17 +2312,17 @@ YY_RULE_SETUP
 	YY_BREAK
 
 
-case 150:
+case 151:
 YY_RULE_SETUP
-#line 407 "../SqlLexer.lpp"
+#line 408 "../SqlLexer.lpp"
 {
     /* Two quotes in a row become a single quote (this is specified by the SQL standard). */
     yylval->string_value_->push_back('"');
   }
 	YY_BREAK
-case 151:
+case 152:
 YY_RULE_SETUP
-#line 411 "../SqlLexer.lpp"
+#line 412 "../SqlLexer.lpp"
 {
     /* End string */
     BEGIN(CONDITION_SQL);
@@ -2370,94 +2330,94 @@ YY_RULE_SETUP
   }
 	YY_BREAK
 
-case 152:
-/* rule 152 can match eol */
+case 153:
+/* rule 153 can match eol */
 YY_RULE_SETUP
-#line 418 "../SqlLexer.lpp"
+#line 419 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 153:
-/* rule 153 can match eol */
+case 154:
+/* rule 154 can match eol */
 YY_RULE_SETUP
-#line 423 "../SqlLexer.lpp"
+#line 424 "../SqlLexer.lpp"
 {
   /* Scan up to a quote or escape sequence. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
-case 154:
-/* rule 154 can match eol */
+case 155:
+/* rule 155 can match eol */
 YY_RULE_SETUP
-#line 428 "../SqlLexer.lpp"
+#line 429 "../SqlLexer.lpp"
 {
   /* Scan up to a quote. */
   yylval->string_value_->append(yytext, yyleng);
 }
 	YY_BREAK
 
-case 155:
+case 156:
 YY_RULE_SETUP
-#line 434 "../SqlLexer.lpp"
+#line 435 "../SqlLexer.lpp"
 {
     yylval->string_value_ = new quickstep::ParseString(
         yylloc->first_line, yylloc->first_column, std::string(yytext, yyleng));
     return TOKEN_NAME;
   }
 	YY_BREAK
-case 156:
+case 157:
 YY_RULE_SETUP
-#line 440 "../SqlLexer.lpp"
+#line 441 "../SqlLexer.lpp"
 {
     yylval->numeric_literal_value_ = new quickstep::NumericParseLiteralValue(
         yylloc->first_line, yylloc->first_column, yytext);
     return TOKEN_UNSIGNED_NUMVAL;
   }
 	YY_BREAK
-case 157:
+case 158:
 YY_RULE_SETUP
-#line 446 "../SqlLexer.lpp"
+#line 447 "../SqlLexer.lpp"
 /* comment */
 	YY_BREAK
-case 158:
-/* rule 158 can match eol */
+case 159:
+/* rule 159 can match eol */
 YY_RULE_SETUP
-#line 448 "../SqlLexer.lpp"
+#line 449 "../SqlLexer.lpp"
 { yycolumn = 0; }
 	YY_BREAK
-case 159:
+case 160:
 YY_RULE_SETUP
-#line 450 "../SqlLexer.lpp"
+#line 451 "../SqlLexer.lpp"
 ; /* ignore white space */
 	YY_BREAK
 /* CONDITION_SQL */
 case YY_STATE_EOF(INITIAL):
 case YY_STATE_EOF(CONDITION_COMMAND):
 case YY_STATE_EOF(CONDITION_SQL):
-#line 454 "../SqlLexer.lpp"
+#line 455 "../SqlLexer.lpp"
 {
   /* All conditions except for mutli-state string extracting conditions. */
   BEGIN(INITIAL);
   return TOKEN_EOF;
 }
 	YY_BREAK
-case 160:
+case 161:
 YY_RULE_SETUP
-#line 460 "../SqlLexer.lpp"
+#line 461 "../SqlLexer.lpp"
 {
   BEGIN(INITIAL);
   quickstep_yyerror(NULL, yyscanner, NULL, "illegal character");
   return TOKEN_LEX_ERROR;
 }
 	YY_BREAK
-case 161:
+case 162:
 YY_RULE_SETUP
-#line 466 "../SqlLexer.lpp"
+#line 467 "../SqlLexer.lpp"
 YY_FATAL_ERROR( "flex scanner jammed" );
 	YY_BREAK
-#line 2460 "SqlLexer_gen.cpp"
+#line 2421 "SqlLexer_gen.cpp"
 
 	case YY_END_OF_BUFFER:
 		{
@@ -2602,7 +2562,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
     struct yyguts_t * yyg = (struct yyguts_t*)yyscanner;
 	char *dest = YY_CURRENT_BUFFER_LVALUE->yy_ch_buf;
 	char *source = yyg->yytext_ptr;
-	int number_to_move, i;
+	yy_size_t number_to_move, i;
 	int ret_val;
 
 	if ( yyg->yy_c_buf_p > &YY_CURRENT_BUFFER_LVALUE->yy_ch_buf[yyg->yy_n_chars + 1] )
@@ -2631,7 +2591,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner)
 	/* Try to read more data. */
 
 	/* First move last chars to start of buffer. */
-	number_to_move = (int) (yyg->yy_c_buf_p - yyg->yytext_ptr - 1);
+	number_to_move = (yy_size_t) (yyg->yy_c_buf_p - yyg->yytext_ptr) - 1;
 
 	f

<TRUNCATED>


[3/8] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/CMakeLists.txt b/types/operations/CMakeLists.txt
index c5dad0f..6d38d80 100644
--- a/types/operations/CMakeLists.txt
+++ b/types/operations/CMakeLists.txt
@@ -25,19 +25,28 @@ QS_PROTOBUF_GENERATE_CPP(types_operations_Operation_proto_srcs
 
 # Declare micro-libs:
 add_library(quickstep_types_operations_Operation Operation.cpp Operation.hpp)
+add_library(quickstep_types_operations_OperationSignature OperationSignature.cpp OperationSignature.hpp)
 add_library(quickstep_types_operations_Operation_proto ${types_operations_Operation_proto_srcs})
 
 # Link dependencies:
 target_link_libraries(quickstep_types_operations_Operation
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_types_operations_OperationSignature
+                      quickstep_types_TypeID
+                      quickstep_types_Type_proto
+                      quickstep_types_operations_Operation_proto
+                      quickstep_utility_HashPair
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_types_operations_Operation_proto
                       quickstep_types_Type_proto
+                      quickstep_types_TypedValue_proto
                       ${PROTOBUF_LIBRARY})
 
 # Module all-in-one library:
 add_library(quickstep_types_operations ../../empty_src.cpp)
 target_link_libraries(quickstep_types_operations
                       quickstep_types_operations_Operation
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_binaryoperations
                       quickstep_types_operations_comparisons

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/Operation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/Operation.hpp b/types/operations/Operation.hpp
index 51178b5..ad7c868 100644
--- a/types/operations/Operation.hpp
+++ b/types/operations/Operation.hpp
@@ -20,6 +20,9 @@
 #ifndef QUICKSTEP_TYPES_OPERATIONS_OPERATION_HPP_
 #define QUICKSTEP_TYPES_OPERATIONS_OPERATION_HPP_
 
+#include <string>
+#include <vector>
+
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -72,7 +75,7 @@ class Operation {
    * @return The human-readable name of this Operation.
    **/
   inline const char* getName() const {
-    return name_;
+    return "NoName";
   }
 
   /**
@@ -81,7 +84,7 @@ class Operation {
    * @return The short name of this Operation.
    **/
   inline const char* getShortName() const {
-    return short_name_;
+    return "NoShortName";
   }
 
   /**
@@ -98,19 +101,12 @@ class Operation {
   }
 
  protected:
-  Operation(const OperationSuperTypeID super_type_id,
-            const char *name,
-            const char *short_name)
-      : super_type_id_(super_type_id),
-        name_(name),
-        short_name_(short_name) {
+  Operation(const OperationSuperTypeID super_type_id)
+      : super_type_id_(super_type_id) {
   }
 
  private:
   const OperationSuperTypeID super_type_id_;
-  const char *name_;
-  const char *short_name_;
-
 
   DISALLOW_COPY_AND_ASSIGN(Operation);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/Operation.proto
----------------------------------------------------------------------
diff --git a/types/operations/Operation.proto b/types/operations/Operation.proto
index d6391f0..006b07c 100644
--- a/types/operations/Operation.proto
+++ b/types/operations/Operation.proto
@@ -20,6 +20,7 @@ syntax = "proto2";
 package quickstep.serialization;
 
 import "types/Type.proto";
+import "types/TypedValue.proto";
 
 message Comparison {
   enum ComparisonID {
@@ -38,48 +39,10 @@ message Comparison {
   required ComparisonID comparison_id = 1;
 }
 
-message UnaryOperation {
-  enum UnaryOperationID {
-    NEGATE = 0;
-    CAST = 1;
-    DATE_EXTRACT = 2;
-    SUBSTRING = 3;
-  }
-
-  required UnaryOperationID operation_id = 1;
-
-  extensions 32 to max;
-}
-
-message CastOperation {
-  extend UnaryOperation {
-    // Required when operation_id = CAST.
-    optional Type target_type = 64;
-  }
-}
-
-message DateExtractOperation {
-  enum Unit {
-    YEAR = 0;
-    MONTH = 1;
-    DAY = 2;
-    HOUR = 3;
-    MINUTE = 4;
-    SECOND = 5;
-  }
-
-  extend UnaryOperation {
-    // Required when operation_id = DATE_EXTRACT.
-    optional Unit unit = 96;
-  }
-}
-
-message SubstringOperation {
-  extend UnaryOperation {
-    // Required when operation_id = SUBSTRING.
-    optional int64 start_position = 100;
-    optional int64 substring_length = 101;
-  }
+message OperationSignature {
+  required string operation_name = 1;
+  repeated TypeID argument_type_ids = 2;
+  required uint32 num_static_arguments = 3;
 }
 
 message BinaryOperation {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/OperationSignature.cpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationSignature.cpp b/types/operations/OperationSignature.cpp
new file mode 100644
index 0000000..6b6c4a6
--- /dev/null
+++ b/types/operations/OperationSignature.cpp
@@ -0,0 +1,91 @@
+/**
+ * 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/OperationSignature.hpp"
+
+#include <cstdint>
+#include <string>
+#include <type_traits>
+#include <vector>
+
+#include "types/TypeID.hpp"
+#include "types/Type.pb.h"
+#include "types/operations/Operation.pb.h"
+
+namespace quickstep {
+
+serialization::OperationSignature OperationSignature::getProto() const {
+  serialization::OperationSignature op_signature;
+
+  op_signature.set_operation_name(operation_name_);
+  for (const TypeID tid : argument_type_ids_) {
+    op_signature.add_argument_type_ids()->CopyFrom(TypeIDFactory::GetProto(tid));
+  }
+  op_signature.set_num_static_arguments(
+      static_cast<std::uint32_t>(num_static_arguments_));
+
+  return op_signature;
+}
+
+OperationSignaturePtr OperationSignature::ReconstructFromProto(
+    const serialization::OperationSignature &proto) {
+  std::vector<TypeID> argument_type_ids;
+  for (int i = 0; i < proto.argument_type_ids_size(); ++i) {
+    argument_type_ids.emplace_back(
+        TypeIDFactory::ReconstructFromProto(proto.argument_type_ids(i)));
+  }
+
+  return Create(proto.operation_name(),
+                argument_type_ids,
+                proto.num_static_arguments());
+}
+
+std::string OperationSignature::toString() const {
+  const std::size_t num_regular_arguments =
+      argument_type_ids_.size() - num_static_arguments_;
+
+  std::string str;
+  str.append(operation_name_);
+  str.push_back('(');
+  for (std::size_t i = 0; i < num_regular_arguments; ++i) {
+    if (i != 0) {
+      str.append(", ");
+    }
+    str.append(
+        kTypeNames[static_cast<std::underlying_type_t<TypeID>>(
+            argument_type_ids_[i])]);
+  }
+  if (num_static_arguments_ > 0) {
+    str.append(", static(");
+    for (std::size_t i = 0; i < num_static_arguments_; ++i) {
+      if (i != 0) {
+        str.append(", ");
+      }
+      str.append(
+          kTypeNames[static_cast<std::underlying_type_t<TypeID>>(
+              argument_type_ids_[i + num_regular_arguments])]);
+    }
+    str.push_back(')');
+  }
+  str.push_back(')');
+
+  return str;
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/OperationSignature.hpp
----------------------------------------------------------------------
diff --git a/types/operations/OperationSignature.hpp b/types/operations/OperationSignature.hpp
new file mode 100644
index 0000000..7020e50
--- /dev/null
+++ b/types/operations/OperationSignature.hpp
@@ -0,0 +1,192 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_OPERATIONS_OPERATION_SIGNATURE_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_OPERATION_SIGNATURE_HPP_
+
+#include <memory>
+#include <string>
+#include <type_traits>
+#include <vector>
+
+#include "types/TypeID.hpp"
+#include "types/operations/Operation.pb.h"
+
+#include "utility/HashPair.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+class OperationSignature;
+typedef std::shared_ptr<const OperationSignature> OperationSignaturePtr;
+
+class OperationSignature {
+ public:
+  serialization::OperationSignature getProto() const;
+
+  static OperationSignaturePtr ReconstructFromProto(
+      const serialization::OperationSignature &proto);
+
+  inline const std::string& getName() const {
+    return operation_name_;
+  }
+
+  inline std::size_t getArity() const {
+    return argument_type_ids_.size();
+  }
+
+  inline std::size_t getNonStaticArity() const {
+    return argument_type_ids_.size() - num_static_arguments_;
+  }
+
+  inline const std::vector<TypeID>& getArgumentTypeIDs() const {
+    return argument_type_ids_;
+  }
+
+  inline std::size_t getNumStaticArguments() const {
+    return num_static_arguments_;
+  }
+
+  inline bool operator==(const OperationSignature &r) const {
+    return operation_name_ == r.operation_name_
+        && argument_type_ids_ == r.argument_type_ids_
+        && num_static_arguments_ == r.num_static_arguments_;
+  }
+
+  inline bool operator<(const OperationSignature &r) const {
+    int cmp_code = operation_name_.compare(r.operation_name_);
+    if (cmp_code != 0) {
+      return cmp_code < 0;
+    }
+    cmp_code = static_cast<int>(getArity() - r.getArity());
+    if (cmp_code != 0) {
+      return cmp_code < 0;
+    }
+    cmp_code = static_cast<int>(num_static_arguments_ - r.num_static_arguments_);
+    if (cmp_code != 0) {
+      return cmp_code < 0;
+    }
+    for (std::size_t i = 0; i < getArity(); ++i) {
+      const auto l_tid =
+          static_cast<std::underlying_type_t<TypeID>>(argument_type_ids_.at(i));
+      const auto r_tid =
+          static_cast<std::underlying_type_t<TypeID>>(r.argument_type_ids_.at(i));
+      if (l_tid != r_tid) {
+        return l_tid < r_tid;
+      }
+    }
+    return false;
+  }
+
+  std::string toString() const;
+
+  static OperationSignaturePtr Create(
+      const std::string &operation_name,
+      const std::vector<TypeID> &argument_type_ids,
+      const std::size_t num_static_arguments) {
+    return OperationSignaturePtr(
+        new OperationSignature(operation_name,
+                               argument_type_ids,
+                               num_static_arguments));
+  }
+
+  static OperationSignaturePtr Create(
+      const std::string &operation_name,
+      const std::vector<TypeID> &regular_argument_type_ids,
+      const std::vector<TypeID> &static_argument_type_ids) {
+    std::vector<TypeID> argument_type_ids = regular_argument_type_ids;
+    argument_type_ids.insert(argument_type_ids.end(),
+                             static_argument_type_ids.begin(),
+                             static_argument_type_ids.end());
+    return OperationSignaturePtr(
+        new OperationSignature(operation_name,
+                               argument_type_ids,
+                               static_argument_type_ids.size()));
+  }
+
+ private:
+  OperationSignature(const std::string &operation_name,
+                     const std::vector<TypeID> &argument_type_ids,
+                     const std::size_t num_static_arguments)
+      : operation_name_(operation_name),
+        argument_type_ids_(argument_type_ids),
+        num_static_arguments_(num_static_arguments) {
+    DCHECK_GE(argument_type_ids_.size(), num_static_arguments_);
+  }
+
+  const std::string operation_name_;
+  const std::vector<TypeID> argument_type_ids_;
+  const std::size_t num_static_arguments_;
+
+  friend struct OperationSignatureHash;
+  friend struct OperationSignatureNumStaticArgumentsGreater;
+
+  DISALLOW_COPY_AND_ASSIGN(OperationSignature);
+};
+
+/**
+ * @brief Implements the equal function for operation signatures.
+ */
+struct OperationSignatureEqual {
+  inline bool operator()(const OperationSignaturePtr &lhs,
+                         const OperationSignaturePtr &rhs) const {
+    return *lhs == *rhs;
+  }
+};
+
+/**
+ * @brief Implements the hash function for operation signatures.
+ */
+struct OperationSignatureHash {
+  inline std::size_t operator()(const OperationSignaturePtr &op_sig) const {
+    std::size_t hash = std::hash<std::string>()(op_sig->operation_name_);
+    for (const TypeID tid : op_sig->argument_type_ids_) {
+      hash = CombineHashes(hash, static_cast<std::size_t>(tid));
+    }
+    hash = CombineHashes(hash, op_sig->num_static_arguments_);
+    return hash;
+  }
+};
+
+/**
+ * @brief Implements the greater function for sorting operation signature
+ *        in descending order of num_static_arguments_.
+ */
+struct OperationSignatureNumStaticArgumentsGreater {
+  inline bool operator()(const OperationSignaturePtr &lhs,
+                         const OperationSignaturePtr &rhs) const {
+    if (lhs->num_static_arguments_ != rhs->num_static_arguments_) {
+      return lhs->num_static_arguments_ > rhs->num_static_arguments_;
+    } else {
+      return *lhs < *rhs;
+    }
+  }
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_OPERATION_SIGNATURE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/ArithmeticBinaryOperators.hpp b/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
index 7224a0c..a9ccd61 100644
--- a/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
+++ b/types/operations/binary_operations/ArithmeticBinaryOperators.hpp
@@ -144,8 +144,10 @@ struct DivideFunctor<float, std::int64_t> {
   }
 };
 
-template <typename LeftArgument, typename RightArgument> struct IntegerModuloFunctor {
-  inline auto operator() (const LeftArgument &left, const RightArgument &right) const -> decltype(left % right) {
+template <typename LeftArgument, typename RightArgument>
+struct IntegerModuloFunctor {
+  inline auto operator() (const LeftArgument &left,
+                          const RightArgument &right) const -> decltype(left % right) {
     return left % right;
   }
 };
@@ -155,9 +157,10 @@ template <typename LeftArgument, typename RightArgument> struct IntegerModuloFun
 // (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)) {
+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);
   }
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/binary_operations/BinaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/binary_operations/BinaryOperation.hpp b/types/operations/binary_operations/BinaryOperation.hpp
index 585a1c6..b3115d6 100644
--- a/types/operations/binary_operations/BinaryOperation.hpp
+++ b/types/operations/binary_operations/BinaryOperation.hpp
@@ -510,11 +510,7 @@ class BinaryOperation : public Operation {
 
  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(Operation::kBinaryOperation),
         operation_id_(operation_id) {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/comparisons/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/CMakeLists.txt b/types/operations/comparisons/CMakeLists.txt
index 321c0f6..dcf8908 100644
--- a/types/operations/comparisons/CMakeLists.txt
+++ b/types/operations/comparisons/CMakeLists.txt
@@ -105,11 +105,13 @@ target_link_libraries(quickstep_types_operations_comparisons_ComparisonID
 target_link_libraries(quickstep_types_operations_comparisons_ComparisonUtil
                       glog
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_types_CharType
                       quickstep_types_DatetimeLit
                       quickstep_types_IntervalLit
                       quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
+                      quickstep_types_VarCharType
                       quickstep_types_containers_Tuple
                       quickstep_types_operations_comparisons_AsciiStringComparators
                       quickstep_types_operations_comparisons_Comparison

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/comparisons/Comparison.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/Comparison.hpp b/types/operations/comparisons/Comparison.hpp
index c300e74..33c853d 100644
--- a/types/operations/comparisons/Comparison.hpp
+++ b/types/operations/comparisons/Comparison.hpp
@@ -606,11 +606,7 @@ class Comparison : public Operation {
 
  protected:
   explicit Comparison(const ComparisonID comparison_id)
-      : Operation(Operation::kComparison,
-                  kComparisonNames[
-                      static_cast<typename std::underlying_type<ComparisonID>::type>(comparison_id)],
-                  kComparisonShortNames[
-                      static_cast<typename std::underlying_type<ComparisonID>::type>(comparison_id)]),
+      : Operation(Operation::kComparison),
         comparison_id_(comparison_id) {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/comparisons/ComparisonUtil.hpp
----------------------------------------------------------------------
diff --git a/types/operations/comparisons/ComparisonUtil.hpp b/types/operations/comparisons/ComparisonUtil.hpp
index 5d868fc..425566d 100644
--- a/types/operations/comparisons/ComparisonUtil.hpp
+++ b/types/operations/comparisons/ComparisonUtil.hpp
@@ -28,11 +28,13 @@
 #include <type_traits>
 
 #include "catalog/CatalogTypedefs.hpp"
+#include "types/CharType.hpp"
 #include "types/DatetimeLit.hpp"
 #include "types/IntervalLit.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
+#include "types/VarCharType.hpp"
 #include "types/containers/Tuple.hpp"
 #include "types/operations/comparisons/AsciiStringComparators.hpp"
 #include "types/operations/comparisons/AsciiStringComparators-inl.hpp"
@@ -153,7 +155,7 @@ auto InvokeOnLessComparatorForTypeIgnoreNullability(const Type &type,
     }
     case kChar: {
       const std::size_t string_length
-          = static_cast<const AsciiStringSuperType&>(type).getStringLength();
+          = static_cast<const CharType&>(type).getStringLength();
       LessAsciiStringUncheckedComparator<false, false, false,
                                          false, false, false>
           comp(string_length, string_length);
@@ -340,11 +342,11 @@ auto InvokeOnLessComparatorForDifferentTypesIgnoreNullability(
     }
     case kChar: {
       const std::size_t left_string_length
-          = static_cast<const AsciiStringSuperType&>(left_type).getStringLength();
+          = static_cast<const CharType&>(left_type).getStringLength();
       switch (right_type.getTypeID()) {
         case kChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const CharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, false, false,
                                                false, false, true>
@@ -364,7 +366,7 @@ auto InvokeOnLessComparatorForDifferentTypesIgnoreNullability(
         }
         case kVarChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const VarCharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, false, false,
                                                false, true, true>
@@ -389,11 +391,11 @@ auto InvokeOnLessComparatorForDifferentTypesIgnoreNullability(
     }
     case kVarChar: {
       const std::size_t left_string_length
-          = static_cast<const AsciiStringSuperType&>(left_type).getStringLength();
+          = static_cast<const VarCharType&>(left_type).getStringLength();
       switch (right_type.getTypeID()) {
         case kChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const CharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, true, false,
                                                false, false, true>
@@ -413,7 +415,7 @@ auto InvokeOnLessComparatorForDifferentTypesIgnoreNullability(
         }
         case kVarChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const VarCharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, true, false,
                                                false, true, true>
@@ -653,11 +655,11 @@ auto InvokeOnBothLessComparatorsForDifferentTypesIgnoreNullability(
     }
     case kChar: {
       const std::size_t left_string_length
-          = static_cast<const AsciiStringSuperType&>(left_type).getStringLength();
+          = static_cast<const CharType&>(left_type).getStringLength();
       switch (right_type.getTypeID()) {
         case kChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const CharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, false, false,
                                                false, false, true>
@@ -686,7 +688,7 @@ auto InvokeOnBothLessComparatorsForDifferentTypesIgnoreNullability(
         }
         case kVarChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const VarCharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, false, false,
                                                false, true, true>
@@ -720,11 +722,11 @@ auto InvokeOnBothLessComparatorsForDifferentTypesIgnoreNullability(
     }
     case kVarChar: {
       const std::size_t left_string_length
-          = static_cast<const AsciiStringSuperType&>(left_type).getStringLength();
+          = static_cast<const VarCharType&>(left_type).getStringLength();
       switch (right_type.getTypeID()) {
         case kChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const CharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, true, false,
                                                false, false, true>
@@ -753,7 +755,7 @@ auto InvokeOnBothLessComparatorsForDifferentTypesIgnoreNullability(
         }
         case kVarChar: {
           const std::size_t right_string_length
-              = static_cast<const AsciiStringSuperType&>(right_type).getStringLength();
+              = static_cast<const VarCharType&>(right_type).getStringLength();
           if (left_string_length < right_string_length) {
             LessAsciiStringUncheckedComparator<false, true, false,
                                                false, true, true>
@@ -991,7 +993,7 @@ inline bool CheckUntypedValuesEqual(const Type &type, const void *left, const vo
     case kDouble:
       return STLLiteralEqual<double>()(left, right);
     case kChar:
-      return STLCharEqual(static_cast<const AsciiStringSuperType&>(type).getStringLength())(left, right);
+      return STLCharEqual(static_cast<const CharType&>(type).getStringLength())(left, right);
     case kVarChar:
       return STLVarCharEqual()(left, right);
     case kDate:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/ArithmeticUnaryOperations.cpp b/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
deleted file mode 100644
index c10d5cf..0000000
--- a/types/operations/unary_operations/ArithmeticUnaryOperations.cpp
+++ /dev/null
@@ -1,145 +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/unary_operations/ArithmeticUnaryOperations.hpp"
-
-#include <string>
-
-#include "types/DatetimeIntervalType.hpp"
-#include "types/DoubleType.hpp"
-#include "types/FloatType.hpp"
-#include "types/IntType.hpp"
-#include "types/LongType.hpp"
-#include "types/Type.hpp"
-#include "types/TypeErrors.hpp"
-#include "types/TypeID.hpp"
-#include "types/TypedValue.hpp"
-#include "types/YearMonthIntervalType.hpp"
-#include "types/operations/unary_operations/ArithmeticUnaryOperators.hpp"
-#include "utility/EqualsAnyConstant.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-bool ArithmeticUnaryOperation::canApplyToType(const Type &type) const {
-  return QUICKSTEP_EQUALS_ANY_CONSTANT(
-      type.getTypeID(),
-      kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-}
-
-const Type* ArithmeticUnaryOperation::resultTypeForArgumentType(const Type &type) const {
-  if (canApplyToType(type)) {
-    return &type;
-  } else {
-    return nullptr;
-  }
-}
-
-const Type* ArithmeticUnaryOperation::pushDownTypeHint(const Type *type_hint) const {
-  if (type_hint == nullptr) {
-    return nullptr;
-  }
-
-  if (canApplyToType(*type_hint)) {
-    return type_hint;
-  } else {
-    return nullptr;
-  }
-}
-
-bool NegateUnaryOperation::resultTypeIsPlausible(const Type &result_type) const {
-  return QUICKSTEP_EQUALS_ANY_CONSTANT(
-      result_type.getTypeID(),
-      kInt, kLong, kFloat, kDouble, kDatetimeInterval, kYearMonthInterval);
-}
-
-TypedValue NegateUnaryOperation::applyToChecked(const TypedValue &argument,
-                                                const Type &argument_type) const {
-  DCHECK_EQ(argument.getTypeID(), argument_type.getTypeID());
-
-  if (argument.isNull()) {
-    return argument;
-  }
-
-  switch (argument.getTypeID()) {
-    case kInt:
-      return TypedValue(-argument.getLiteral<typename IntType::cpptype>());
-    case kLong:
-      return TypedValue(-argument.getLiteral<typename LongType::cpptype>());
-    case kFloat:
-      return TypedValue(-argument.getLiteral<typename FloatType::cpptype>());
-    case kDouble:
-      return TypedValue(-argument.getLiteral<typename DoubleType::cpptype>());
-    case kDatetimeInterval:
-      return TypedValue(-argument.getLiteral<typename DatetimeIntervalType::cpptype>());
-    case kYearMonthInterval:
-      return TypedValue(-argument.getLiteral<typename YearMonthIntervalType::cpptype>());
-    default: {
-      LOG(FATAL) << "Can not apply UnaryOperation " << getName()
-                 << " to argument of type " << argument_type.getName();
-    }
-  }
-}
-
-UncheckedUnaryOperator* NegateUnaryOperation::makeUncheckedUnaryOperatorForType(const Type &type) const {
-  switch (type.getTypeID()) {
-    case kInt:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<IntType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<IntType, false>();
-      }
-    case kLong:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<LongType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<LongType, false>();
-      }
-    case kFloat:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<FloatType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<FloatType, false>();
-      }
-    case kDouble:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<DoubleType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<DoubleType, false>();
-      }
-    case kDatetimeInterval:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<DatetimeIntervalType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<DatetimeIntervalType, false>();
-      }
-    case kYearMonthInterval:
-      if (type.isNullable()) {
-        return new NegateUncheckedUnaryOperator<YearMonthIntervalType, true>();
-      } else {
-        return new NegateUncheckedUnaryOperator<YearMonthIntervalType, false>();
-      }
-    default:
-      throw OperationInapplicableToType(getName(), 1, type.getName().c_str());
-  }
-}
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/ArithmeticUnaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/ArithmeticUnaryOperations.hpp b/types/operations/unary_operations/ArithmeticUnaryOperations.hpp
index 5eed073..089b5a1 100644
--- a/types/operations/unary_operations/ArithmeticUnaryOperations.hpp
+++ b/types/operations/unary_operations/ArithmeticUnaryOperations.hpp
@@ -20,73 +20,45 @@
 #ifndef QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATIONS_HPP_
 #define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATIONS_HPP_
 
-#include "types/TypedValue.hpp"
+#include <string>
+
+#include "types/DatetimeIntervalType.hpp"
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/YearMonthIntervalType.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
-class Type;
-
 /** \addtogroup Types
  *  @{
  */
 
-/**
- * @brief A UnaryOperation which applies to and yields numeric values.
- **/
-class ArithmeticUnaryOperation : public UnaryOperation {
- public:
-  bool canApplyToType(const Type &type) const override;
-
-  const Type* resultTypeForArgumentType(const Type &type) const override;
-
-  const Type* pushDownTypeHint(const Type *type_hint) const override;
-
- protected:
-  explicit ArithmeticUnaryOperation(const UnaryOperationID operation_id)
-      : UnaryOperation(operation_id) {
-  }
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(ArithmeticUnaryOperation);
-};
-
-/**
- * @brief The UnaryOperation for negation.
- **/
-class NegateUnaryOperation : public ArithmeticUnaryOperation {
- public:
-  /**
-   * @brief Get a reference to the singleton instance of this Operation.
-   *
-   * @return A reference to the singleton instance of this Operation.
-   **/
-  static const NegateUnaryOperation& Instance() {
-    static NegateUnaryOperation instance;
-    return instance;
+template <typename ArgumentT, typename ResultT>
+struct NegateFunctor : public UnaryFunctor<ArgumentT, ResultT> {
+  inline typename ResultT::cpptype apply(
+      const typename ArgumentT::cpptype &argument) const {
+    return -argument;
   }
-
-  const Type* fixedNullableResultType() const override {
-    return nullptr;
-  }
-
-  bool resultTypeIsPlausible(const Type &result_type) const override;
-
-  TypedValue applyToChecked(const TypedValue &argument,
-                            const Type &argument_type) const override;
-
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorForType(const Type &type) const override;
-
- private:
-  NegateUnaryOperation()
-      : ArithmeticUnaryOperation(UnaryOperationID::kNegate) {
+  inline std::string getName() const {
+    return "Negate";
   }
-
-  DISALLOW_COPY_AND_ASSIGN(NegateUnaryOperation);
 };
 
+using ArithmeticUnaryFunctorPack = UnaryFunctorPack<
+// negate
+  NegateFunctor<IntType, IntType>,
+  NegateFunctor<LongType, LongType>,
+  NegateFunctor<FloatType, FloatType>,
+  NegateFunctor<DoubleType, DoubleType>,
+  NegateFunctor<DatetimeIntervalType, DatetimeIntervalType>,
+  NegateFunctor<YearMonthIntervalType, YearMonthIntervalType>
+>;
+
 /** @} */
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/ArithmeticUnaryOperators.hpp b/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
deleted file mode 100644
index bf3f7b6..0000000
--- a/types/operations/unary_operations/ArithmeticUnaryOperators.hpp
+++ /dev/null
@@ -1,169 +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_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATORS_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATORS_HPP_
-
-#include <cstddef>
-#include <utility>
-#include <vector>
-
-#include "catalog/CatalogTypedefs.hpp"
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-#include "storage/StorageBlockInfo.hpp"
-#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/unary_operations/UnaryOperation.hpp"
-#include "utility/Macros.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief The UncheckedUnaryOperator for negation.
- **/
-template <class ResultType, bool argument_nullable>
-class NegateUncheckedUnaryOperator : public UncheckedUnaryOperator {
- public:
-  NegateUncheckedUnaryOperator() : UncheckedUnaryOperator() {
-  }
-
-  inline TypedValue applyToTypedValue(const TypedValue &argument) const override {
-    return applyToTypedValueInl(argument);
-  }
-
-  inline TypedValue applyToTypedValueInl(const TypedValue &argument) const {
-    if (argument_nullable && argument.isNull()) {
-      return argument;
-    }
-    return TypedValue(-argument.getLiteral<typename ResultType::cpptype>());
-  }
-
-  inline TypedValue applyToDataPtr(const void *argument) const override {
-    return applyToDataPtrInl(argument);
-  }
-
-  inline TypedValue applyToDataPtrInl(const void *argument) const {
-    if (argument_nullable && (argument == nullptr)) {
-      return TypedValue(ResultType::kStaticTypeID);
-    }
-    return TypedValue(-*static_cast<const typename ResultType::cpptype*>(argument));
-  }
-
-  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override {
-    DCHECK(NativeColumnVector::UsableForType(ResultType::Instance(argument_nullable)));
-    // All arithmetic types (numbers, datetime, and intervals) are usable with
-    // NativeColumnVector, so 'argument' should always be native.
-    DCHECK(argument.isNative());
-    const NativeColumnVector &native_argument = static_cast<const NativeColumnVector&>(argument);
-    NativeColumnVector *result = new NativeColumnVector(
-        ResultType::Instance(argument_nullable),
-        native_argument.size());
-    for (std::size_t pos = 0;
-         pos < native_argument.size();
-         ++pos) {
-      const typename ResultType::cpptype *scalar_arg
-          = static_cast<const typename ResultType::cpptype*>(
-              native_argument.getUntypedValue<argument_nullable>(pos));
-      if (argument_nullable && (scalar_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-            = -(*scalar_arg);
-      }
-    }
-    return result;
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
-                                     const attribute_id argument_attr_id) const override {
-    DCHECK(NativeColumnVector::UsableForType(ResultType::Instance(argument_nullable)));
-    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        accessor,
-        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-      NativeColumnVector *result = new NativeColumnVector(
-          ResultType::Instance(argument_nullable),
-          accessor->getNumTuples());
-      accessor->beginIteration();
-      while (accessor->next()) {
-        const typename ResultType::cpptype *scalar_arg
-            = static_cast<const typename ResultType::cpptype*>(
-                accessor->template getUntypedValue<argument_nullable>(argument_attr_id));
-        if (argument_nullable && (scalar_arg == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-              = -(*scalar_arg);
-        }
-      }
-      return result;
-    });
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  ColumnVector* applyToValueAccessorForJoin(
-      ValueAccessor *accessor,
-      const bool use_left_relation,
-      const attribute_id argument_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    DCHECK(NativeColumnVector::UsableForType(ResultType::Instance(argument_nullable)));
-    NativeColumnVector *result = new NativeColumnVector(ResultType::Instance(argument_nullable),
-                                                        joined_tuple_ids.size());
-    InvokeOnValueAccessorNotAdapter(
-        accessor,
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-      for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-        const typename ResultType::cpptype *scalar_arg
-            = static_cast<const typename ResultType::cpptype*>(
-                accessor->template getUntypedValueAtAbsolutePosition<argument_nullable>(
-                    argument_attr_id,
-                    use_left_relation ? joined_pair.first : joined_pair.second));
-        if (argument_nullable && (scalar_arg == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename ResultType::cpptype*>(result->getPtrForDirectWrite())
-              = -(*scalar_arg);
-        }
-      }
-    });
-    return result;
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(NegateUncheckedUnaryOperator);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_ARITHMETIC_UNARY_OPERATORS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CMakeLists.txt b/types/operations/unary_operations/CMakeLists.txt
index 6e1923a..fca7070 100644
--- a/types/operations/unary_operations/CMakeLists.txt
+++ b/types/operations/unary_operations/CMakeLists.txt
@@ -16,14 +16,26 @@
 # under the License.
 
 # Declare micro-libs:
-add_library(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations ArithmeticUnaryOperations.cpp ArithmeticUnaryOperations.hpp)
-add_library(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperators ../../../empty_src.cpp ArithmeticUnaryOperators.hpp)
-add_library(quickstep_types_operations_unaryoperations_DateExtractOperation DateExtractOperation.cpp DateExtractOperation.hpp)
-add_library(quickstep_types_operations_unaryoperations_NumericCastOperation ../../../empty_src.cpp NumericCastOperation.hpp)
-add_library(quickstep_types_operations_unaryoperations_SubstringOperation SubstringOperation.cpp SubstringOperation.hpp)
+add_library(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
+            ../../../empty_src.cpp
+            ArithmeticUnaryOperations.hpp)
+add_library(quickstep_types_operations_unaryoperations_CMathUnaryOperations
+            ../../../empty_src.cpp
+            CMathUnaryOperations.hpp)
+add_library(quickstep_types_operations_unaryoperations_CastOperation CastOperation.cpp CastOperation.hpp)
+add_library(quickstep_types_operations_unaryoperations_DateExtractOperation
+            DateExtractOperation.cpp
+            DateExtractOperation.hpp)
+add_library(quickstep_types_operations_unaryoperations_SubstringOperation
+            SubstringOperation.cpp
+            SubstringOperation.hpp)
 add_library(quickstep_types_operations_unaryoperations_UnaryOperation UnaryOperation.cpp UnaryOperation.hpp)
-add_library(quickstep_types_operations_unaryoperations_UnaryOperationFactory UnaryOperationFactory.cpp UnaryOperationFactory.hpp)
-add_library(quickstep_types_operations_unaryoperations_UnaryOperationID UnaryOperationID.cpp UnaryOperationID.hpp)
+add_library(quickstep_types_operations_unaryoperations_UnaryOperationFactory
+            UnaryOperationFactory.cpp
+            UnaryOperationFactory.hpp)
+add_library(quickstep_types_operations_unaryoperations_UnaryOperationWrapper
+            ../../../empty_src.cpp
+            UnaryOperationWrapper.hpp)
 
 # Link dependencies:
 target_link_libraries(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
@@ -38,104 +50,118 @@ target_link_libraries(quickstep_types_operations_unaryoperations_ArithmeticUnary
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_types_YearMonthIntervalType
-                      quickstep_types_operations_unaryoperations_ArithmeticUnaryOperators
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_unaryoperations_ArithmeticUnaryOperators
-                      glog
-                      quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_StorageBlockInfo
-                      quickstep_storage_ValueAccessor
-                      quickstep_storage_ValueAccessorUtil
-                      quickstep_types_TypedValue
-                      quickstep_types_containers_ColumnVector
-                      quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_unaryoperations_DateExtractOperation
+target_link_libraries(quickstep_types_operations_unaryoperations_CMathUnaryOperations
+                      quickstep_types_DoubleType
+                      quickstep_types_FloatType
+                      quickstep_types_IntType
+                      quickstep_types_LongType
+                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper
+                      quickstep_utility_TemplateUtil)
+target_link_libraries(quickstep_types_operations_unaryoperations_CastOperation
                       glog
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_StorageBlockInfo
                       quickstep_storage_ValueAccessor
                       quickstep_storage_ValueAccessorUtil
-                      quickstep_types_DatetimeLit
+                      quickstep_types_DoubleType
+                      quickstep_types_FloatType
                       quickstep_types_IntType
                       quickstep_types_LongType
                       quickstep_types_Type
-                      quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_Macros)
-target_link_libraries(quickstep_types_operations_unaryoperations_NumericCastOperation
+target_link_libraries(quickstep_types_operations_unaryoperations_DateExtractOperation
                       glog
                       quickstep_catalog_CatalogTypedefs
+                      quickstep_storage_StorageBlockInfo
                       quickstep_storage_ValueAccessor
                       quickstep_storage_ValueAccessorUtil
-                      quickstep_types_DoubleType
-                      quickstep_types_FloatType
+                      quickstep_types_DatetimeLit
                       quickstep_types_IntType
                       quickstep_types_LongType
                       quickstep_types_Type
+                      quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_types_containers_ColumnVector
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
-                      quickstep_utility_Macros
-                      quickstep_utility_PtrMap)
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_types_operations_unaryoperations_SubstringOperation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_storage_ValueAccessor
                       quickstep_storage_ValueAccessorUtil
+                      quickstep_types_CharType
                       quickstep_types_Type
                       quickstep_types_TypeFactory
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
+                      quickstep_types_VarCharType
                       quickstep_types_containers_ColumnVector
                       quickstep_types_containers_ColumnVectorUtil
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_types_port_strnlen
                       quickstep_utility_HashPair
                       quickstep_utility_Macros
+                      quickstep_utility_StringUtil
                       quickstep_utility_TemplateUtil)
 target_link_libraries(quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_catalog_CatalogTypedefs
-                      quickstep_storage_StorageBlockInfo
                       quickstep_types_TypedValue
                       quickstep_types_operations_Operation
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_Operation_proto
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_types_operations_unaryoperations_UnaryOperationFactory
                       glog
-                      quickstep_types_TypeFactory
-                      quickstep_types_operations_Operation_proto
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
+                      quickstep_types_operations_unaryoperations_CMathUnaryOperations
+                      quickstep_types_operations_unaryoperations_CastOperation
                       quickstep_types_operations_unaryoperations_DateExtractOperation
-                      quickstep_types_operations_unaryoperations_NumericCastOperation
                       quickstep_types_operations_unaryoperations_SubstringOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
+                      quickstep_types_operations_unaryoperations_UnaryOperation
+                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper
+                      quickstep_utility_HashPair
+                      quickstep_utility_Macros
+                      quickstep_utility_StringUtil)
+target_link_libraries(quickstep_types_operations_unaryoperations_UnaryOperationWrapper
+                      glog
+                      quickstep_catalog_CatalogTypedefs
+                      quickstep_storage_ValueAccessor
+                      quickstep_storage_ValueAccessorUtil
+                      quickstep_types_CharType
+                      quickstep_types_IntType
+                      quickstep_types_LongType
+                      quickstep_types_Type
+                      quickstep_types_TypeFactory
+                      quickstep_types_TypeID
+                      quickstep_types_TypedValue
+                      quickstep_types_VarCharType
+                      quickstep_types_containers_ColumnVector
+                      quickstep_types_operations_Operation
+                      quickstep_types_operations_OperationSignature
+                      quickstep_types_operations_Operation_proto
                       quickstep_utility_Macros)
 
 # Module all-in-one library:
 add_library(quickstep_types_operations_unaryoperations ../../../empty_src.cpp)
 target_link_libraries(quickstep_types_operations_unaryoperations
                       quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
-                      quickstep_types_operations_unaryoperations_ArithmeticUnaryOperators
+                      quickstep_types_operations_unaryoperations_CMathUnaryOperations
+                      quickstep_types_operations_unaryoperations_CastOperation
                       quickstep_types_operations_unaryoperations_DateExtractOperation
-                      quickstep_types_operations_unaryoperations_NumericCastOperation
                       quickstep_types_operations_unaryoperations_SubstringOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_types_operations_unaryoperations_UnaryOperationFactory
-                      quickstep_types_operations_unaryoperations_UnaryOperationID)
+                      quickstep_types_operations_unaryoperations_UnaryOperationWrapper)
 
 # Tests:
 
@@ -160,11 +186,10 @@ target_link_libraries(UnaryOperation_tests
                       quickstep_types_containers_ColumnVector
                       quickstep_types_operations_Operation_proto
                       quickstep_types_operations_unaryoperations_ArithmeticUnaryOperations
+                      quickstep_types_operations_unaryoperations_CastOperation
                       quickstep_types_operations_unaryoperations_DateExtractOperation
-                      quickstep_types_operations_unaryoperations_NumericCastOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_types_operations_unaryoperations_UnaryOperationFactory
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_EqualsAnyConstant
                       quickstep_utility_Macros)
 add_test(UnaryOperation_tests UnaryOperation_tests)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/CMathUnaryOperations.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CMathUnaryOperations.hpp b/types/operations/unary_operations/CMathUnaryOperations.hpp
new file mode 100644
index 0000000..7d1625b
--- /dev/null
+++ b/types/operations/unary_operations/CMathUnaryOperations.hpp
@@ -0,0 +1,108 @@
+/**
+ * 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_CMATH_UNARY_OPERATIONS_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_CMATH_UNARY_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/unary_operations/UnaryOperationWrapper.hpp"
+#include "utility/TemplateUtil.hpp"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+template <typename ArgumentT, typename ResultT,
+          typename ResultT::cpptype f(typename ArgumentT::cpptype),
+          typename FunctorNameT>
+struct CMathUnaryFunctorWrapper {
+  struct Implemenation : public UnaryFunctor<ArgumentT, ResultT> {
+    inline typename ResultT::cpptype apply(
+        const typename ArgumentT::cpptype &argument) const {
+      return f(argument);
+    }
+    inline std::string getName() const {
+      return FunctorNameT::ToString();
+    }
+  };
+
+  typedef Implemenation type;
+};
+
+template <typename ArgumentT, typename ReturnT,
+          typename ReturnT::cpptype f(typename ArgumentT::cpptype),
+          typename FunctorNameT>
+using CMathUnaryFunctor =
+    typename CMathUnaryFunctorWrapper<ArgumentT, ReturnT, f, FunctorNameT>::type;
+
+using CMathUnaryFunctorPack = UnaryFunctorPack<
+// abs
+    CMathUnaryFunctor<IntType, IntType,
+                      std::abs, StringLiteral<'a','b','s'>>,
+    CMathUnaryFunctor<LongType, LongType,
+                      std::abs, StringLiteral<'a','b','s'>>,
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::fabs, StringLiteral<'a','b','s'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::fabs, StringLiteral<'a','b','s'>>,
+// sqrt
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::sqrt, StringLiteral<'s','q','r','t'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::sqrt, StringLiteral<'s','q','r','t'>>,
+// exp
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::exp, StringLiteral<'e','x','p'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::exp, StringLiteral<'e','x','p'>>,
+// log
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::log, StringLiteral<'l','o','g'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::log, StringLiteral<'l','o','g'>>,
+// ceil
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::ceil, StringLiteral<'c','e','i','l'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::ceil, StringLiteral<'c','e','i','l'>>,
+// floor
+    CMathUnaryFunctor<FloatType, FloatType,
+                      std::floor, StringLiteral<'f','l','o','o','r'>>,
+    CMathUnaryFunctor<DoubleType, DoubleType,
+                      std::floor, StringLiteral<'f','l','o','o','r'>>,
+// round
+    CMathUnaryFunctor<FloatType, LongType,
+                      std::llround, StringLiteral<'r','o','u','n','d'>>,
+    CMathUnaryFunctor<DoubleType, LongType,
+                      std::llround, StringLiteral<'r','o','u','n','d'>>
+>;
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_CMATH_UNARY_OPERATIONS_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/CastOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CastOperation.cpp b/types/operations/unary_operations/CastOperation.cpp
new file mode 100644
index 0000000..ef5b8f8
--- /dev/null
+++ b/types/operations/unary_operations/CastOperation.cpp
@@ -0,0 +1,282 @@
+/**
+ * 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/unary_operations/CastOperation.hpp"
+
+#include <algorithm>
+#include <map>
+#include <cstdint>
+#include <cstdlib>
+#include <string>
+#include <vector>
+
+#include "types/CharType.hpp"
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeUtil.hpp"
+#include "types/TypedValue.hpp"
+#include "types/VarCharType.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
+#include "types/port/strnlen.hpp"
+#include "utility/EqualsAnyConstant.hpp"
+
+namespace quickstep {
+
+namespace {
+
+template <typename ArgumentT, typename ResultT>
+struct NumericCastToNumericFunctor
+    : public UnaryFunctor<ArgumentT, ResultT> {
+  inline typename ResultT::cpptype apply(
+      const typename ArgumentT::cpptype &argument) const {
+    return static_cast<typename ResultT::cpptype>(argument);
+  }
+};
+
+template <typename ArgumentT, typename ResultT>
+class NumericCastToAsciiStringFunctor : public UnaryFunctor<ArgumentT, ResultT> {
+ public:
+  explicit NumericCastToAsciiStringFunctor(const std::size_t max_length)
+      : max_length_(max_length) {}
+
+  inline void apply(const typename ArgumentT::cpptype &argument, void *result) const {
+    std::string str = std::to_string(argument);
+    const std::size_t str_len = str.length();
+
+    if (str_len < max_length_) {
+      std::memcpy(result, str.c_str(), str_len);
+      static_cast<char *>(result)[str_len] = 0;
+    } else {
+      std::memcpy(result, str.c_str(), max_length_);
+    }
+  }
+
+  inline TypedValue apply(const typename ArgumentT::cpptype &argument) const {
+    std::string str = std::to_string(argument);
+    const std::size_t len = std::min(str.length(), max_length_);
+    const std::size_t buf_len = len + 1;
+
+    char *buf = static_cast<char *>(std::malloc(buf_len));
+    std::memcpy(buf, str.c_str(), len);
+    buf[len] = 0;
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, buf_len);
+  }
+
+ private:
+  const std::size_t max_length_;
+};
+
+
+template <typename ResultCppType>
+ResultCppType CastStringToNumericImpl(const char *str);
+
+template <>
+int CastStringToNumericImpl(const char *str) {
+  return std::atoi(str);
+}
+template <>
+float CastStringToNumericImpl(const char *str) {
+  return static_cast<float>(std::atof(str));
+}
+template <>
+std::int64_t CastStringToNumericImpl(const char *str) {
+  return std::atoll(str);
+}
+template <>
+double CastStringToNumericImpl(const char *str) {
+  return std::atof(str);
+}
+
+template <typename ArgumentT, typename ResultT,
+          typename ResultT::cpptype f(const char*)>
+struct AsciiStringCastToNumericFunctor
+    : public UnaryFunctor<ArgumentT, ResultT> {
+  explicit AsciiStringCastToNumericFunctor(const std::size_t max_length)
+      : max_length_(max_length) {}
+
+  inline typename ResultT::cpptype apply(const TypedValue &argument) const {
+    return f(static_cast<const char*>(argument.getDataPtr()));
+  }
+
+  inline typename ResultT::cpptype apply(const void *argument) const {
+    const char *str = static_cast<const char*>(argument);
+    const std::string value(str, strnlen(str, max_length_));
+    return f(value.c_str());
+  }
+
+ private:
+  const std::size_t max_length_;
+};
+
+template <typename ArgumentT, typename ResultT>
+struct AsciiStringCastToAsciiStringFunctor
+    : public UnaryFunctor<ArgumentT, ResultT> {
+  explicit AsciiStringCastToAsciiStringFunctor(const std::size_t max_string_length)
+      : max_string_length_(max_string_length) {}
+
+  inline void apply(const void *argument, void *result) const {
+    std::memcpy(result, argument, max_string_length_);
+  }
+
+  inline void apply(const TypedValue &argument, void *result) const {
+    std::memcpy(result,
+                argument.getOutOfLineData(),
+                std::min(argument.getDataSize(), max_string_length_));
+  }
+
+  inline TypedValue apply(const void *argument) const {
+    const std::size_t len =
+        strnlen(static_cast<const char*>(argument), max_string_length_);
+
+    char *buf = static_cast<char *>(std::malloc(len+1));
+    std::memcpy(buf, argument, len);
+    buf[len] = 0;
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, len+1);
+  }
+
+  inline TypedValue apply(const TypedValue &argument) const {
+    const std::size_t len =
+        std::min(argument.getDataSize() - 1, max_string_length_);
+
+    char *buf = static_cast<char *>(std::malloc(len+1));
+    std::memcpy(buf, argument.getDataPtr(), len);
+    buf[len] = 0;
+    return TypedValue::CreateWithOwnedData(kVarChar, buf, len+1);
+  }
+
+ private:
+  const std::size_t max_string_length_;
+};
+
+}  // namespace
+
+const re2::RE2 CastOperation::kTypePattern("([a-z]+)(\\(([0-9]+)\\))?");
+
+const std::map<std::string, TypeID> CastOperation::kNameToTypeIDMap = {
+    { "int",     kInt },
+    { "long",    kLong },
+    { "float",   kFloat },
+    { "double",  kDouble },
+    { "char",    kChar },
+    { "varchar", kVarChar }
+};
+
+UncheckedUnaryOperator* CastOperation::makeUncheckedUnaryOperator(
+    const Type &type,
+    const std::vector<TypedValue> &static_arguments) const {
+  const Type *result_type = getResultType(type, static_arguments);
+  DCHECK(result_type != nullptr);
+
+  const TypeID argument_type_id = type.getTypeID();
+  const TypeID result_type_id = result_type->getTypeID();
+
+  if (QUICKSTEP_EQUALS_ANY_CONSTANT(argument_type_id, kInt, kLong, kFloat, kDouble)) {
+    return InvokeOnTypeID<TypeIDSelectorEqualsAny<kInt, kLong, kFloat, kDouble>>(
+        argument_type_id,
+        [&](auto arg_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+      using ArgumentT = typename TypeGenerator<decltype(arg_tid)::value>::type;
+
+      switch (result_type_id) {
+        case kInt:  // Fall through
+        case kLong:
+        case kFloat:
+        case kDouble: {
+          return InvokeOnTypeID<TypeIDSelectorEqualsAny<kInt, kLong, kFloat, kDouble>>(
+              result_type_id,
+              [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+            using ResultT = typename TypeGenerator<decltype(result_tid)::value>::type;
+
+            return new UncheckedUnaryOperatorWrapperCodegen<
+                NumericCastToNumericFunctor<ArgumentT, ResultT>>(type, *result_type);
+          });
+        }
+        case kChar:  // Fall through
+        case kVarChar: {
+          return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
+              result_type_id,
+              [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+            using ResultT = typename TypeGenerator<decltype(result_tid)::value>::type;
+
+            return new UncheckedUnaryOperatorWrapperCodegen<
+                 NumericCastToAsciiStringFunctor<ArgumentT, ResultT>>(
+                     type, *result_type,
+                     static_cast<const ResultT*>(result_type)->getStringLength());
+          });
+        }
+        default:
+          LOG(FATAL) << "Unexpected result type " << result_type->getName()
+                     << " in CastOperation::makeUncheckedUnaryOperator "
+                     << "for argument type " << type.getName();
+      }
+    });
+  } else if (QUICKSTEP_EQUALS_ANY_CONSTANT(argument_type_id, kChar, kVarChar)) {
+    return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
+        argument_type_id,
+        [&](auto arg_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+      using ArgumentT = typename TypeGenerator<decltype(arg_tid)::value>::type;
+
+      switch (result_type_id) {
+        case kInt:  // Fall through
+        case kLong:
+        case kFloat:
+        case kDouble: {
+          return InvokeOnTypeID<TypeIDSelectorEqualsAny<kInt, kLong, kFloat, kDouble>>(
+              result_type_id,
+              [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+            using ResultT = typename TypeGenerator<decltype(result_tid)::value>::type;
+
+            return new UncheckedUnaryOperatorWrapperCodegen<
+                AsciiStringCastToNumericFunctor<
+                    ArgumentT, ResultT,
+                    CastStringToNumericImpl<typename ResultT::cpptype>>>(
+                        type, *result_type,
+                        static_cast<const ArgumentT&>(type).getStringLength());
+          });
+        }
+        case kChar:  // Fall through
+        case kVarChar: {
+          return InvokeOnTypeID<TypeIDSelectorEqualsAny<kChar, kVarChar>>(
+              result_type_id,
+              [&](auto result_tid) -> UncheckedUnaryOperator* {  // NOLINT(build/c++11)
+            using ResultT = typename TypeGenerator<decltype(result_tid)::value>::type;
+
+            return new UncheckedUnaryOperatorWrapperCodegen<
+                 AsciiStringCastToAsciiStringFunctor<ArgumentT, ResultT>>(
+                     type, *result_type,
+                     std::min(static_cast<const ArgumentT&>(type).getStringLength(),
+                              static_cast<const ResultT*>(result_type)->getStringLength()));
+          });
+        }
+        default:
+          LOG(FATAL) << "Unexpected result type " << result_type->getName()
+                     << " in CastOperation::makeUncheckedUnaryOperator "
+                     << "for argument type " << type.getName();
+      }
+    });
+  }
+
+  LOG(FATAL) << "Unexpected argument type in "
+             << "CastOperation::makeUncheckedUnaryOperator: "
+             << result_type->getName();
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/CastOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/CastOperation.hpp b/types/operations/unary_operations/CastOperation.hpp
new file mode 100644
index 0000000..23dbec2
--- /dev/null
+++ b/types/operations/unary_operations/CastOperation.hpp
@@ -0,0 +1,149 @@
+/**
+ * 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_NUMERIC_CAST_OPERATION_HPP_
+#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_
+
+#include <cstddef>
+#include <map>
+#include <string>
+#include <utility>
+
+#include "types/IntType.hpp"
+#include "types/Type.hpp"
+#include "types/TypeFactory.hpp"
+#include "types/TypeID.hpp"
+#include "types/TypedValue.hpp"
+#include "types/operations/unary_operations/UnaryOperation.hpp"
+#include "utility/Macros.hpp"
+#include "utility/StringUtil.hpp"
+
+#include "glog/logging.h"
+
+#include "re2/stringpiece.h"
+#include "re2/re2.h"
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+/**
+ * @brief UnaryOperation for CAST.
+ */
+class CastOperation : public UnaryOperation {
+ public:
+  CastOperation() {}
+
+  std::string getName() const override {
+    return "Cast";
+  }
+
+  std::string getShortName() const override {
+    return "Cast";
+  }
+
+  std::vector<OperationSignaturePtr> getSignatures() const override {
+    const std::vector<TypeID> source_type_ids =
+        { kInt, kLong, kFloat, kDouble, kChar, kVarChar };
+    const std::vector<TypeID> target_type_carrier = { kVarChar };
+
+    std::vector<OperationSignaturePtr> signatures;
+    for (const TypeID source_type_id : source_type_ids) {
+      signatures.emplace_back(
+          OperationSignature::Create(getName(), {source_type_id}, target_type_carrier));
+    }
+    return signatures;
+  }
+
+  bool canApplyTo(const Type &type,
+                  const std::vector<TypedValue> &static_arguments,
+                  std::string *message) const override {
+    DCHECK_EQ(1u, static_arguments.size());
+    if (parseType(static_arguments.front()) == nullptr) {
+      *message = "Invalid target type for CAST";
+      return false;
+    }
+    return true;
+  }
+
+  const Type* getResultType(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK_EQ(1u, static_arguments.size());
+    const Type *target_type = parseType(static_arguments.front());
+
+    DCHECK(target_type != nullptr);
+    if (type.isNullable()) {
+      return &target_type->getNullableVersion();
+    } else {
+      return target_type;
+    }
+  }
+
+  UncheckedUnaryOperator* makeUncheckedUnaryOperator(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override;
+
+ private:
+  static const Type* parseType(const TypedValue &type_arg) {
+    DCHECK(type_arg.getTypeID() == kVarChar);
+    const std::string type_str =
+        ToLower(std::string(static_cast<const char*>(type_arg.getOutOfLineData())));
+
+    const re2::StringPiece type_piece(type_str);
+    std::string type_name;
+    std::string length_str;
+    if (!re2::RE2::FullMatch(type_piece,
+                             kTypePattern,
+                             &type_name,
+                             static_cast<void *>(nullptr),
+                             &length_str)) {
+      return nullptr;
+    }
+
+    auto it = kNameToTypeIDMap.find(type_name);
+    if (it == kNameToTypeIDMap.end()) {
+      return nullptr;
+    }
+
+    if (length_str.empty()) {
+      return &TypeFactory::GetType(it->second);
+    } else {
+      TypedValue length_value;
+      if (IntType::InstanceNonNullable().parseValueFromString(length_str, &length_value)) {
+        return &TypeFactory::GetType(
+            it->second, static_cast<std::size_t>(length_value.getLiteral<int>()));
+      }
+    }
+    return nullptr;
+  }
+
+  static const re2::RE2 kTypePattern;
+  static const std::map<std::string, TypeID> kNameToTypeIDMap;
+
+  DISALLOW_COPY_AND_ASSIGN(CastOperation);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_


[4/8] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index f1876b8..18a3aa0 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -79,103 +79,104 @@ extern int quickstep_yydebug;
     TOKEN_CSB_TREE = 289,
     TOKEN_BY = 290,
     TOKEN_CASE = 291,
-    TOKEN_CHARACTER = 292,
-    TOKEN_CHECK = 293,
-    TOKEN_COLUMN = 294,
-    TOKEN_CONSTRAINT = 295,
-    TOKEN_COPY = 296,
-    TOKEN_CREATE = 297,
-    TOKEN_CURRENT = 298,
-    TOKEN_DATE = 299,
-    TOKEN_DATETIME = 300,
-    TOKEN_DAY = 301,
-    TOKEN_DECIMAL = 302,
-    TOKEN_DEFAULT = 303,
-    TOKEN_DELETE = 304,
-    TOKEN_DELIMITER = 305,
-    TOKEN_DESC = 306,
-    TOKEN_DISTINCT = 307,
-    TOKEN_DOUBLE = 308,
-    TOKEN_DROP = 309,
-    TOKEN_ELSE = 310,
-    TOKEN_END = 311,
-    TOKEN_ESCAPE_STRINGS = 312,
-    TOKEN_EXISTS = 313,
-    TOKEN_EXTRACT = 314,
-    TOKEN_FALSE = 315,
-    TOKEN_FIRST = 316,
-    TOKEN_FLOAT = 317,
-    TOKEN_FOLLOWING = 318,
-    TOKEN_FOR = 319,
-    TOKEN_FOREIGN = 320,
-    TOKEN_FROM = 321,
-    TOKEN_FULL = 322,
-    TOKEN_GROUP = 323,
-    TOKEN_HASH = 324,
-    TOKEN_HAVING = 325,
-    TOKEN_HOUR = 326,
-    TOKEN_IN = 327,
-    TOKEN_INDEX = 328,
-    TOKEN_INNER = 329,
-    TOKEN_INSERT = 330,
-    TOKEN_INTEGER = 331,
-    TOKEN_INTERVAL = 332,
-    TOKEN_INTO = 333,
-    TOKEN_JOIN = 334,
-    TOKEN_KEY = 335,
-    TOKEN_LAST = 336,
-    TOKEN_LEFT = 337,
-    TOKEN_LIMIT = 338,
-    TOKEN_LONG = 339,
-    TOKEN_MINUTE = 340,
-    TOKEN_MONTH = 341,
-    TOKEN_NULL = 342,
-    TOKEN_NULLS = 343,
-    TOKEN_OFF = 344,
-    TOKEN_ON = 345,
-    TOKEN_ORDER = 346,
-    TOKEN_OUTER = 347,
-    TOKEN_OVER = 348,
-    TOKEN_PARTITION = 349,
-    TOKEN_PARTITIONS = 350,
-    TOKEN_PERCENT = 351,
-    TOKEN_PRECEDING = 352,
-    TOKEN_PRIMARY = 353,
-    TOKEN_PRIORITY = 354,
-    TOKEN_QUIT = 355,
-    TOKEN_RANGE = 356,
-    TOKEN_REAL = 357,
-    TOKEN_REFERENCES = 358,
-    TOKEN_RIGHT = 359,
-    TOKEN_ROW = 360,
-    TOKEN_ROW_DELIMITER = 361,
-    TOKEN_ROWS = 362,
-    TOKEN_SECOND = 363,
-    TOKEN_SELECT = 364,
-    TOKEN_SET = 365,
-    TOKEN_SMA = 366,
-    TOKEN_SMALLINT = 367,
-    TOKEN_SUBSTRING = 368,
-    TOKEN_TABLE = 369,
-    TOKEN_THEN = 370,
-    TOKEN_TIME = 371,
-    TOKEN_TIMESTAMP = 372,
-    TOKEN_TRUE = 373,
-    TOKEN_TUPLESAMPLE = 374,
-    TOKEN_UNBOUNDED = 375,
-    TOKEN_UNIQUE = 376,
-    TOKEN_UPDATE = 377,
-    TOKEN_USING = 378,
-    TOKEN_VALUES = 379,
-    TOKEN_VARCHAR = 380,
-    TOKEN_WHEN = 381,
-    TOKEN_WHERE = 382,
-    TOKEN_WINDOW = 383,
-    TOKEN_WITH = 384,
-    TOKEN_YEAR = 385,
-    TOKEN_YEARMONTH = 386,
-    TOKEN_EOF = 387,
-    TOKEN_LEX_ERROR = 388
+    TOKEN_CAST = 292,
+    TOKEN_CHARACTER = 293,
+    TOKEN_CHECK = 294,
+    TOKEN_COLUMN = 295,
+    TOKEN_CONSTRAINT = 296,
+    TOKEN_COPY = 297,
+    TOKEN_CREATE = 298,
+    TOKEN_CURRENT = 299,
+    TOKEN_DATE = 300,
+    TOKEN_DATETIME = 301,
+    TOKEN_DAY = 302,
+    TOKEN_DECIMAL = 303,
+    TOKEN_DEFAULT = 304,
+    TOKEN_DELETE = 305,
+    TOKEN_DELIMITER = 306,
+    TOKEN_DESC = 307,
+    TOKEN_DISTINCT = 308,
+    TOKEN_DOUBLE = 309,
+    TOKEN_DROP = 310,
+    TOKEN_ELSE = 311,
+    TOKEN_END = 312,
+    TOKEN_ESCAPE_STRINGS = 313,
+    TOKEN_EXISTS = 314,
+    TOKEN_EXTRACT = 315,
+    TOKEN_FALSE = 316,
+    TOKEN_FIRST = 317,
+    TOKEN_FLOAT = 318,
+    TOKEN_FOLLOWING = 319,
+    TOKEN_FOR = 320,
+    TOKEN_FOREIGN = 321,
+    TOKEN_FROM = 322,
+    TOKEN_FULL = 323,
+    TOKEN_GROUP = 324,
+    TOKEN_HASH = 325,
+    TOKEN_HAVING = 326,
+    TOKEN_HOUR = 327,
+    TOKEN_IN = 328,
+    TOKEN_INDEX = 329,
+    TOKEN_INNER = 330,
+    TOKEN_INSERT = 331,
+    TOKEN_INTEGER = 332,
+    TOKEN_INTERVAL = 333,
+    TOKEN_INTO = 334,
+    TOKEN_JOIN = 335,
+    TOKEN_KEY = 336,
+    TOKEN_LAST = 337,
+    TOKEN_LEFT = 338,
+    TOKEN_LIMIT = 339,
+    TOKEN_LONG = 340,
+    TOKEN_MINUTE = 341,
+    TOKEN_MONTH = 342,
+    TOKEN_NULL = 343,
+    TOKEN_NULLS = 344,
+    TOKEN_OFF = 345,
+    TOKEN_ON = 346,
+    TOKEN_ORDER = 347,
+    TOKEN_OUTER = 348,
+    TOKEN_OVER = 349,
+    TOKEN_PARTITION = 350,
+    TOKEN_PARTITIONS = 351,
+    TOKEN_PERCENT = 352,
+    TOKEN_PRECEDING = 353,
+    TOKEN_PRIMARY = 354,
+    TOKEN_PRIORITY = 355,
+    TOKEN_QUIT = 356,
+    TOKEN_RANGE = 357,
+    TOKEN_REAL = 358,
+    TOKEN_REFERENCES = 359,
+    TOKEN_RIGHT = 360,
+    TOKEN_ROW = 361,
+    TOKEN_ROW_DELIMITER = 362,
+    TOKEN_ROWS = 363,
+    TOKEN_SECOND = 364,
+    TOKEN_SELECT = 365,
+    TOKEN_SET = 366,
+    TOKEN_SMA = 367,
+    TOKEN_SMALLINT = 368,
+    TOKEN_SUBSTRING = 369,
+    TOKEN_TABLE = 370,
+    TOKEN_THEN = 371,
+    TOKEN_TIME = 372,
+    TOKEN_TIMESTAMP = 373,
+    TOKEN_TRUE = 374,
+    TOKEN_TUPLESAMPLE = 375,
+    TOKEN_UNBOUNDED = 376,
+    TOKEN_UNIQUE = 377,
+    TOKEN_UPDATE = 378,
+    TOKEN_USING = 379,
+    TOKEN_VALUES = 380,
+    TOKEN_VARCHAR = 381,
+    TOKEN_WHEN = 382,
+    TOKEN_WHERE = 383,
+    TOKEN_WINDOW = 384,
+    TOKEN_WITH = 385,
+    TOKEN_YEAR = 386,
+    TOKEN_YEARMONTH = 387,
+    TOKEN_EOF = 388,
+    TOKEN_LEX_ERROR = 389
   };
 #endif
 
@@ -184,7 +185,7 @@ extern int quickstep_yydebug;
 
 union YYSTYPE
 {
-#line 120 "../SqlParser.ypp" /* yacc.c:1915  */
+#line 117 "../SqlParser.ypp" /* yacc.c:1915  */
 
   quickstep::ParseString *string_value_;
 
@@ -255,7 +256,7 @@ union YYSTYPE
   quickstep::ParseStatementQuit *quit_statement_;
 
   const quickstep::Comparison *comparison_;
-  const quickstep::UnaryOperation *unary_operation_;
+  quickstep::ParseString *unary_operation_;
   const quickstep::BinaryOperation *binary_operation_;
 
   quickstep::ParseFunctionCall *function_call_;
@@ -284,7 +285,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 288 "SqlParser_gen.hpp" /* yacc.c:1915  */
+#line 289 "SqlParser_gen.hpp" /* yacc.c:1915  */
 };
 
 typedef union YYSTYPE YYSTYPE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/query_optimizer/LogicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/LogicalGenerator.cpp b/query_optimizer/LogicalGenerator.cpp
index abeca53..49481d3 100644
--- a/query_optimizer/LogicalGenerator.cpp
+++ b/query_optimizer/LogicalGenerator.cpp
@@ -53,7 +53,8 @@ L::LogicalPtr LogicalGenerator::generatePlan(
   resolver::Resolver resolver(catalog_database, optimizer_context_);
   DVLOG(4) << "Parse tree:\n" << parse_statement.toString();
   logical_plan_ = resolver.resolve(parse_statement);
-  DVLOG(4) << "Initial logical plan:\n" << logical_plan_->toString();
+  std::cerr << "Initial logical plan:\n" << logical_plan_->toString();
+//  exit(0);
 
   optimizePlan();
   DVLOG(4) << "Optimized logical plan:\n" << logical_plan_->toString();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/query_optimizer/expressions/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/CMakeLists.txt b/query_optimizer/expressions/CMakeLists.txt
index 35fac90..d7ce686 100644
--- a/query_optimizer/expressions/CMakeLists.txt
+++ b/query_optimizer/expressions/CMakeLists.txt
@@ -104,7 +104,7 @@ target_link_libraries(quickstep_queryoptimizer_expressions_Cast
                       quickstep_queryoptimizer_expressions_PatternMatcher
                       quickstep_queryoptimizer_expressions_Scalar
                       quickstep_types_Type
-                      quickstep_types_operations_unaryoperations_NumericCastOperation
+                      quickstep_types_operations_unaryoperations_CastOperation
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_ComparisonExpression
                       glog
@@ -299,8 +299,9 @@ target_link_libraries(quickstep_queryoptimizer_expressions_UnaryExpression
                       quickstep_queryoptimizer_expressions_ExpressionType
                       quickstep_queryoptimizer_expressions_PatternMatcher
                       quickstep_queryoptimizer_expressions_Scalar
+                      quickstep_queryoptimizer_expressions_ScalarLiteral
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_unaryoperations_UnaryOperation
-                      quickstep_types_operations_unaryoperations_UnaryOperationID
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_expressions_WindowAggregateFunction
                       glog

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/query_optimizer/expressions/Cast.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/Cast.cpp b/query_optimizer/expressions/Cast.cpp
index c0813c5..7180aa3 100644
--- a/query_optimizer/expressions/Cast.cpp
+++ b/query_optimizer/expressions/Cast.cpp
@@ -32,7 +32,7 @@
 #include "query_optimizer/expressions/PatternMatcher.hpp"
 #include "query_optimizer/expressions/Scalar.hpp"
 #include "types/Type.hpp"
-#include "types/operations/unary_operations/NumericCastOperation.hpp"
+#include "types/operations/unary_operations/CastOperation.hpp"
 
 #include "glog/logging.h"
 
@@ -51,8 +51,9 @@ ExpressionPtr Cast::copyWithNewChildren(
 
 ::quickstep::Scalar *Cast::concretize(
     const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const {
-  return new ::quickstep::ScalarUnaryExpression(::quickstep::NumericCastOperation::Instance(target_type_),
-                                                operand_->concretize(substitution_map));
+//  return new ::quickstep::ScalarUnaryExpression(::quickstep::NumericCastOperation::Instance(target_type_),
+//                                                operand_->concretize(substitution_map));
+  return nullptr;
 }
 
 void Cast::getFieldStringItems(
@@ -62,11 +63,11 @@ void Cast::getFieldStringItems(
     std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
     std::vector<std::string> *container_child_field_names,
     std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
-  inline_field_names->push_back("target_type");
-  inline_field_values->push_back(target_type_.getName());
-
-  non_container_child_field_names->push_back("operand");
-  non_container_child_fields->push_back(operand_);
+//  inline_field_names->push_back("target_type");
+//  inline_field_values->push_back(target_type_.getName());
+//
+//  non_container_child_field_names->push_back("operand");
+//  non_container_child_fields->push_back(operand_);
 }
 
 }  // namespace expressions

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/query_optimizer/expressions/UnaryExpression.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/UnaryExpression.cpp b/query_optimizer/expressions/UnaryExpression.cpp
index b0fff62..d7bde71 100644
--- a/query_optimizer/expressions/UnaryExpression.cpp
+++ b/query_optimizer/expressions/UnaryExpression.cpp
@@ -29,8 +29,8 @@
 #include "query_optimizer/expressions/Expression.hpp"
 #include "query_optimizer/expressions/PatternMatcher.hpp"
 #include "query_optimizer/expressions/Scalar.hpp"
+#include "query_optimizer/expressions/ScalarLiteral.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
 
 #include "glog/logging.h"
 
@@ -39,7 +39,7 @@ namespace optimizer {
 namespace expressions {
 
 std::string UnaryExpression::getName() const {
-  return operation_.getName();
+  return operation_->getName();
 }
 
 ExpressionPtr UnaryExpression::copyWithNewChildren(
@@ -47,13 +47,20 @@ ExpressionPtr UnaryExpression::copyWithNewChildren(
   DCHECK_EQ(new_children.size(), children().size());
   DCHECK(SomeScalar::Matches(new_children[0]));
   return UnaryExpression::Create(
-      operation_, std::static_pointer_cast<const Scalar>(new_children[0]));
+      op_signature_,
+      operation_,
+      std::static_pointer_cast<const Scalar>(new_children[0]),
+      static_arguments_,
+      static_argument_types_);
 }
 
 ::quickstep::Scalar* UnaryExpression::concretize(
     const std::unordered_map<ExprId, const CatalogAttribute*> &substitution_map) const {
   return new ::quickstep::ScalarUnaryExpression(
-      operation_, operand_->concretize(substitution_map));
+      op_signature_,
+      operation_,
+      operand_->concretize(substitution_map),
+      static_arguments_);
 }
 
 void UnaryExpression::getFieldStringItems(
@@ -63,8 +70,21 @@ void UnaryExpression::getFieldStringItems(
     std::vector<OptimizerTreeBaseNodePtr> *non_container_child_fields,
     std::vector<std::string> *container_child_field_names,
     std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const {
-  non_container_child_field_names->push_back("Operand");
-  non_container_child_fields->push_back(operand_);
+  inline_field_names->emplace_back("op_signature");
+  inline_field_values->emplace_back(op_signature_->toString());
+
+  non_container_child_field_names->emplace_back("operand");
+  non_container_child_fields->emplace_back(operand_);
+
+  if (!static_arguments_->empty()) {
+    container_child_field_names->emplace_back("static_arguments");
+    container_child_fields->emplace_back();
+    for (std::size_t i = 0; i < static_arguments_->size(); ++i) {
+      container_child_fields->back().emplace_back(
+          ScalarLiteral::Create(static_arguments_->at(i),
+                                *static_argument_types_->at(i)));
+    }
+  }
 }
 
 }  // namespace expressions

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/query_optimizer/expressions/UnaryExpression.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/expressions/UnaryExpression.hpp b/query_optimizer/expressions/UnaryExpression.hpp
index c4542d0..c08519f 100644
--- a/query_optimizer/expressions/UnaryExpression.hpp
+++ b/query_optimizer/expressions/UnaryExpression.hpp
@@ -30,6 +30,7 @@
 #include "query_optimizer/expressions/Expression.hpp"
 #include "query_optimizer/expressions/ExpressionType.hpp"
 #include "query_optimizer/expressions/Scalar.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
 #include "utility/Macros.hpp"
 
@@ -64,7 +65,7 @@ class UnaryExpression : public Scalar {
   /**
    * @return The unary operator.
    */
-  const UnaryOperation& operation() const { return operation_; }
+  const UnaryOperationPtr& operation() const { return operation_; }
 
   /**
    * @return The operand of the unary operator.
@@ -72,7 +73,7 @@ class UnaryExpression : public Scalar {
   const ScalarPtr& operand() const { return operand_; }
 
   const Type& getValueType() const override {
-    return *(operation_.resultTypeForArgumentType(operand_->getValueType()));
+    return result_type_;
   }
 
   ExpressionPtr copyWithNewChildren(
@@ -93,9 +94,18 @@ class UnaryExpression : public Scalar {
    * @return An immutable UnaryExpression that applies the operation to the
    *         operand.
    */
-  static UnaryExpressionPtr Create(const UnaryOperation &operation,
-                                   const ScalarPtr &operand) {
-    return UnaryExpressionPtr(new UnaryExpression(operation, operand));
+  static UnaryExpressionPtr Create(
+      const OperationSignaturePtr &op_signature,
+      const UnaryOperationPtr &operation,
+      const ScalarPtr &operand,
+      const std::shared_ptr<const std::vector<TypedValue>> &static_arguments,
+      const std::shared_ptr<const std::vector<const Type*>> &static_argument_types) {
+    return UnaryExpressionPtr(
+        new UnaryExpression(op_signature,
+                            operation,
+                            operand,
+                            static_arguments,
+                            static_argument_types));
   }
 
  protected:
@@ -108,15 +118,27 @@ class UnaryExpression : public Scalar {
       std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const override;
 
  private:
-  UnaryExpression(const UnaryOperation &operation,
-                  const ScalarPtr &operand)
-      : operation_(operation), operand_(operand) {
-    DCHECK(operation_.canApplyToType(operand_->getValueType())) << toString();
+  UnaryExpression(const OperationSignaturePtr &op_signature,
+                  const UnaryOperationPtr &operation,
+                  const ScalarPtr &operand,
+                  const std::shared_ptr<const std::vector<TypedValue>> &static_arguments,
+                  const std::shared_ptr<const std::vector<const Type*>> &static_argument_types)
+      : op_signature_(op_signature),
+        operation_(operation),
+        operand_(operand),
+        static_arguments_(static_arguments),
+        static_argument_types_(static_argument_types),
+        result_type_(*(operation_->getResultType(operand_->getValueType(), *static_arguments_))) {
+    DCHECK(operation_->canApplyTo(operand_->getValueType(), *static_arguments)) << toString();
     addChild(operand);
   }
 
-  const UnaryOperation &operation_;
-  ScalarPtr operand_;
+  const OperationSignaturePtr op_signature_;
+  const UnaryOperationPtr operation_;
+  const ScalarPtr operand_;
+  const std::shared_ptr<const std::vector<TypedValue>> static_arguments_;
+  const std::shared_ptr<const std::vector<const Type*>> static_argument_types_;
+  const Type &result_type_;
 
   DISALLOW_COPY_AND_ASSIGN(UnaryExpression);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/query_optimizer/resolver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/CMakeLists.txt b/query_optimizer/resolver/CMakeLists.txt
index a34273e..27497a2 100644
--- a/query_optimizer/resolver/CMakeLists.txt
+++ b/query_optimizer/resolver/CMakeLists.txt
@@ -121,15 +121,16 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_storage_StorageConstants
                       quickstep_types_IntType
                       quickstep_types_Type
+                      quickstep_types_TypeUtil
                       quickstep_types_TypedValue
                       quickstep_types_TypeFactory
+                      quickstep_types_operations_OperationSignature
                       quickstep_types_operations_binaryoperations_BinaryOperation
                       quickstep_types_operations_comparisons_Comparison
                       quickstep_types_operations_comparisons_ComparisonFactory
                       quickstep_types_operations_comparisons_ComparisonID
-                      quickstep_types_operations_unaryoperations_DateExtractOperation
-                      quickstep_types_operations_unaryoperations_SubstringOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
+                      quickstep_types_operations_unaryoperations_UnaryOperationFactory
                       quickstep_utility_Macros
                       quickstep_utility_PtrList
                       quickstep_utility_PtrVector

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index df589fd..1741409 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -22,7 +22,6 @@
 #include <algorithm>
 #include <map>
 #include <memory>
-#include <set>
 #include <string>
 #include <unordered_map>
 #include <unordered_set>
@@ -116,15 +115,16 @@
 #include "storage/StorageConstants.hpp"
 #include "types/IntType.hpp"
 #include "types/Type.hpp"
-#include "types/TypedValue.hpp"
 #include "types/TypeFactory.hpp"
+#include "types/TypeUtil.hpp"
+#include "types/TypedValue.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "types/operations/binary_operations/BinaryOperation.hpp"
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
-#include "types/operations/unary_operations/DateExtractOperation.hpp"
-#include "types/operations/unary_operations/SubstringOperation.hpp"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
+#include "types/operations/unary_operations/UnaryOperationFactory.hpp"
 #include "utility/PtrList.hpp"
 #include "utility/PtrVector.hpp"
 #include "utility/SqlError.hpp"
@@ -142,6 +142,26 @@ namespace E = ::quickstep::optimizer::expressions;
 namespace L = ::quickstep::optimizer::logical;
 namespace S = ::quickstep::serialization;
 
+namespace {
+
+attribute_id GetAttributeIdFromName(const PtrList<ParseAttributeDefinition> &attribute_definition_list,
+                                    const std::string &attribute_name) {
+  const std::string lower_attribute_name = ToLower(attribute_name);
+
+  attribute_id attr_id = 0;
+  for (const ParseAttributeDefinition &attribute_definition : attribute_definition_list) {
+    if (lower_attribute_name == ToLower(attribute_definition.name()->value())) {
+      return attr_id;
+    }
+
+    ++attr_id;
+  }
+
+  return kInvalidAttributeID;
+}
+
+}  // namespace
+
 struct Resolver::ExpressionResolutionInfo {
   /**
    * @brief Constructs an ExpressionResolutionInfo that disallows aggregate
@@ -490,26 +510,6 @@ L::LogicalPtr Resolver::resolveCreateTable(
   return L::CreateTable::Create(relation_name, attributes, block_properties, partition_scheme_header_proto);
 }
 
-namespace {
-
-attribute_id GetAttributeIdFromName(const PtrList<ParseAttributeDefinition> &attribute_definition_list,
-                                    const std::string &attribute_name) {
-  const std::string lower_attribute_name = ToLower(attribute_name);
-
-  attribute_id attr_id = 0;
-  for (const ParseAttributeDefinition &attribute_definition : attribute_definition_list) {
-    if (lower_attribute_name == ToLower(attribute_definition.name()->value())) {
-      return attr_id;
-    }
-
-    ++attr_id;
-  }
-
-  return kInvalidAttributeID;
-}
-
-}  // namespace
-
 StorageBlockLayoutDescription* Resolver::resolveBlockProperties(
     const ParseStatementCreateTable &create_table_statement) {
   const ParseBlockProperties *block_properties
@@ -2247,57 +2247,6 @@ E::ScalarPtr Resolver::resolveExpression(
           type_hint,
           expression_resolution_info);
     }
-    case ParseExpression::kUnaryExpression: {
-      const ParseUnaryExpression &parse_unary_expr =
-          static_cast<const ParseUnaryExpression&>(parse_expression);
-
-      E::ScalarPtr argument = resolveExpression(
-          *parse_unary_expr.operand(),
-          parse_unary_expr.op().pushDownTypeHint(type_hint),
-          expression_resolution_info);
-
-      // If the argument is a NULL of unknown Type, try to resolve result Type
-      // of this UnaryExpression as follows:
-      //
-      //     1. If the UnaryExpression can only return one type, then the
-      //        result is a NULL of that type.
-      //     2. If there is a type hint for the UnaryExpression's result, and
-      //        it is possible for the UnaryExpression to return the hinted
-      //        type, then the result is a NULL of that type.
-      //     3. Otherwise, the result is a NULL of unknown type (i.e.
-      //        NullType).
-      //
-      // NOTE(chasseur): As with binary expressions above, step #3 does not
-      // always completely capture information about what types the NULL result
-      // can take on, since NullType is implicitly convertable to any Type.
-      if (argument->getValueType().getTypeID() == kNullType) {
-        const Type *fixed_result_type = parse_unary_expr.op().fixedNullableResultType();
-        if (fixed_result_type != nullptr) {
-          return E::ScalarLiteral::Create(fixed_result_type->makeNullValue(),
-                                          *fixed_result_type);
-        }
-
-        if (type_hint != nullptr) {
-          const Type &nullable_type_hint = type_hint->getNullableVersion();
-          if (parse_unary_expr.op().resultTypeIsPlausible(nullable_type_hint)) {
-            return E::ScalarLiteral::Create(nullable_type_hint.makeNullValue(),
-                                            nullable_type_hint);
-          }
-        }
-
-        const Type &null_type = TypeFactory::GetType(kNullType, true);
-        return E::ScalarLiteral::Create(null_type.makeNullValue(),
-                                        null_type);
-      }
-
-      if (!parse_unary_expr.op().canApplyToType(argument->getValueType())) {
-        THROW_SQL_ERROR_AT(&parse_unary_expr)
-            << "Can not apply unary operation \"" << parse_unary_expr.op().getName()
-            << "\" to argument of type " << argument->getValueType().getName();
-      }
-
-      return E::UnaryExpression::Create(parse_unary_expr.op(), argument);
-    }
     case ParseExpression::kFunctionCall: {
       return resolveFunctionCall(
           static_cast<const ParseFunctionCall&>(parse_expression),
@@ -2311,75 +2260,6 @@ E::ScalarPtr Resolver::resolveExpression(
           expression_resolution_info,
           true /* has_single_column */);
     }
-    case ParseExpression::kExtract: {
-      const ParseExtractFunction &parse_extract =
-          static_cast<const ParseExtractFunction&>(parse_expression);
-
-      const ParseString &extract_field = *parse_extract.extract_field();
-      const std::string lowered_unit = ToLower(extract_field.value());
-      DateExtractUnit extract_unit;
-      if (lowered_unit == "year") {
-        extract_unit = DateExtractUnit::kYear;
-      } else if (lowered_unit == "month") {
-        extract_unit = DateExtractUnit::kMonth;
-      } else if (lowered_unit == "day") {
-        extract_unit = DateExtractUnit::kDay;
-      } else if (lowered_unit == "hour") {
-        extract_unit = DateExtractUnit::kHour;
-      } else if (lowered_unit == "minute") {
-        extract_unit = DateExtractUnit::kMinute;
-      } else if (lowered_unit == "second") {
-        extract_unit = DateExtractUnit::kSecond;
-      } else {
-        THROW_SQL_ERROR_AT(&extract_field)
-            << "Invalid extract unit: " << extract_field.value();
-      }
-
-      const DateExtractOperation &op = DateExtractOperation::Instance(extract_unit);
-      const E::ScalarPtr argument = resolveExpression(
-          *parse_extract.date_expression(),
-          op.pushDownTypeHint(type_hint),
-          expression_resolution_info);
-
-      if (!op.canApplyToType(argument->getValueType())) {
-        THROW_SQL_ERROR_AT(parse_extract.date_expression())
-            << "Can not extract from argument of type: "
-            << argument->getValueType().getName();
-      }
-
-      return E::UnaryExpression::Create(op, argument);
-    }
-    case ParseExpression::kSubstring: {
-      const ParseSubstringFunction &parse_substring =
-          static_cast<const ParseSubstringFunction&>(parse_expression);
-
-      // Validate start position and substring length.
-      if (parse_substring.start_position() <= 0) {
-        THROW_SQL_ERROR_AT(&parse_expression)
-            << "The start position must be greater than 0";
-      }
-      if (parse_substring.length() <= 0) {
-        THROW_SQL_ERROR_AT(&parse_expression)
-            << "The substring length must be greater than 0";
-      }
-
-      // Convert 1-base position to 0-base position
-      const std::size_t zero_base_start_position = parse_substring.start_position() - 1;
-      const SubstringOperation &op =
-          SubstringOperation::Instance(zero_base_start_position,
-                                       parse_substring.length());
-
-      const E::ScalarPtr argument =
-          resolveExpression(*parse_substring.operand(),
-                            op.pushDownTypeHint(type_hint),
-                            expression_resolution_info);
-      if (!op.canApplyToType(argument->getValueType())) {
-        THROW_SQL_ERROR_AT(&parse_substring)
-            << "Can not apply substring function to argument of type "
-            << argument->getValueType().getName();
-      }
-      return E::UnaryExpression::Create(op, argument);
-    }
     default:
       LOG(FATAL) << "Unknown scalar type: "
                  << parse_expression.getExpressionType();
@@ -2639,6 +2519,126 @@ E::ScalarPtr Resolver::resolveSimpleCaseExpression(
                                *result_data_type);
 }
 
+E::ScalarPtr Resolver::resolveScalarFunction(
+    const ParseFunctionCall &parse_function_call,
+    const std::string &function_name,
+    const std::vector<E::ScalarPtr> &resolved_arguments,
+    ExpressionResolutionInfo *expression_resolution_info) {
+  const std::size_t arity = resolved_arguments.size();
+
+  std::vector<const Type*> argument_types;
+  std::vector<TypeID> argument_type_ids;
+  std::size_t first_static_argument_position = 0;
+  for (std::size_t i = 0; i < arity; ++i) {
+    const E::ScalarPtr &argument = resolved_arguments[i];
+    if (argument->getExpressionType() != E::ExpressionType::kScalarLiteral) {
+      first_static_argument_position = i + 1;
+    }
+
+    const Type &type = argument->getValueType();
+    argument_types.emplace_back(&type);
+    argument_type_ids.emplace_back(type.getTypeID());
+  }
+
+  std::vector<TypedValue> maximal_static_arguments;
+  for (std::size_t i = first_static_argument_position; i < arity; ++i) {
+    maximal_static_arguments.emplace_back(
+        std::static_pointer_cast<const E::ScalarLiteral>(
+            resolved_arguments[i])->value());
+    DCHECK(maximal_static_arguments.back().getTypeID() == argument_type_ids[i]);
+  }
+  const std::size_t max_num_static_arguments = maximal_static_arguments.size();
+
+  const UnaryOperationFactory &factory = UnaryOperationFactory::Instance();
+
+  // First, check exact matching of name + non static types, possibly safe
+  // coercion of static arguments.
+  const auto &op_sig_candidates = factory.getUnaryOperations(function_name, arity);
+  for (const OperationSignaturePtr &op_sig : op_sig_candidates) {
+    const std::size_t op_num_static_arguments = op_sig->getNumStaticArguments();
+    if (op_num_static_arguments > max_num_static_arguments) {
+      continue;
+    }
+
+    bool is_match = true;
+    for (std::size_t i = 0; i < arity - op_num_static_arguments; ++i) {
+      if (op_sig->getArgumentTypeIDs().at(i) != argument_type_ids[i]) {
+        is_match = false;
+        break;
+      }
+    }
+    if (!is_match) {
+      continue;
+    }
+
+    std::vector<TypedValue> coerced_static_arguments;
+    std::vector<const Type*> coerced_static_argument_types;
+    bool is_coercible = true;
+    for (std::size_t i = arity - op_num_static_arguments; i < arity; ++i) {
+      const Type &arg_type = *argument_types.at(i);
+      const TypedValue &arg_value =
+          maximal_static_arguments.at(i - first_static_argument_position);
+      const TypeID expected_type_id = op_sig->getArgumentTypeIDs().at(i);
+      if (arg_type.getTypeID() == expected_type_id) {
+        coerced_static_arguments.emplace_back(arg_value);
+        coerced_static_argument_types.emplace_back(&arg_type);
+      } else {
+        const Type *expected_type = nullptr;
+        if (TypeUtil::IsParameterized(expected_type_id)) {
+          // TODO: refactor type system to make this coercion extensible.
+          if (expected_type_id == kChar && arg_type.getTypeID() == kVarChar) {
+            expected_type = &TypeFactory::GetType(
+                expected_type_id, arg_type.maximumByteLength() - 1);
+          } else if (expected_type_id == kVarChar && arg_type.getTypeID() == kChar) {
+            expected_type = &TypeFactory::GetType(
+                expected_type_id, arg_type.maximumByteLength() + 1);
+          }
+        } else {
+          expected_type = &TypeFactory::GetType(expected_type_id);
+        }
+        if (expected_type != nullptr && expected_type->isSafelyCoercibleFrom(arg_type)) {
+          coerced_static_arguments.emplace_back(
+              expected_type->coerceValue(arg_value, arg_type));
+          coerced_static_argument_types.emplace_back(expected_type);
+        } else {
+          is_coercible = false;
+          break;
+        }
+      }
+    }
+    if (!is_coercible) {
+      continue;
+    }
+
+    // TODO: switch on operation non-static arity here.
+    CHECK_EQ(1u, arity - op_num_static_arguments);
+
+    const UnaryOperationPtr op = factory.getUnaryOperation(op_sig);
+    const auto static_arguments =
+        std::make_shared<const std::vector<TypedValue>>(
+            std::move(coerced_static_arguments));
+    const auto static_argument_types =
+        std::make_shared<const std::vector<const Type*>>(
+            std::move(coerced_static_argument_types));
+
+    std::string message;
+    if (op->canApplyTo(*argument_types.front(), *static_arguments, &message)) {
+      return E::UnaryExpression::Create(op_sig,
+                                        op,
+                                        resolved_arguments.front(),
+                                        static_arguments,
+                                        static_argument_types);
+    } else {
+      THROW_SQL_ERROR_AT(&parse_function_call) << message;
+    }
+  }
+
+  THROW_SQL_ERROR_AT(&parse_function_call)
+      << "Cannot resolve scalar function";
+
+  return nullptr;
+}
+
 // TODO(chasseur): For now this only handles resolving aggregate functions. In
 // the future it should be extended to resolve scalar functions as well.
 // TODO(Shixuan): This will handle resolving window aggregation function as well,
@@ -2659,8 +2659,7 @@ E::ScalarPtr Resolver::resolveFunctionCall(
   }
 
   std::vector<E::ScalarPtr> resolved_arguments;
-  const PtrList<ParseExpression> *unresolved_arguments =
-      parse_function_call.arguments();
+  const PtrList<ParseExpression> *unresolved_arguments = parse_function_call.arguments();
   // The first aggregate function and window aggregate function in the arguments.
   const ParseTreeNode *first_aggregate_function = nullptr;
   const ParseTreeNode *first_window_aggregate_function = nullptr;
@@ -2668,7 +2667,7 @@ E::ScalarPtr Resolver::resolveFunctionCall(
     for (const ParseExpression &unresolved_argument : *unresolved_arguments) {
       ExpressionResolutionInfo expr_resolution_info(
           *expression_resolution_info);
-      resolved_arguments.push_back(
+      resolved_arguments.emplace_back(
           resolveExpression(unresolved_argument,
                             nullptr,  // No Type hint.
                             &expr_resolution_info));
@@ -2687,6 +2686,17 @@ E::ScalarPtr Resolver::resolveFunctionCall(
     }
   }
 
+  if (UnaryOperationFactory::Instance().hasUnaryOperation(function_name,
+                                                          resolved_arguments.size())) {
+    E::ScalarPtr scalar = resolveScalarFunction(parse_function_call,
+                                                function_name,
+                                                resolved_arguments,
+                                                expression_resolution_info);
+    expression_resolution_info->parse_aggregate_expression = first_aggregate_function;
+    expression_resolution_info->parse_window_aggregate_expression = first_window_aggregate_function;
+    return scalar;
+  }
+
   if (count_star && !resolved_arguments.empty()) {
     THROW_SQL_ERROR_AT(&parse_function_call)
         << "COUNT aggregate has both star (*) and non-star arguments.";

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/query_optimizer/resolver/Resolver.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.hpp b/query_optimizer/resolver/Resolver.hpp
index a3d0833..ea26e0c 100644
--- a/query_optimizer/resolver/Resolver.hpp
+++ b/query_optimizer/resolver/Resolver.hpp
@@ -452,9 +452,6 @@ class Resolver {
    * @brief Resolves a function call. For a non-scalar function, the returned
    *        expression is an AttributeReference to the actual resolved expression.
    *
-   * @note This currently only handles resolving aggregate functions and window
-   *       aggregate functions.
-   *
    * @param parse_function_call The function call to be resolved.
    * @param expression_resolution_info Resolution info that contains the name
    *                                   resolver and info to be updated after
@@ -465,6 +462,12 @@ class Resolver {
       const ParseFunctionCall &parse_function_call,
       ExpressionResolutionInfo *expression_resolution_info);
 
+  expressions::ScalarPtr resolveScalarFunction(
+      const ParseFunctionCall &parse_function_call,
+      const std::string &function_name,
+      const std::vector<expressions::ScalarPtr> &resolved_arguments,
+      ExpressionResolutionInfo *expression_resolution_info);
+
   /**
    * @brief Resolves a window aggregate function.
    *

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/types/CMakeLists.txt b/types/CMakeLists.txt
index 769187b..1fac11a 100644
--- a/types/CMakeLists.txt
+++ b/types/CMakeLists.txt
@@ -49,9 +49,11 @@ add_library(quickstep_types_NullType ../empty_src.cpp NullType.hpp)
 add_library(quickstep_types_NumericSuperType ../empty_src.cpp NumericSuperType.hpp)
 add_library(quickstep_types_NumericTypeUnifier ../empty_src.cpp NumericTypeUnifier.hpp)
 add_library(quickstep_types_Type Type.cpp Type.hpp)
+add_library(quickstep_types_TypeConcept ../empty_src.cpp TypeConcept.hpp)
 add_library(quickstep_types_TypeErrors ../empty_src.cpp TypeErrors.hpp)
 add_library(quickstep_types_TypeFactory TypeFactory.cpp TypeFactory.hpp)
 add_library(quickstep_types_TypeID TypeID.cpp TypeID.hpp)
+add_library(quickstep_types_TypeUtil ../empty_src.cpp TypeUtil.hpp)
 add_library(quickstep_types_Type_proto ${types_Type_proto_srcs})
 add_library(quickstep_types_TypedValue TypedValue.cpp TypedValue.hpp)
 add_library(quickstep_types_TypedValue_proto ${types_TypedValue_proto_srcs})
@@ -165,6 +167,8 @@ target_link_libraries(quickstep_types_Type
                       quickstep_types_TypeID
                       quickstep_types_TypedValue
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_types_TypeConcept
+                      glog)
 target_link_libraries(quickstep_types_TypeFactory
                       glog
                       quickstep_types_CharType
@@ -182,6 +186,25 @@ target_link_libraries(quickstep_types_TypeFactory
                       quickstep_types_VarCharType
                       quickstep_types_YearMonthIntervalType
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_types_TypeID
+                      quickstep_types_Type_proto
+                      quickstep_utility_Macros)
+target_link_libraries(quickstep_types_TypeUtil
+                      quickstep_types_CharType
+                      quickstep_types_DateType
+                      quickstep_types_DatetimeIntervalType
+                      quickstep_types_DatetimeType
+                      quickstep_types_DoubleType
+                      quickstep_types_FloatType
+                      quickstep_types_IntType
+                      quickstep_types_LongType
+                      quickstep_types_NullType
+                      quickstep_types_Type
+                      quickstep_types_TypeID
+                      quickstep_types_Type_proto
+                      quickstep_types_VarCharType
+                      quickstep_types_YearMonthIntervalType
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_types_TypedValue
                       farmhash
                       glog
@@ -238,6 +261,7 @@ target_link_libraries(quickstep_types
                       quickstep_types_NumericSuperType
                       quickstep_types_NumericTypeUnifier
                       quickstep_types_Type
+                      quickstep_types_TypeUtil
                       quickstep_types_Type_proto
                       quickstep_types_TypeErrors
                       quickstep_types_TypeFactory

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/CharType.cpp
----------------------------------------------------------------------
diff --git a/types/CharType.cpp b/types/CharType.cpp
index 591c038..bc05711 100644
--- a/types/CharType.cpp
+++ b/types/CharType.cpp
@@ -66,8 +66,7 @@ const CharType& CharType::InstanceFromProto(const serialization::Type &proto) {
 
 serialization::Type CharType::getProto() const {
   serialization::Type proto;
-  proto.set_type_id(serialization::Type::CHAR);
-
+  proto.mutable_type_id()->CopyFrom(TypeIDFactory::GetProto(type_id_));
   proto.set_nullable(nullable_);
 
   proto.SetExtension(serialization::CharType::length, length_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/CharType.hpp
----------------------------------------------------------------------
diff --git a/types/CharType.hpp b/types/CharType.hpp
index c7321f4..e591953 100644
--- a/types/CharType.hpp
+++ b/types/CharType.hpp
@@ -43,7 +43,7 @@ namespace quickstep {
  *       represented WITHOUT a null-terminator character. Any strings shorter
  *       than the maximum length will have a null-terminator.
  **/
-class CharType : public AsciiStringSuperType {
+class CharType : public AsciiStringSuperType<kChar, kNonNativeInline> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this Type
@@ -134,7 +134,7 @@ class CharType : public AsciiStringSuperType {
 
  private:
   CharType(const std::size_t length, const bool nullable)
-      : AsciiStringSuperType(kChar, nullable, length, length, length) {
+      : AsciiStringSuperType<kChar, kNonNativeInline>(nullable, length, length, length) {
   }
 
   template <bool nullable_internal>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/DateType.hpp
----------------------------------------------------------------------
diff --git a/types/DateType.hpp b/types/DateType.hpp
index 07225d5..a39a28c 100644
--- a/types/DateType.hpp
+++ b/types/DateType.hpp
@@ -40,12 +40,8 @@ class TypedValue;
 /**
  * @brief A type representing the date.
  **/
-class DateType : public Type {
+class DateType : public TypeConcept<kDate, false, kNativeEmbedded, DateLit> {
  public:
-  typedef DateLit cpptype;
-
-  static const TypeID kStaticTypeID = kDate;
-
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
    *        Type.
@@ -124,7 +120,9 @@ class DateType : public Type {
 
  private:
   explicit DateType(const bool nullable)
-      : Type(Type::kOther, kDate, nullable, sizeof(DateLit), sizeof(DateLit)) {
+      : TypeConcept<kDate, false, kNativeEmbedded, DateLit>(
+             Type::kOther, kStaticTypeID, nullable,
+             sizeof(DateLit), sizeof(DateLit)) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(DateType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/DatetimeIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeIntervalType.hpp b/types/DatetimeIntervalType.hpp
index 005cb31..3b22471 100644
--- a/types/DatetimeIntervalType.hpp
+++ b/types/DatetimeIntervalType.hpp
@@ -40,12 +40,9 @@ namespace quickstep {
 /**
  * @brief A type representing the datetime interval.
  **/
-class DatetimeIntervalType : public Type {
+class DatetimeIntervalType :
+    public TypeConcept<kDatetimeInterval, false, kNativeEmbedded, DatetimeIntervalLit> {
  public:
-  typedef DatetimeIntervalLit cpptype;
-
-  static const TypeID kStaticTypeID = kDatetimeInterval;
-
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
    *        Type.
@@ -116,7 +113,9 @@ class DatetimeIntervalType : public Type {
 
  private:
   explicit DatetimeIntervalType(const bool nullable)
-      : Type(Type::kOther, kDatetimeInterval, nullable, sizeof(DatetimeIntervalLit), sizeof(DatetimeIntervalLit)) {
+      : TypeConcept<kDatetimeInterval, false, kNativeEmbedded, DatetimeIntervalLit>(
+             Type::kOther, kStaticTypeID, nullable,
+             sizeof(DatetimeIntervalLit), sizeof(DatetimeIntervalLit)) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(DatetimeIntervalType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/DatetimeLit.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeLit.hpp b/types/DatetimeLit.hpp
index 58c852f..db887eb 100644
--- a/types/DatetimeLit.hpp
+++ b/types/DatetimeLit.hpp
@@ -99,6 +99,10 @@ struct DateLit {
   inline std::int32_t monthField() const {
     return static_cast<std::int32_t>(month);
   }
+
+  inline std::int32_t dayField() const {
+    return static_cast<std::int32_t>(day);
+  }
 };
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/DatetimeType.hpp
----------------------------------------------------------------------
diff --git a/types/DatetimeType.hpp b/types/DatetimeType.hpp
index aad536a..b6742f9 100644
--- a/types/DatetimeType.hpp
+++ b/types/DatetimeType.hpp
@@ -40,12 +40,8 @@ class TypedValue;
 /**
  * @brief A type representing the datetime.
  **/
-class DatetimeType : public Type {
+class DatetimeType : public TypeConcept<kDatetime, false, kNativeEmbedded, DatetimeLit> {
  public:
-  typedef DatetimeLit cpptype;
-
-  static const TypeID kStaticTypeID = kDatetime;
-
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
    *        Type.
@@ -132,7 +128,9 @@ class DatetimeType : public Type {
 
  private:
   explicit DatetimeType(const bool nullable)
-      : Type(Type::kOther, kDatetime, nullable, sizeof(DatetimeLit), sizeof(DatetimeLit)) {
+      : TypeConcept<kDatetime, false, kNativeEmbedded, quickstep::DatetimeLit>(
+             Type::kOther, kStaticTypeID, nullable,
+             sizeof(DatetimeLit), sizeof(DatetimeLit)) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(DatetimeType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/DoubleType.cpp
----------------------------------------------------------------------
diff --git a/types/DoubleType.cpp b/types/DoubleType.cpp
index 6a7914c..52645f8 100644
--- a/types/DoubleType.cpp
+++ b/types/DoubleType.cpp
@@ -41,8 +41,6 @@ using std::snprintf;
 
 namespace quickstep {
 
-const TypeID DoubleType::kStaticTypeID = kDouble;
-
 bool DoubleType::isSafelyCoercibleFrom(const Type &original_type) const {
   QUICKSTEP_NULL_COERCIBILITY_CHECK();
   return QUICKSTEP_EQUALS_ANY_CONSTANT(original_type.getTypeID(),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/DoubleType.hpp
----------------------------------------------------------------------
diff --git a/types/DoubleType.hpp b/types/DoubleType.hpp
index b4175b0..4d1dd7e 100644
--- a/types/DoubleType.hpp
+++ b/types/DoubleType.hpp
@@ -40,10 +40,8 @@ class Type;
 /**
  * @brief A type representing a double-precision floating-point number.
  **/
-class DoubleType : public NumericSuperType<double> {
+class DoubleType : public NumericSuperType<kDouble, double> {
  public:
-  static const TypeID kStaticTypeID;
-
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
    *        Type.
@@ -122,7 +120,7 @@ class DoubleType : public NumericSuperType<double> {
               // exponent never takes more than 3 base-10 digits to represent.
 
   explicit DoubleType(const bool nullable)
-      : NumericSuperType<double>(kDouble, nullable) {
+      : NumericSuperType<kDouble, double>(nullable) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(DoubleType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/FloatType.cpp
----------------------------------------------------------------------
diff --git a/types/FloatType.cpp b/types/FloatType.cpp
index e904f29..3aa2eb9 100644
--- a/types/FloatType.cpp
+++ b/types/FloatType.cpp
@@ -41,8 +41,6 @@ using std::snprintf;
 
 namespace quickstep {
 
-const TypeID FloatType::kStaticTypeID = kFloat;
-
 bool FloatType::isSafelyCoercibleFrom(const Type &original_type) const {
   QUICKSTEP_NULL_COERCIBILITY_CHECK();
   return QUICKSTEP_EQUALS_ANY_CONSTANT(original_type.getTypeID(),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/FloatType.hpp
----------------------------------------------------------------------
diff --git a/types/FloatType.hpp b/types/FloatType.hpp
index 2a156e1..ee9c45c 100644
--- a/types/FloatType.hpp
+++ b/types/FloatType.hpp
@@ -40,10 +40,8 @@ class Type;
 /**
  * @brief A type representing a single-precision floating-point number.
  **/
-class FloatType : public NumericSuperType<float> {
+class FloatType : public NumericSuperType<kFloat, float> {
  public:
-  static const TypeID kStaticTypeID;
-
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
    *        Type.
@@ -122,7 +120,7 @@ class FloatType : public NumericSuperType<float> {
               // never takes more than 2 base-10 digits to represent.
 
   explicit FloatType(const bool nullable)
-      : NumericSuperType<float>(kFloat, nullable) {
+      : NumericSuperType<kFloat, float>(nullable) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(FloatType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/IntType.cpp
----------------------------------------------------------------------
diff --git a/types/IntType.cpp b/types/IntType.cpp
index 9781675..8b7fbb1 100644
--- a/types/IntType.cpp
+++ b/types/IntType.cpp
@@ -32,8 +32,6 @@
 
 namespace quickstep {
 
-const TypeID IntType::kStaticTypeID = kInt;
-
 bool IntType::isSafelyCoercibleFrom(const Type &original_type) const {
   QUICKSTEP_NULL_COERCIBILITY_CHECK();
   return original_type.getTypeID() == kInt;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/IntType.hpp
----------------------------------------------------------------------
diff --git a/types/IntType.hpp b/types/IntType.hpp
index 08d6b3d..25ce3c8 100644
--- a/types/IntType.hpp
+++ b/types/IntType.hpp
@@ -40,10 +40,8 @@ class Type;
 /**
  * @brief A type representing a 32-bit integer.
  **/
-class IntType : public NumericSuperType<int> {
+class IntType : public NumericSuperType<kInt, int> {
  public:
-  static const TypeID kStaticTypeID;
-
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
    *        Type.
@@ -109,7 +107,7 @@ class IntType : public NumericSuperType<int> {
 
  private:
   explicit IntType(const bool nullable)
-      : NumericSuperType<int>(kInt, nullable) {
+      : NumericSuperType<kInt, int>(nullable) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(IntType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/LongType.cpp
----------------------------------------------------------------------
diff --git a/types/LongType.cpp b/types/LongType.cpp
index fbf8d30..153e239 100644
--- a/types/LongType.cpp
+++ b/types/LongType.cpp
@@ -38,8 +38,6 @@
 
 namespace quickstep {
 
-const TypeID LongType::kStaticTypeID = kLong;
-
 bool LongType::isSafelyCoercibleFrom(const Type &original_type) const {
   QUICKSTEP_NULL_COERCIBILITY_CHECK();
   return QUICKSTEP_EQUALS_ANY_CONSTANT(original_type.getTypeID(),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/LongType.hpp
----------------------------------------------------------------------
diff --git a/types/LongType.hpp b/types/LongType.hpp
index a90dd32..52244a7 100644
--- a/types/LongType.hpp
+++ b/types/LongType.hpp
@@ -41,10 +41,8 @@ class Type;
 /**
  * @brief A type representing a 64-bit integer.
  **/
-class LongType : public NumericSuperType<std::int64_t> {
+class LongType : public NumericSuperType<kLong, std::int64_t> {
  public:
-  static const TypeID kStaticTypeID;
-
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
    *        Type.
@@ -110,7 +108,7 @@ class LongType : public NumericSuperType<std::int64_t> {
 
  private:
   explicit LongType(const bool nullable)
-      : NumericSuperType<std::int64_t>(kLong, nullable) {
+      : NumericSuperType<kLong, std::int64_t>(nullable) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(LongType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/NullCoercibilityCheckMacro.hpp
----------------------------------------------------------------------
diff --git a/types/NullCoercibilityCheckMacro.hpp b/types/NullCoercibilityCheckMacro.hpp
index 70e1beb..2dcbec3 100644
--- a/types/NullCoercibilityCheckMacro.hpp
+++ b/types/NullCoercibilityCheckMacro.hpp
@@ -34,7 +34,7 @@
  **/
 #define QUICKSTEP_NULL_COERCIBILITY_CHECK()               \
   do {                                                    \
-    if (original_type.isNullable() && !nullable_) {       \
+    if (original_type.isNullable() && !this->nullable_) {       \
       return false;                                       \
     } else if (original_type.getTypeID() == kNullType) {  \
       return true;                                        \

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/NullType.hpp
----------------------------------------------------------------------
diff --git a/types/NullType.hpp b/types/NullType.hpp
index c27a584..6c6dd06 100644
--- a/types/NullType.hpp
+++ b/types/NullType.hpp
@@ -48,10 +48,8 @@ class TypedValue;
  *       a particular operation may accept. It is also assumed that applying
  *       any operation to an argument of NullType always yields NULL values.
  **/
-class NullType : public Type {
+class NullType : public TypeConcept<kNullType, false, kNativeEmbedded> {
  public:
-  static const TypeID kStaticTypeID = kNullType;
-
   /**
    * @brief Get a reference to the nullable singleton instance of this Type.
    * @note Unlike other Types, there is no corresponding method to get a
@@ -106,7 +104,9 @@ class NullType : public Type {
  private:
   // NOTE(chasseur): NullType requires 0 bytes of inherent storage. It does,
   // however, require a bit in NULL bitmaps.
-  NullType() : Type(Type::kOther, kNullType, true, 0, 0) {
+  NullType()
+      : TypeConcept<kNullType, false, kNativeEmbedded>(
+            Type::kOther, kStaticTypeID, true, 0, 0) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(NullType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/NumericSuperType.hpp
----------------------------------------------------------------------
diff --git a/types/NumericSuperType.hpp b/types/NumericSuperType.hpp
index 0cc1546..ceb24b6 100644
--- a/types/NumericSuperType.hpp
+++ b/types/NumericSuperType.hpp
@@ -38,18 +38,16 @@ namespace quickstep {
  * @brief Templatized superclass for Numeric types. Contains code common to all
  *        Numeric types.
  **/
-template <typename CppType>
-class NumericSuperType : public Type {
+template <TypeID type_id, typename CppType>
+class NumericSuperType : public TypeConcept<type_id, false, kNativeEmbedded, CppType> {
  public:
-  typedef CppType cpptype;
-
   std::size_t estimateAverageByteLength() const override {
     return sizeof(CppType);
   }
 
   bool isCoercibleFrom(const Type &original_type) const override {
     QUICKSTEP_NULL_COERCIBILITY_CHECK();
-    return (original_type.getSuperTypeID() == kNumeric);
+    return (original_type.getSuperTypeID() == Type::kNumeric);
   }
 
   TypedValue makeZeroValue() const override {
@@ -57,8 +55,9 @@ class NumericSuperType : public Type {
   }
 
  protected:
-  NumericSuperType(const TypeID type_id, const bool nullable)
-      : Type(Type::kNumeric, type_id, nullable, sizeof(CppType), sizeof(CppType)) {
+  NumericSuperType(const bool nullable)
+      : TypeConcept<type_id, false, kNativeEmbedded, CppType>(
+            Type::kNumeric, type_id, nullable, sizeof(CppType), sizeof(CppType)) {
   }
 
  private:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/Type.cpp
----------------------------------------------------------------------
diff --git a/types/Type.cpp b/types/Type.cpp
index f3d3f1b..144f39a 100644
--- a/types/Type.cpp
+++ b/types/Type.cpp
@@ -30,38 +30,7 @@ namespace quickstep {
 
 serialization::Type Type::getProto() const {
   serialization::Type proto;
-  switch (type_id_) {
-    case kInt:
-      proto.set_type_id(serialization::Type::INT);
-      break;
-    case kLong:
-      proto.set_type_id(serialization::Type::LONG);
-      break;
-    case kFloat:
-      proto.set_type_id(serialization::Type::FLOAT);
-      break;
-    case kDouble:
-      proto.set_type_id(serialization::Type::DOUBLE);
-      break;
-    case kDate:
-      proto.set_type_id(serialization::Type::DATE);
-      break;
-    case kDatetime:
-      proto.set_type_id(serialization::Type::DATETIME);
-      break;
-    case kDatetimeInterval:
-      proto.set_type_id(serialization::Type::DATETIME_INTERVAL);
-      break;
-    case kYearMonthInterval:
-      proto.set_type_id(serialization::Type::YEAR_MONTH_INTERVAL);
-      break;
-    case kNullType:
-      proto.set_type_id(serialization::Type::NULL_TYPE);
-      break;
-    default:
-      FATAL_ERROR("Unrecognized TypeID in Type::getProto");
-  }
-
+  proto.mutable_type_id()->CopyFrom(TypeIDFactory::GetProto(type_id_));
   proto.set_nullable(nullable_);
 
   return proto;
@@ -85,12 +54,4 @@ TypedValue Type::coerceValue(const TypedValue &original_value,
   return original_value;
 }
 
-bool AsciiStringSuperType::isCoercibleFrom(const Type &original_type) const {
-  if (original_type.isNullable() && !nullable_) {
-    return false;
-  }
-  return (original_type.getSuperTypeID() == kAsciiString)
-         || (original_type.getTypeID() == kNullType);
-}
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/Type.hpp
----------------------------------------------------------------------
diff --git a/types/Type.hpp b/types/Type.hpp
index 0e8c4e5..afc46eb 100644
--- a/types/Type.hpp
+++ b/types/Type.hpp
@@ -264,7 +264,7 @@ class Type {
    * @note It is NOT possible to coerce a nullable type to a non-nullable type,
    *       even if coercion would otherwise be possible.
    * @note Integer types are safely coercible to other integer or
-   *       floating-poin types of equal or greater length.
+   *       floating-point types of equal or greater length.
    * @note Floating-point types are safely coercible to other floating-point
    *       types of equal or greater precision.
    * @note ASCII string types are safely coercible to other ASCII string types
@@ -471,12 +471,59 @@ class Type {
   DISALLOW_COPY_AND_ASSIGN(Type);
 };
 
+template <TypeID type_id,
+          bool parameterized,
+          TypeStorageLayout layout,
+          typename CppType = void>
+class TypeConcept : public Type {
+ public:
+  static constexpr TypeID kStaticTypeID = type_id;
+  static constexpr bool kParameterized = parameterized;
+  static constexpr TypeStorageLayout kLayout = layout;
+
+  typedef CppType cpptype;
+
+ protected:
+  template <typename ...ArgTypes>
+  TypeConcept(ArgTypes &&...args)
+      : Type(std::forward<ArgTypes>(args)...) {}
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(TypeConcept);
+};
+
+
+template <TypeID type_id,
+          bool parameterized,
+          TypeStorageLayout layout,
+          typename CppType>
+constexpr TypeID TypeConcept<type_id, parameterized, layout, CppType>::kStaticTypeID;
+
+template <TypeID type_id,
+          bool parameterized,
+          TypeStorageLayout layout,
+          typename CppType>
+constexpr bool TypeConcept<type_id, parameterized, layout, CppType>::kParameterized;
+
+template <TypeID type_id,
+          bool parameterized,
+          TypeStorageLayout layout,
+          typename CppType>
+constexpr TypeStorageLayout TypeConcept<type_id, parameterized, layout, CppType>::kLayout;
+
 /**
  * @brief A superclass for ASCII string types.
  **/
-class AsciiStringSuperType : public Type {
+template <TypeID type_id, TypeStorageLayout layout>
+class AsciiStringSuperType : public TypeConcept<type_id, true, layout> {
  public:
-  bool isCoercibleFrom(const Type &original_type) const override;
+  bool isCoercibleFrom(const Type &original_type) const override {
+    if (original_type.isNullable() && !this->nullable_) {
+      return false;
+    }
+    return (original_type.getSuperTypeID() == Type::kAsciiString)
+           || (original_type.getTypeID() == kNullType);
+  }
 
   /**
    * @brief Get the character-length of this string type.
@@ -488,12 +535,12 @@ class AsciiStringSuperType : public Type {
   }
 
  protected:
-  AsciiStringSuperType(const TypeID type_id,
-                       const bool nullable,
+  AsciiStringSuperType(const bool nullable,
                        const std::size_t minimum_byte_length,
                        const std::size_t maximum_byte_length,
                        const std::size_t string_length)
-      : Type(Type::kAsciiString, type_id, nullable, minimum_byte_length, maximum_byte_length),
+      : TypeConcept<type_id, true, layout>(
+            Type::kAsciiString, type_id, nullable, minimum_byte_length, maximum_byte_length),
         length_(string_length) {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/Type.proto
----------------------------------------------------------------------
diff --git a/types/Type.proto b/types/Type.proto
index d03b5a4..d454923 100644
--- a/types/Type.proto
+++ b/types/Type.proto
@@ -19,21 +19,11 @@ syntax = "proto2";
 
 package quickstep.serialization;
 
-message Type {
-  enum TypeID {
-    INT = 0;
-    LONG = 1;
-    FLOAT = 2;
-    DOUBLE = 3;
-    CHAR = 4;
-    VAR_CHAR = 5;
-    DATETIME = 6;
-    DATETIME_INTERVAL = 7;
-    YEAR_MONTH_INTERVAL = 8;
-    NULL_TYPE = 9;
-    DATE = 10;
-  }
+message TypeID {
+  required int32 id = 1;
+}
 
+message Type {
   required TypeID type_id = 1;
   required bool nullable = 2;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/TypeConcept.hpp
----------------------------------------------------------------------
diff --git a/types/TypeConcept.hpp b/types/TypeConcept.hpp
new file mode 100644
index 0000000..d1ec79d
--- /dev/null
+++ b/types/TypeConcept.hpp
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_TYPE_CONCEPT_HPP_
+#define QUICKSTEP_TYPES_TYPE_CONCEPT_HPP_
+
+namespace quickstep {
+
+/** \addtogroup Types
+ *  @{
+ */
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_TYPE_CONCEPT_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/TypeFactory.cpp
----------------------------------------------------------------------
diff --git a/types/TypeFactory.cpp b/types/TypeFactory.cpp
index 7403dc9..df21d50 100644
--- a/types/TypeFactory.cpp
+++ b/types/TypeFactory.cpp
@@ -91,21 +91,22 @@ bool TypeFactory::ProtoIsValid(const serialization::Type &proto) {
   }
 
   // Check that the type_id is valid, and extensions if any.
-  switch (proto.type_id()) {
-    case serialization::Type::INT:
-    case serialization::Type::LONG:
-    case serialization::Type::FLOAT:
-    case serialization::Type::DOUBLE:
-    case serialization::Type::DATE:
-    case serialization::Type::DATETIME:
-    case serialization::Type::DATETIME_INTERVAL:
-    case serialization::Type::YEAR_MONTH_INTERVAL:
+  const TypeID type_id = TypeIDFactory::ReconstructFromProto(proto.type_id());
+  switch (type_id) {
+    case kInt:
+    case kLong:
+    case kFloat:
+    case kDouble:
+    case kDate:
+    case kDatetime:
+    case kDatetimeInterval:
+    case kYearMonthInterval:
       return true;
-    case serialization::Type::CHAR:
+    case kChar:
       return proto.HasExtension(serialization::CharType::length);
-    case serialization::Type::VAR_CHAR:
+    case kVarChar:
       return proto.HasExtension(serialization::VarCharType::length);
-    case serialization::Type::NULL_TYPE:
+    case kNullType:
       return proto.nullable();
     default:
       return false;
@@ -117,28 +118,29 @@ const Type& TypeFactory::ReconstructFromProto(const serialization::Type &proto)
       << "Attempted to create Type from an invalid proto description:\n"
       << proto.DebugString();
 
-  switch (proto.type_id()) {
-    case serialization::Type::INT:
+  const TypeID type_id = TypeIDFactory::ReconstructFromProto(proto.type_id());
+  switch (type_id) {
+    case kInt:
       return IntType::Instance(proto.nullable());
-    case serialization::Type::LONG:
+    case kLong:
       return LongType::Instance(proto.nullable());
-    case serialization::Type::FLOAT:
+    case kFloat:
       return FloatType::Instance(proto.nullable());
-    case serialization::Type::DOUBLE:
+    case kDouble:
       return DoubleType::Instance(proto.nullable());
-    case serialization::Type::DATE:
+    case kDate:
       return DateType::Instance(proto.nullable());
-    case serialization::Type::DATETIME:
+    case kDatetime:
       return DatetimeType::Instance(proto.nullable());
-    case serialization::Type::DATETIME_INTERVAL:
+    case kDatetimeInterval:
       return DatetimeIntervalType::Instance(proto.nullable());
-    case serialization::Type::YEAR_MONTH_INTERVAL:
+    case kYearMonthInterval:
       return YearMonthIntervalType::Instance(proto.nullable());
-    case serialization::Type::CHAR:
+    case kChar:
       return CharType::InstanceFromProto(proto);
-    case serialization::Type::VAR_CHAR:
+    case kVarChar:
       return VarCharType::InstanceFromProto(proto);
-    case serialization::Type::NULL_TYPE:
+    case kNullType:
       DCHECK(proto.nullable());
       return NullType::InstanceNullable();
     default:

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/TypeID.hpp
----------------------------------------------------------------------
diff --git a/types/TypeID.hpp b/types/TypeID.hpp
index c54d8a5..26731b9 100644
--- a/types/TypeID.hpp
+++ b/types/TypeID.hpp
@@ -21,6 +21,10 @@
 #define QUICKSTEP_TYPES_TYPE_ID_HPP_
 
 #include <cstddef>
+#include <type_traits>
+
+#include "types/Type.pb.h"
+#include "utility/Macros.hpp"
 
 namespace quickstep {
 
@@ -29,7 +33,7 @@ namespace quickstep {
  *
  * @note TypedValue assumes that this doesn't exceed 64 TypeIDs.
  **/
-enum TypeID {
+enum TypeID : int {
   kInt = 0,
   kLong,
   kFloat,
@@ -44,6 +48,13 @@ enum TypeID {
   kNumTypeIDs  // Not a real TypeID, exists for counting purposes.
 };
 
+enum TypeStorageLayout {
+  kNativeEmbedded,
+  kNativeInline,
+  kNonNativeInline,
+  kOutOfLine
+};
+
 /**
  * @brief Provides basic information about a Type in the Quickstep type system.
  *
@@ -65,6 +76,26 @@ struct TypeSignature {
  **/
 extern const char *kTypeNames[kNumTypeIDs];
 
+class TypeIDFactory {
+ public:
+  inline static serialization::TypeID GetProto(const TypeID type_id) {
+    serialization::TypeID proto;
+    proto.set_id(static_cast<std::underlying_type_t<TypeID>>(type_id));
+    return proto;
+  }
+
+  inline static TypeID ReconstructFromProto(const serialization::TypeID &proto) {
+    return static_cast<TypeID>(proto.id());
+  }
+
+  inline static bool ProtoIsValid(const serialization::TypeID &proto) {
+    return proto.id() < static_cast<std::underlying_type_t<TypeID>>(kNumTypeIDs);
+  }
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(TypeIDFactory);
+};
+
 }  // namespace quickstep
 
 #endif  // QUICKSTEP_TYPES_TYPE_ID_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/TypeUtil.hpp
----------------------------------------------------------------------
diff --git a/types/TypeUtil.hpp b/types/TypeUtil.hpp
new file mode 100644
index 0000000..fe55952
--- /dev/null
+++ b/types/TypeUtil.hpp
@@ -0,0 +1,181 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ **/
+
+#ifndef QUICKSTEP_TYPES_TYPE_UTIL_HPP_
+#define QUICKSTEP_TYPES_TYPE_UTIL_HPP_
+
+#include <type_traits>
+
+#include "types/CharType.hpp"
+#include "types/DateType.hpp"
+#include "types/DatetimeIntervalType.hpp"
+#include "types/DatetimeType.hpp"
+#include "types/DoubleType.hpp"
+#include "types/FloatType.hpp"
+#include "types/IntType.hpp"
+#include "types/LongType.hpp"
+#include "types/NullType.hpp"
+#include "types/Type.hpp"
+#include "types/Type.pb.h"
+#include "types/TypeID.hpp"
+#include "types/VarCharType.hpp"
+#include "types/YearMonthIntervalType.hpp"
+#include "utility/Macros.hpp"
+#include "utility/TemplateUtil.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+template <TypeID type_id>
+struct TypeGenerator {};
+
+#define REGISTER_TYPE(T) \
+  template <> struct TypeGenerator<T::kStaticTypeID> { typedef T type; };
+
+REGISTER_TYPE(IntType);
+REGISTER_TYPE(LongType);
+REGISTER_TYPE(FloatType);
+REGISTER_TYPE(DoubleType);
+REGISTER_TYPE(DateType);
+REGISTER_TYPE(DatetimeType);
+REGISTER_TYPE(DatetimeIntervalType);
+REGISTER_TYPE(YearMonthIntervalType);
+REGISTER_TYPE(CharType);
+REGISTER_TYPE(VarCharType);
+REGISTER_TYPE(NullType);
+
+#undef REGISTER_TYPE
+
+namespace type_util_internal {
+
+template <bool require_parameterized>
+struct TypeIDSelectorParameterizedHelper {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+    inline static auto Invoke(const FunctorT &functor)
+        -> decltype(functor(TypeIDConstant())) {
+      DLOG(FATAL) << "Unexpected TypeID: "
+                  << kTypeNames[static_cast<int>(TypeIDConstant::value)];
+    }
+  };
+};
+
+template <bool require_non_parameterized>
+template <typename TypeIDConstant, typename FunctorT>
+struct TypeIDSelectorParameterizedHelper<require_non_parameterized>::Implementation<
+    TypeIDConstant, FunctorT,
+    std::enable_if_t<TypeGenerator<TypeIDConstant::value>::type::kParameterized
+                         ^ require_non_parameterized>> {
+  inline static auto Invoke(const FunctorT &functor) {
+    return functor(TypeIDConstant());
+  }
+};
+
+}  // namespace type_util_internal
+
+struct TypeIDSelectorAll {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+    inline static auto Invoke(const FunctorT &functor) {
+      return functor(TypeIDConstant());
+    }
+  };
+};
+
+using TypeIDSelectorNonParameterized =
+    type_util_internal::TypeIDSelectorParameterizedHelper<true>;
+
+using TypeIDSelectorParameterized =
+    type_util_internal::TypeIDSelectorParameterizedHelper<false>;
+
+template <TypeID ...candidates>
+struct TypeIDSelectorEqualsAny {
+  template <typename TypeIDConstant, typename FunctorT, typename EnableT = void>
+  struct Implementation {
+    inline static auto Invoke(const FunctorT &functor)
+        -> decltype(functor(TypeIDConstant())) {
+      DLOG(FATAL) << "Unexpected TypeID: "
+                  << kTypeNames[static_cast<int>(TypeIDConstant::value)];
+    }
+  };
+};
+
+template <TypeID ...candidates>
+template <typename TypeIDConstant, typename FunctorT>
+struct TypeIDSelectorEqualsAny<candidates...>::Implementation<
+    TypeIDConstant, FunctorT,
+    std::enable_if_t<
+        EqualsAny<TypeIDConstant,
+                  std::integral_constant<TypeID, candidates>...>::value>> {
+  inline static auto Invoke(const FunctorT &functor) {
+      return functor(TypeIDConstant());
+  }
+};
+
+template <typename Selector = TypeIDSelectorAll, typename FunctorT>
+inline auto InvokeOnTypeID(const TypeID type_id,
+                           const FunctorT &functor) {
+#define REGISTER_TYPE_ID(type_id) \
+  case type_id: \
+    return Selector::template Implementation< \
+        std::integral_constant<TypeID, type_id>, FunctorT>::Invoke(functor)
+
+  switch (type_id) {
+
+    REGISTER_TYPE_ID(kInt);
+    REGISTER_TYPE_ID(kLong);
+    REGISTER_TYPE_ID(kFloat);
+    REGISTER_TYPE_ID(kDouble);
+    REGISTER_TYPE_ID(kDate);
+    REGISTER_TYPE_ID(kDatetime);
+    REGISTER_TYPE_ID(kDatetimeInterval);
+    REGISTER_TYPE_ID(kYearMonthInterval);
+    REGISTER_TYPE_ID(kChar);
+    REGISTER_TYPE_ID(kVarChar);
+    REGISTER_TYPE_ID(kNullType);
+
+    default:
+      FATAL_ERROR("Unrecognized TypeID in InvokeOnTypeID()");
+  }
+
+#undef REGISTER_TYPE_ID
+}
+
+class TypeUtil {
+ public:
+  static bool IsParameterized(const TypeID type_id) {
+    return InvokeOnTypeID(
+        type_id,
+        [&](auto tid) -> bool {  // NOLINT(build/c++11)
+      return TypeGenerator<decltype(tid)::value>::type::kParameterized;
+    });
+  }
+
+ private:
+  TypeUtil() {}
+
+  DISALLOW_COPY_AND_ASSIGN(TypeUtil);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_TYPES_TYPE_UTIL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/TypedValue.cpp
----------------------------------------------------------------------
diff --git a/types/TypedValue.cpp b/types/TypedValue.cpp
index 8dd8b60..1360237 100644
--- a/types/TypedValue.cpp
+++ b/types/TypedValue.cpp
@@ -82,33 +82,29 @@ serialization::TypedValue TypedValue::getProto() const {
 
   // NOTE(chasseur): To represent a NULL value, only the 'type_id' field of the
   // proto is filled in, and all the optional value fields are omitted.
+  proto.mutable_type_id()->CopyFrom(TypeIDFactory::GetProto(getTypeID()));
   switch (getTypeID()) {
     case kInt:
-      proto.set_type_id(serialization::Type::INT);
       if (!isNull()) {
         proto.set_int_value(getLiteral<int>());
       }
       break;
     case kLong:
-      proto.set_type_id(serialization::Type::LONG);
       if (!isNull()) {
         proto.set_long_value(getLiteral<std::int64_t>());
       }
       break;
     case kFloat:
-      proto.set_type_id(serialization::Type::FLOAT);
       if (!isNull()) {
         proto.set_float_value(getLiteral<float>());
       }
       break;
     case kDouble:
-      proto.set_type_id(serialization::Type::DOUBLE);
       if (!isNull()) {
         proto.set_double_value(getLiteral<double>());
       }
       break;
     case kDate:
-      proto.set_type_id(serialization::Type::DATE);
       if (!isNull()) {
         serialization::TypedValue::DateLit *literal_date_proto = proto.mutable_date_value();
         literal_date_proto->set_year(value_union_.date_value.year);
@@ -117,37 +113,31 @@ serialization::TypedValue TypedValue::getProto() const {
       }
       break;
     case kDatetime:
-      proto.set_type_id(serialization::Type::DATETIME);
       if (!isNull()) {
         proto.set_datetime_value(value_union_.datetime_value.ticks);
       }
       break;
     case kDatetimeInterval:
-      proto.set_type_id(serialization::Type::DATETIME_INTERVAL);
       if (!isNull()) {
         proto.set_datetime_interval_value(value_union_.datetime_interval_value.interval_ticks);
       }
       break;
     case kYearMonthInterval:
-      proto.set_type_id(serialization::Type::YEAR_MONTH_INTERVAL);
       if (!isNull()) {
         proto.set_year_month_interval_value(value_union_.year_month_interval_value.months);
       }
       break;
     case kChar:
-      proto.set_type_id(serialization::Type::CHAR);
       if (!isNull()) {
         proto.set_out_of_line_data(static_cast<const char*>(getOutOfLineData()), getDataSize());
       }
       break;
     case kVarChar:
-      proto.set_type_id(serialization::Type::VAR_CHAR);
       if (!isNull()) {
         proto.set_out_of_line_data(static_cast<const char*>(getOutOfLineData()), getDataSize());
       }
       break;
     case kNullType:
-      proto.set_type_id(serialization::Type::NULL_TYPE);
       DCHECK(isNull());
       break;
     default:
@@ -166,24 +156,25 @@ TypedValue TypedValue::ReconstructFromProto(const serialization::TypedValue &pro
       << "Attempted to create TypedValue from an invalid proto description:\n"
       << proto.DebugString();
 
-  switch (proto.type_id()) {
-    case serialization::Type::INT:
+  const TypeID type_id = TypeIDFactory::ReconstructFromProto(proto.type_id());
+  switch (type_id) {
+    case kInt:
       return proto.has_int_value() ?
           TypedValue(static_cast<int>(proto.int_value())) :
           TypedValue(kInt);
-    case serialization::Type::LONG:
+    case kLong:
       return proto.has_long_value() ?
           TypedValue(static_cast<std::int64_t>(proto.long_value())) :
           TypedValue(kLong);
-    case serialization::Type::FLOAT:
+    case kFloat:
       return proto.has_float_value() ?
           TypedValue(static_cast<float>(proto.float_value())) :
           TypedValue(kFloat);
-    case serialization::Type::DOUBLE:
+    case kDouble:
       return proto.has_double_value() ?
           TypedValue(static_cast<double>(proto.double_value())) :
           TypedValue(kDouble);
-    case serialization::Type::DATE:
+    case kDate:
       if (proto.has_date_value()) {
         return TypedValue(DateLit::Create(proto.date_value().year(),
                                           proto.date_value().month(),
@@ -191,7 +182,7 @@ TypedValue TypedValue::ReconstructFromProto(const serialization::TypedValue &pro
       } else {
         return TypedValue(kDate);
       }
-    case serialization::Type::DATETIME:
+    case kDatetime:
       if (proto.has_datetime_value()) {
         DatetimeLit datetime;
         datetime.ticks = proto.datetime_value();
@@ -199,7 +190,7 @@ TypedValue TypedValue::ReconstructFromProto(const serialization::TypedValue &pro
       } else {
         return TypedValue(kDatetime);
       }
-    case serialization::Type::DATETIME_INTERVAL:
+    case kDatetimeInterval:
       if (proto.has_datetime_interval_value()) {
         DatetimeIntervalLit interval;
         interval.interval_ticks = proto.datetime_interval_value();
@@ -207,7 +198,7 @@ TypedValue TypedValue::ReconstructFromProto(const serialization::TypedValue &pro
       } else {
         return TypedValue(kDatetimeInterval);
       }
-    case serialization::Type::YEAR_MONTH_INTERVAL:
+    case kYearMonthInterval:
       if (proto.has_year_month_interval_value()) {
         YearMonthIntervalLit interval;
         interval.months = proto.year_month_interval_value();
@@ -215,19 +206,19 @@ TypedValue TypedValue::ReconstructFromProto(const serialization::TypedValue &pro
       } else {
         return TypedValue(kYearMonthInterval);
       }
-    case serialization::Type::CHAR:
+    case kChar:
       return proto.has_out_of_line_data() ?
           TypedValue(kChar,
                      static_cast<const void*>(proto.out_of_line_data().c_str()),
                      proto.out_of_line_data().size()).ensureNotReference() :
           TypedValue(kChar);
-    case serialization::Type::VAR_CHAR:
+    case kVarChar:
       return proto.has_out_of_line_data() ?
           TypedValue(kVarChar,
                      static_cast<const void*>(proto.out_of_line_data().c_str()),
                      proto.out_of_line_data().size()).ensureNotReference() :
           TypedValue(kVarChar);
-    case serialization::Type::NULL_TYPE:
+    case kNullType:
       return TypedValue(kNullType);
     default:
       FATAL_ERROR("Unrecognized TypeID in TypedValue::ReconstructFromProto");

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/TypedValue.hpp
----------------------------------------------------------------------
diff --git a/types/TypedValue.hpp b/types/TypedValue.hpp
index 0ba3d53..18ed4af 100644
--- a/types/TypedValue.hpp
+++ b/types/TypedValue.hpp
@@ -264,9 +264,9 @@ class TypedValue {
    *        TypedValue will take ownership of this memory.
    * @param value_size The number of bytes of data at value_ptr.
    **/
-  static TypedValue CreateWithOwnedData(const TypeID type_id,
-                                        void *value_ptr,
-                                        const std::size_t value_size) {
+  inline static TypedValue CreateWithOwnedData(const TypeID type_id,
+                                               void *value_ptr,
+                                               const std::size_t value_size) {
     TypedValue val(type_id, value_ptr, value_size);
     val.value_info_ |= kOwnershipMask;
     return val;
@@ -467,7 +467,8 @@ class TypedValue {
    * @return The underlying literal value represented by this TypedValue.
    **/
   template <typename LiteralType>
-  LiteralType getLiteral() const;
+  LiteralType getLiteral() const {
+  }
 
   /**
    * @brief Get the out-of-line data which this TypedValue points to.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/TypedValue.proto
----------------------------------------------------------------------
diff --git a/types/TypedValue.proto b/types/TypedValue.proto
index 7f3ab7a..db44dec 100644
--- a/types/TypedValue.proto
+++ b/types/TypedValue.proto
@@ -22,7 +22,7 @@ package quickstep.serialization;
 import "types/Type.proto";
 
 message TypedValue {
-  required Type.TypeID type_id = 1;
+  required TypeID type_id = 1;
 
   // NOTE(zuyu): For a NULL value, none of the optional fields are filled in.
   optional int32 int_value = 2;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/VarCharType.cpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.cpp b/types/VarCharType.cpp
index 02845b1..05162cf 100644
--- a/types/VarCharType.cpp
+++ b/types/VarCharType.cpp
@@ -66,8 +66,7 @@ const VarCharType& VarCharType::InstanceFromProto(const serialization::Type &pro
 
 serialization::Type VarCharType::getProto() const {
   serialization::Type proto;
-  proto.set_type_id(serialization::Type::VAR_CHAR);
-
+  proto.mutable_type_id()->CopyFrom(TypeIDFactory::GetProto(type_id_));
   proto.set_nullable(nullable_);
 
   proto.SetExtension(serialization::VarCharType::length, length_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/VarCharType.hpp
----------------------------------------------------------------------
diff --git a/types/VarCharType.hpp b/types/VarCharType.hpp
index bb50e92..af3ab64 100644
--- a/types/VarCharType.hpp
+++ b/types/VarCharType.hpp
@@ -43,7 +43,7 @@ namespace quickstep {
  *       character. This means that the VARCHAR(X) type requires from 1 to X+1
  *       bytes of storage, depending on string length.
  **/
-class VarCharType : public AsciiStringSuperType {
+class VarCharType : public AsciiStringSuperType<kVarChar, kOutOfLine> {
  public:
   /**
    * @brief Get a reference to the non-nullable singleton instance of this Type
@@ -137,7 +137,7 @@ class VarCharType : public AsciiStringSuperType {
 
  private:
   VarCharType(const std::size_t length, const bool nullable)
-      : AsciiStringSuperType(kVarChar, nullable, 1, length + 1, length) {
+      : AsciiStringSuperType<kVarChar, kOutOfLine>(nullable, 1, length + 1, length) {
   }
 
   template <bool nullable_internal>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/YearMonthIntervalType.hpp
----------------------------------------------------------------------
diff --git a/types/YearMonthIntervalType.hpp b/types/YearMonthIntervalType.hpp
index a2ba175..f768046 100644
--- a/types/YearMonthIntervalType.hpp
+++ b/types/YearMonthIntervalType.hpp
@@ -39,12 +39,9 @@ namespace quickstep {
 /**
  * @brief A type representing the year-month interval.
  **/
-class YearMonthIntervalType : public Type {
+class YearMonthIntervalType :
+    public TypeConcept<kYearMonthInterval, false, kNativeEmbedded, YearMonthIntervalLit> {
  public:
-  typedef YearMonthIntervalLit cpptype;
-
-  static const TypeID kStaticTypeID = kYearMonthInterval;
-
   /**
    * @brief Get a reference to the non-nullable singleton instance of this
    *        Type.
@@ -115,7 +112,9 @@ class YearMonthIntervalType : public Type {
 
  private:
   explicit YearMonthIntervalType(const bool nullable)
-      : Type(Type::kOther, kYearMonthInterval, nullable, sizeof(YearMonthIntervalLit), sizeof(YearMonthIntervalLit)) {
+      : TypeConcept<kYearMonthInterval, false, kNativeEmbedded, YearMonthIntervalLit>(
+            Type::kOther, kStaticTypeID, nullable,
+            sizeof(YearMonthIntervalLit), sizeof(YearMonthIntervalLit)) {
   }
 
   DISALLOW_COPY_AND_ASSIGN(YearMonthIntervalType);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/containers/ColumnVector.hpp
----------------------------------------------------------------------
diff --git a/types/containers/ColumnVector.hpp b/types/containers/ColumnVector.hpp
index fc65656..f0a2cd5 100644
--- a/types/containers/ColumnVector.hpp
+++ b/types/containers/ColumnVector.hpp
@@ -130,8 +130,8 @@ class NativeColumnVector : public ColumnVector {
   NativeColumnVector(const Type &type, const std::size_t reserved_length)
       : ColumnVector(type),
         type_length_(type.maximumByteLength()),
-        values_(std::malloc(type.maximumByteLength() * reserved_length)),
         reserved_length_(reserved_length),
+        values_(std::malloc(type.maximumByteLength() * reserved_length)),
         actual_length_(0u),
         null_bitmap_(type.isNullable() ? new BitVector<false>(reserved_length) : nullptr) {
     DCHECK(UsableForType(type_));
@@ -385,8 +385,9 @@ class NativeColumnVector : public ColumnVector {
 
  private:
   const std::size_t type_length_;
-  void *values_;
   const std::size_t reserved_length_;
+
+  void *values_;
   std::size_t actual_length_;
   std::unique_ptr<BitVector<false>> null_bitmap_;
 
@@ -498,6 +499,12 @@ class IndirectColumnVector : public ColumnVector {
     values_.emplace_back(std::move(value));
   }
 
+  inline void appendNullValue() {
+    DCHECK(type_.isNullable());
+    DCHECK_LT(values_.size(), reserved_length_);
+    values_.emplace_back(type_.makeNullValue());
+  }
+
   /**
    * @brief Fill this entire ColumnVector with copies of value.
    *
@@ -559,6 +566,7 @@ class IndirectColumnVector : public ColumnVector {
  private:
   const bool type_is_nullable_;
   const std::size_t reserved_length_;
+
   std::vector<TypedValue> values_;
 
   DISALLOW_COPY_AND_ASSIGN(IndirectColumnVector);



[2/8] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/DateExtractOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/DateExtractOperation.cpp b/types/operations/unary_operations/DateExtractOperation.cpp
index c99e403..c09b400 100644
--- a/types/operations/unary_operations/DateExtractOperation.cpp
+++ b/types/operations/unary_operations/DateExtractOperation.cpp
@@ -21,521 +21,136 @@
 
 #include <cstddef>
 #include <cstdint>
-#include <memory>
+#include <map>
 #include <string>
 #include <type_traits>
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-#include <utility>
 #include <vector>
 
-#include "storage/StorageBlockInfo.hpp"
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
+#include "types/DateType.hpp"
 #include "types/DatetimeLit.hpp"
-#include "types/IntType.hpp"
-#include "types/LongType.hpp"
+#include "types/DatetimeType.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/Operation.pb.h"
-#include "utility/Macros.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
 
 #include "glog/logging.h"
 
-using std::int32_t;
-using std::int64_t;
-
 namespace quickstep {
 
-template <DateExtractUnit unit, bool argument_nullable>
-TypedValue DatetimeExtractUncheckedOperator<unit, argument_nullable>::applyToTypedValue(
-    const TypedValue &argument) const {
-  if (argument_nullable && argument.isNull()) {
-    return TypedValue(kLong);
-  }
-
-  return TypedValue(dateExtract(argument.getLiteral<DatetimeLit>()));
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-TypedValue DateExtractUncheckedOperator<unit, argument_nullable>::applyToTypedValue(
-    const TypedValue &argument) const {
-  if (argument_nullable && argument.isNull()) {
-    return TypedValue(kInt);
-  }
-
-  return TypedValue(dateExtract(argument.getLiteral<DateLit>()));
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-TypedValue DatetimeExtractUncheckedOperator<unit, argument_nullable>::applyToDataPtr(const void *argument) const {
-  if (argument_nullable && argument == nullptr) {
-    return TypedValue(kLong);
-  }
-
-  return TypedValue(dateExtract(*static_cast<const DatetimeLit*>(argument)));
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-TypedValue DateExtractUncheckedOperator<unit, argument_nullable>::applyToDataPtr(const void *argument) const {
-  if (argument_nullable && argument == nullptr) {
-    return TypedValue(kInt);
-  }
-
-  return TypedValue(dateExtract(*static_cast<const DateLit*>(argument)));
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-ColumnVector* DatetimeExtractUncheckedOperator<unit, argument_nullable>::applyToColumnVector(
-    const ColumnVector &argument) const {
-  // Datetime are usable with NativeColumnVector, so 'argument' should always
-  // be native.
-  DCHECK(argument.isNative());
-
-  const NativeColumnVector &native_argument = static_cast<const NativeColumnVector&>(argument);
-  std::unique_ptr<NativeColumnVector> result(
-      new NativeColumnVector(LongType::Instance(argument_nullable), native_argument.size()));
-
-  for (std::size_t pos = 0;
-       pos < native_argument.size();
-       ++pos) {
-    const DatetimeLit *datetime_arg =
-        static_cast<const DatetimeLit*>(native_argument.getUntypedValue<argument_nullable>(pos));
-    if (argument_nullable && (datetime_arg == nullptr)) {
-      result->appendNullValue();
-    } else {
-      *static_cast<int64_t*>(result->getPtrForDirectWrite())
-          = dateExtract(*datetime_arg);
-    }
-  }
-
-  return result.release();
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-ColumnVector* DateExtractUncheckedOperator<unit, argument_nullable>::applyToColumnVector(
-    const ColumnVector &argument) const {
-  // Date is usable with NativeColumnVector, so 'argument' should always
-  // be native.
-  DCHECK(argument.isNative());
-
-  const NativeColumnVector &native_argument = static_cast<const NativeColumnVector&>(argument);
-  std::unique_ptr<NativeColumnVector> result(
-      new NativeColumnVector(IntType::Instance(argument_nullable), native_argument.size()));
-
-  for (std::size_t pos = 0;
-       pos < native_argument.size();
-       ++pos) {
-    const DateLit *date_arg =
-        static_cast<const DateLit*>(native_argument.getUntypedValue<argument_nullable>(pos));
-    if (argument_nullable && (date_arg == nullptr)) {
-      result->appendNullValue();
-    } else {
-      *static_cast<int32_t*>(result->getPtrForDirectWrite())
-          = dateExtract(*date_arg);
-    }
-  }
-
-  return result.release();
-}
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-template <DateExtractUnit unit, bool argument_nullable>
-ColumnVector* DatetimeExtractUncheckedOperator<unit, argument_nullable>::applyToValueAccessor(
-    ValueAccessor *accessor,
-    const attribute_id argument_attr_id) const {
-  return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-      accessor,
-      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-    std::unique_ptr<NativeColumnVector> result(
-        new NativeColumnVector(LongType::Instance(argument_nullable), accessor->getNumTuples()));
-    accessor->beginIteration();
-    while (accessor->next()) {
-      const DatetimeLit *datetime_arg =
-          static_cast<const DatetimeLit*>(
-              accessor->template getUntypedValue<argument_nullable>(argument_attr_id));
-      if (argument_nullable && (datetime_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<int64_t*>(result->getPtrForDirectWrite())
-            = this->dateExtract(*datetime_arg);
-      }
-    }
-    return result.release();
-  });
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-ColumnVector* DateExtractUncheckedOperator<unit, argument_nullable>::applyToValueAccessor(
-    ValueAccessor *accessor,
-    const attribute_id argument_attr_id) const {
-  return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-      accessor,
-      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-    std::unique_ptr<NativeColumnVector> result(
-        new NativeColumnVector(IntType::Instance(argument_nullable), accessor->getNumTuples()));
-    accessor->beginIteration();
-    while (accessor->next()) {
-      const DateLit *date_arg =
-          static_cast<const DateLit*>(
-              accessor->template getUntypedValue<argument_nullable>(argument_attr_id));
-      if (argument_nullable && (date_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<int32_t*>(result->getPtrForDirectWrite())
-            = this->dateExtract(*date_arg);
-      }
-    }
-    return result.release();
-  });
-}
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-template <DateExtractUnit unit, bool argument_nullable>
-ColumnVector* DatetimeExtractUncheckedOperator<unit, argument_nullable>::applyToValueAccessorForJoin(
-    ValueAccessor *accessor,
-    const bool use_left_relation,
-    const attribute_id argument_attr_id,
-    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
-  std::unique_ptr<NativeColumnVector> result(
-      new NativeColumnVector(LongType::Instance(argument_nullable), joined_tuple_ids.size()));
-  return InvokeOnValueAccessorNotAdapter(
-      accessor,
-      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-    for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-      const DatetimeLit *datetime_arg =
-          static_cast<const DatetimeLit*>(
-              accessor->template getUntypedValueAtAbsolutePosition<argument_nullable>(
-                  argument_attr_id,
-                  use_left_relation ? joined_pair.first : joined_pair.second));
-      if (argument_nullable && (datetime_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<int64_t*>(result->getPtrForDirectWrite())
-            = this->dateExtract(*datetime_arg);
-      }
-    }
-    return result.release();
-  });
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-ColumnVector* DateExtractUncheckedOperator<unit, argument_nullable>::applyToValueAccessorForJoin(
-    ValueAccessor *accessor,
-    const bool use_left_relation,
-    const attribute_id argument_attr_id,
-    const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const {
-  std::unique_ptr<NativeColumnVector> result(
-      new NativeColumnVector(IntType::Instance(argument_nullable), joined_tuple_ids.size()));
-  return InvokeOnValueAccessorNotAdapter(
-      accessor,
-      [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-    for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-      const DateLit *date_arg =
-          static_cast<const DateLit*>(
-              accessor->template getUntypedValueAtAbsolutePosition<argument_nullable>(
-                  argument_attr_id,
-                  use_left_relation ? joined_pair.first : joined_pair.second));
-      if (argument_nullable && (date_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<int32_t*>(result->getPtrForDirectWrite())
-            = this->dateExtract(*date_arg);
-      }
-    }
-    return result.release();
-  });
-}
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-template <DateExtractUnit unit, bool argument_nullable>
-inline int64_t
-DatetimeExtractUncheckedOperator<unit, argument_nullable>::dateExtract(
-    const DatetimeLit &argument) const {
-  switch (unit) {
-    case DateExtractUnit::kYear:
-      return argument.yearField();
-    case DateExtractUnit::kMonth:
-      return argument.monthField();
-    case DateExtractUnit::kDay:
-      return argument.dayField();
-    case DateExtractUnit::kHour:
-      return argument.hourField();
-    case DateExtractUnit::kMinute:
-      return argument.minuteField();
-    case DateExtractUnit::kSecond:
-      return argument.secondField();
-    default:
-      FATAL_ERROR("Unsupported DateExtractUnit in DatetimeExtractUncheckedOperator::dateExtract.");
-  }
-}
-
-template <DateExtractUnit unit, bool argument_nullable>
-inline int32_t DateExtractUncheckedOperator<unit, argument_nullable>::dateExtract(const DateLit &argument) const {
-  switch (unit) {
-    case DateExtractUnit::kYear:
-      return argument.yearField();
-    case DateExtractUnit::kMonth:
-      return argument.monthField();
-    default:
-      FATAL_ERROR("Unsupported DateExtractUnit in DateExtractUncheckedOperator::dateExtract.");
-  }
-}
-
-const DateExtractOperation& DateExtractOperation::Instance(const DateExtractUnit unit) {
-  switch (unit) {
-    case DateExtractUnit::kYear: {
-      static DateExtractOperation instance(DateExtractUnit::kYear);
-      return instance;
-    }
-    case DateExtractUnit::kMonth: {
-      static DateExtractOperation instance(DateExtractUnit::kMonth);
-      return instance;
-    }
-    case DateExtractUnit::kDay: {
-      static DateExtractOperation instance(DateExtractUnit::kDay);
-      return instance;
-    }
-    case DateExtractUnit::kHour: {
-      static DateExtractOperation instance(DateExtractUnit::kHour);
-      return instance;
-    }
-    case DateExtractUnit::kMinute: {
-      static DateExtractOperation instance(DateExtractUnit::kMinute);
-      return instance;
-    }
-    case DateExtractUnit::kSecond: {
-      static DateExtractOperation instance(DateExtractUnit::kSecond);
-      return instance;
-    }
-    default: {
-      FATAL_ERROR("Unsupported DateExtractUnit in DateExtractOperation::Instance.");
+struct DateExtractFunctor
+    : public UnaryFunctor<DateType, IntType> {
+  template <typename DateExtractUnitT>
+  inline int apply(const DateLit &argument) const {
+    switch (DateExtractUnitT::value) {
+      case DateExtractUnit::kYear:
+        return argument.yearField();
+      case DateExtractUnit::kMonth:
+        return argument.monthField();
+      case DateExtractUnit::kDay:
+        return argument.dayField();
+      default:
+        DLOG(FATAL) << "Unsupported DateExtractUnit in DateExtractFunctor::apply";
     }
   }
-}
-
-serialization::UnaryOperation DateExtractOperation::getProto() const {
-  serialization::UnaryOperation proto;
-  proto.set_operation_id(serialization::UnaryOperation::DATE_EXTRACT);
-
-  switch (unit_) {
-    case DateExtractUnit::kYear:
-      proto.SetExtension(serialization::DateExtractOperation::unit, serialization::DateExtractOperation::YEAR);
-      break;
-    case DateExtractUnit::kMonth:
-      proto.SetExtension(serialization::DateExtractOperation::unit, serialization::DateExtractOperation::MONTH);
-      break;
-    case DateExtractUnit::kDay:
-      proto.SetExtension(serialization::DateExtractOperation::unit, serialization::DateExtractOperation::DAY);
-      break;
-    case DateExtractUnit::kHour:
-      proto.SetExtension(serialization::DateExtractOperation::unit, serialization::DateExtractOperation::HOUR);
-      break;
-    case DateExtractUnit::kMinute:
-      proto.SetExtension(serialization::DateExtractOperation::unit, serialization::DateExtractOperation::MINUTE);
-      break;
-    case DateExtractUnit::kSecond:
-      proto.SetExtension(serialization::DateExtractOperation::unit, serialization::DateExtractOperation::SECOND);
-      break;
-    default:
-      FATAL_ERROR("Unsupported DateExtractUnit in DateExtractOperation::getProto.");
-  }
-
-  return proto;
-}
-
-std::string DateExtractOperation::getName() const {
-  std::string name(kUnaryOperationNames[static_cast<std::size_t>(operation_id_)]);
-  name.push_back('(');
-  switch (unit_) {
-    case DateExtractUnit::kYear:
-      name.append("YEAR)");
-      break;
-    case DateExtractUnit::kMonth:
-      name.append("MONTH)");
-      break;
-    case DateExtractUnit::kDay:
-      name.append("DAY)");
-      break;
-    case DateExtractUnit::kHour:
-      name.append("HOUR)");
-      break;
-    case DateExtractUnit::kMinute:
-      name.append("MINUTE)");
-      break;
-    case DateExtractUnit::kSecond:
-      name.append("SECOND)");
-      break;
-    default:
-      name.append("UNKNOWN)");
-      break;
-  }
-  return name;
-}
-
-const Type* DateExtractOperation::pushDownTypeHint(const Type *type_hint) const {
-  if (type_hint == nullptr) {
-    return nullptr;
-  }
+};
 
-  if (type_hint->getTypeID() == kLong) {
-    switch (unit_) {
-      case DateExtractUnit::kYear:  // Fall through.
+struct DatetimeExtractFunctor
+    : public UnaryFunctor<DatetimeType, IntType> {
+  template <typename DateExtractUnitT>
+  inline std::int64_t apply(const DatetimeLit &argument) const {
+    switch (DateExtractUnitT::value) {
+      case DateExtractUnit::kYear:
+        return argument.yearField();
       case DateExtractUnit::kMonth:
-        // There are two possibilities for the return type, based on whether we
-        // have Datetime or Date as the underlying date implementation.
-        return nullptr;
-      case DateExtractUnit::kDay:  // Fall through.
+        return argument.monthField();
+      case DateExtractUnit::kDay:
+        return argument.dayField();
       case DateExtractUnit::kHour:
+        return argument.hourField();
       case DateExtractUnit::kMinute:
+        return argument.minuteField();
       case DateExtractUnit::kSecond:
-        return &TypeFactory::GetType(kDatetime, type_hint->isNullable());
+        return argument.secondField();
       default:
-        return nullptr;
+        DLOG(FATAL) << "Unsupported DateExtractUnit in DatetimeExtractFunctor::apply";
+    }
+  }
+};
+
+const std::map<std::string, DateExtractUnit> DateExtractOperation::kNameToUnitMap = {
+    { "year",   DateExtractUnit::kYear },
+    { "month",  DateExtractUnit::kMonth },
+    { "day",    DateExtractUnit::kDay },
+    { "hour",   DateExtractUnit::kHour },
+    { "minute", DateExtractUnit::kMinute },
+    { "second", DateExtractUnit::kSecond }
+};
+
+UncheckedUnaryOperator* DateExtractOperation::makeUncheckedUnaryOperator(
+    const Type &type,
+    const std::vector<TypedValue> &static_arguments) const {
+  DCHECK_EQ(1u, static_arguments.size());
+  DateExtractUnit unit = parseUnit(static_arguments.front());
+  const Type *result_type = getResultType(type, static_arguments);
+
+  if (type.getTypeID() == kDate) {
+    switch (unit) {
+      case DateExtractUnit::kYear:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DateExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kYear>>(
+                 type, *result_type);
+      case DateExtractUnit::kMonth:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DateExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kMonth>>(
+                 type, *result_type);
+      case DateExtractUnit::kDay:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DateExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kDay>>(
+                 type, *result_type);
+      default:
+        LOG(FATAL) << "Unsupported DateExtractUnit for DateType in "
+                   << "DateExtractOperation::makeUncheckedUnaryOperator";
     }
   } else {
-    return nullptr;
-  }
-}
-
-TypedValue DateExtractOperation::applyToChecked(const TypedValue &argument,
-                                                const Type &argument_type) const {
-  if (((argument.getTypeID() != TypeID::kDatetime) ||
-       (argument_type.getTypeID() != TypeID::kDatetime)) &&
-      ((argument.getTypeID() != TypeID::kDate) ||
-       (argument_type.getTypeID() != TypeID::kDate))) {
-    LOG(FATAL) << "UnaryOperation " << getName() << " is only applicable to Type "
-               << kTypeNames[TypeID::kDatetime] << ", but applyToChecked() was "
-               << "called with 'argument' of Type " << kTypeNames[argument.getTypeID()]
-               << " and explicit 'argument_type' parameter of "
-               << argument_type.getName();
-  }
-
-  if (argument.isNull()) {
-    if (argument.getTypeID() == TypeID::kDatetime) {
-      return TypedValue(kLong);
-    } else {
-      // argument type is kDate.
-      DCHECK_EQ(TypeID::kDate, argument.getTypeID());
-      return TypedValue(kInt);
-    }
-  }
-
-  switch (unit_) {
-    case DateExtractUnit::kYear: {
-      if (argument.getTypeID() == TypeID::kDatetime) {
-        return TypedValue(argument.getLiteral<DatetimeLit>().yearField());
-      } else {
-        // argument type is kDate.
-        DCHECK_EQ(TypeID::kDate, argument.getTypeID());
-        return TypedValue(argument.getLiteral<DateLit>().yearField());
-      }
-    }
-    case DateExtractUnit::kMonth: {
-      if (argument.getTypeID() == TypeID::kDatetime) {
-        return TypedValue(argument.getLiteral<DatetimeLit>().monthField());
-      } else {
-        // argument type is kDate.
-        DCHECK_EQ(TypeID::kDate, argument.getTypeID());
-        return TypedValue(argument.getLiteral<DateLit>().monthField());
-      }
-    }
-    case DateExtractUnit::kDay:
-      return TypedValue(argument.getLiteral<DatetimeLit>().dayField());
-    case DateExtractUnit::kHour:
-      return TypedValue(argument.getLiteral<DatetimeLit>().hourField());
-    case DateExtractUnit::kMinute:
-      return TypedValue(argument.getLiteral<DatetimeLit>().minuteField());
-    case DateExtractUnit::kSecond:
-      return TypedValue(argument.getLiteral<DatetimeLit>().secondField());
-    default: {
-      LOG(FATAL) << "Unsupported DateExtractUnit in DateExtractOperation::applyToChecked().";
-    }
-  }
-}
-
-UncheckedUnaryOperator* DateExtractOperation::makeUncheckedUnaryOperatorForTypeHelper(const Type &type) const {
-  switch (unit_) {
-    case DateExtractUnit::kYear: {
-      if (type.getTypeID() == TypeID::kDatetime) {
-        if (type.isNullable()) {
-          return new DatetimeExtractUncheckedOperator<DateExtractUnit::kYear, true>();
-        } else {
-          return new DatetimeExtractUncheckedOperator<DateExtractUnit::kYear, false>();
-        }
-      } else {
-        DCHECK_EQ(TypeID::kDate, type.getTypeID());
-        // type is kDate.
-        if (type.isNullable()) {
-          return new DateExtractUncheckedOperator<DateExtractUnit::kYear, true>();
-        } else {
-          return new DateExtractUncheckedOperator<DateExtractUnit::kYear, false>();
-        }
-      }
-    }
-    case DateExtractUnit::kMonth: {
-      if (type.getTypeID() == TypeID::kDatetime) {
-        if (type.isNullable()) {
-          return new DatetimeExtractUncheckedOperator<DateExtractUnit::kMonth, true>();
-        } else {
-          return new DatetimeExtractUncheckedOperator<DateExtractUnit::kMonth, false>();
-        }
-      } else {
-        // type is kDate.
-        DCHECK_EQ(TypeID::kDate, type.getTypeID());
-        if (type.isNullable()) {
-          return new DateExtractUncheckedOperator<DateExtractUnit::kMonth, true>();
-        } else {
-          return new DateExtractUncheckedOperator<DateExtractUnit::kMonth, false>();
-        }
-      }
-    }
-    case DateExtractUnit::kDay:
-      if (type.isNullable()) {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kDay, true>();
-      } else {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kDay, false>();
-      }
-    case DateExtractUnit::kHour:
-      if (type.isNullable()) {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kHour, true>();
-      } else {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kHour, false>();
-      }
-    case DateExtractUnit::kMinute:
-      if (type.isNullable()) {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kMinute, true>();
-      } else {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kMinute, false>();
-      }
-    case DateExtractUnit::kSecond:
-      if (type.isNullable()) {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kSecond, true>();
-      } else {
-        return new DatetimeExtractUncheckedOperator<DateExtractUnit::kSecond, false>();
-      }
-    default:
-      FATAL_ERROR("Unsupported DateExtractUnit in DateExtractOperation::makeUncheckedUnaryOperatorForTypeHelper.");
-  }
-}
-
-const Type* DateExtractOperation::resultTypeForArgumentType(const Type &type) const {
-  if (canApplyToType(type)) {
-    if (type.getTypeID() == kDatetime) {
-      return &LongType::Instance(type.isNullable());
-    } else {
-      DCHECK_EQ(kDate, type.getTypeID());
-      return &IntType::Instance(type.isNullable());
+    switch (unit) {
+      case DateExtractUnit::kYear:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DatetimeExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kYear>>(
+                 type, *result_type);
+      case DateExtractUnit::kMonth:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DatetimeExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kMonth>>(
+                 type, *result_type);
+      case DateExtractUnit::kDay:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DatetimeExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kDay>>(
+                 type, *result_type);
+      case DateExtractUnit::kHour:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DatetimeExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kHour>>(
+                 type, *result_type);
+      case DateExtractUnit::kMinute:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DatetimeExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kMinute>>(
+                 type, *result_type);
+      case DateExtractUnit::kSecond:
+        return new UncheckedUnaryOperatorWrapperCodegen<
+             DatetimeExtractFunctor,
+             std::integral_constant<DateExtractUnit, DateExtractUnit::kSecond>>(
+                 type, *result_type);
+      default:
+        LOG(FATAL) << "Unsupported DateExtractUnit for DatetimeType in "
+                   << "DateExtractOperation::makeUncheckedUnaryOperator";
     }
-  } else {
-    return nullptr;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/DateExtractOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/DateExtractOperation.hpp b/types/operations/unary_operations/DateExtractOperation.hpp
index f8c091b..00948a4 100644
--- a/types/operations/unary_operations/DateExtractOperation.hpp
+++ b/types/operations/unary_operations/DateExtractOperation.hpp
@@ -31,101 +31,26 @@
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
 
 #include "catalog/CatalogTypedefs.hpp"
+#include "types/IntType.hpp"
 #include "types/LongType.hpp"
 #include "types/Type.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
 #include "types/operations/Operation.pb.h"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
 #include "utility/Macros.hpp"
+#include "utility/StringUtil.hpp"
 
 namespace quickstep {
 
-class ColumnVector;
-class ValueAccessor;
-
-struct DatetimeLit;
-
-/** \addtogroup Types
- *  @{
- */
-
 enum class DateExtractUnit {
   kYear = 0,
   kMonth,
   kDay,
   kHour,
   kMinute,
-  kSecond
-};
-
-/**
- * @brief UncheckedUnaryOperator for Datetime Extract.
- */
-template <DateExtractUnit unit, bool argument_nullable>
-class DatetimeExtractUncheckedOperator : public UncheckedUnaryOperator {
- public:
-  DatetimeExtractUncheckedOperator()
-      : UncheckedUnaryOperator() {}
-
-  TypedValue applyToTypedValue(const TypedValue &argument) const override;
-
-  TypedValue applyToDataPtr(const void *argument) const override;
-
-  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
-                                     const attribute_id argument_attr_id) const override;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  ColumnVector* applyToValueAccessorForJoin(
-      ValueAccessor *accessor,
-      const bool use_left_relation,
-      const attribute_id argument_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
- private:
-  inline std::int64_t dateExtract(const DatetimeLit &argument) const;
-
-  DISALLOW_COPY_AND_ASSIGN(DatetimeExtractUncheckedOperator);
-};
-
-/**
- * @brief UncheckedUnaryOperator for Date Extract.
- */
-template <DateExtractUnit unit, bool argument_nullable>
-class DateExtractUncheckedOperator : public UncheckedUnaryOperator {
- public:
-  DateExtractUncheckedOperator()
-      : UncheckedUnaryOperator() {}
-
-  TypedValue applyToTypedValue(const TypedValue &argument) const override;
-
-  TypedValue applyToDataPtr(const void *argument) const override;
-
-  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override;
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
-                                     const attribute_id argument_attr_id) const override;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  ColumnVector* applyToValueAccessorForJoin(
-      ValueAccessor *accessor,
-      const bool use_left_relation,
-      const attribute_id argument_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
- private:
-  inline std::int32_t dateExtract(const DateLit &argument) const;
-
-  DISALLOW_COPY_AND_ASSIGN(DateExtractUncheckedOperator);
+  kSecond,
+  kInvalid
 };
 
 /**
@@ -133,55 +58,67 @@ class DateExtractUncheckedOperator : public UncheckedUnaryOperator {
  */
 class DateExtractOperation : public UnaryOperation {
  public:
-  /**
-   * @brief Get a reference to the singleton instance of this Operation for a
-   *        particular DateExtractUnit.
-   *
-   * @param unit The date unit to extract.
-   * @return A reference to the singleton instance of this Operation for the
-   *         specified DateExtractUnit.
-   **/
-  static const DateExtractOperation& Instance(const DateExtractUnit unit);
-
-  serialization::UnaryOperation getProto() const override;
-
-  std::string getName() const override;
-
-  bool canApplyToType(const Type &type) const override {
-    return type.getTypeID() == TypeID::kDatetime || type.getTypeID() == kDate;
-  }
-
-  const Type* resultTypeForArgumentType(const Type &type) const override;
+  DateExtractOperation() {}
 
-  const Type* fixedNullableResultType() const override {
-    return nullptr;
+  std::string getName() const override {
+    return "DateExtract";
   }
 
-  bool resultTypeIsPlausible(const Type &result_type) const override {
-    return result_type.getTypeID() == kLong || result_type.getTypeID() == kInt;
+  std::string getShortName() const override {
+    return "DateExtract";
   }
 
-  const Type* pushDownTypeHint(const Type *type_hint) const override;
-
-  TypedValue applyToChecked(const TypedValue &argument,
-                            const Type &argument_type) const override;
+  std::vector<OperationSignaturePtr> getSignatures() const override {
+    const std::vector<TypeID> target_type_carrier = { kVarChar };
+    return {
+        OperationSignature::Create(getName(), {kDate}, target_type_carrier),
+        OperationSignature::Create(getName(), {kDatetime}, target_type_carrier)
+    };
+  }
 
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorForType(const Type &type) const override {
-    DCHECK(canApplyToType(type));
+  bool canApplyTo(const Type &type,
+                  const std::vector<TypedValue> &static_arguments,
+                  std::string *message) const override {
+    DCHECK_EQ(1u, static_arguments.size());
+    if (parseUnit(static_arguments.front()) == DateExtractUnit::kInvalid) {
+      *message = "Invalid extraction unit for DateExtract";
+      return false;
+    }
+    return true;
+  }
 
-    return makeUncheckedUnaryOperatorForTypeHelper(type);
+  const Type* getResultType(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(UnaryOperation::canApplyTo(type, static_arguments));
+    if (type.getTypeID() == kDatetime) {
+      return &LongType::Instance(type.isNullable());
+    } else {
+      DCHECK_EQ(kDate, type.getTypeID());
+      return &IntType::Instance(type.isNullable());
+    }
   }
 
- private:
-  explicit DateExtractOperation(const DateExtractUnit unit)
-      : UnaryOperation(UnaryOperationID::kDateExtract),
-        unit_(unit) {}
+  UncheckedUnaryOperator* makeUncheckedUnaryOperator(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override;
 
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorForTypeHelper(const Type &type) const;
+ private:
+  static DateExtractUnit parseUnit(const TypedValue &unit_arg) {
+    DCHECK(unit_arg.getTypeID() == kVarChar);
+    const std::string unit_str =
+        ToLower(std::string(static_cast<const char*>(unit_arg.getOutOfLineData())));
+
+    auto it = kNameToUnitMap.find(unit_str);
+    if (it != kNameToUnitMap.end()) {
+      return it->second;
+    } else {
+      return DateExtractUnit::kInvalid;
+    }
+  }
 
-  const DateExtractUnit unit_;
+  static const std::map<std::string, DateExtractUnit> kNameToUnitMap;
 
- private:
   DISALLOW_COPY_AND_ASSIGN(DateExtractOperation);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/NumericCastOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/NumericCastOperation.hpp b/types/operations/unary_operations/NumericCastOperation.hpp
deleted file mode 100644
index 1c5e3d4..0000000
--- a/types/operations/unary_operations/NumericCastOperation.hpp
+++ /dev/null
@@ -1,313 +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_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_
-
-#include <cstddef>
-#include <string>
-#include <utility>
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-#include <vector>
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
-#include "catalog/CatalogTypedefs.hpp"
-#include "storage/ValueAccessor.hpp"
-#include "storage/ValueAccessorUtil.hpp"
-#include "types/DoubleType.hpp"
-#include "types/FloatType.hpp"
-#include "types/IntType.hpp"
-#include "types/LongType.hpp"
-#include "types/Type.hpp"
-#include "types/TypeID.hpp"
-#include "types/TypedValue.hpp"
-#include "types/containers/ColumnVector.hpp"
-#include "types/operations/Operation.pb.h"
-#include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
-#include "utility/Macros.hpp"
-#include "utility/PtrMap.hpp"
-
-#include "glog/logging.h"
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-// TODO(quickstep-team): Support CAST on Datetime, YearMonthInterval, DatetimeInterval, VarChar, Char.
-//                       Currently we only need cast operations on numeric values.
-
-/**
- * @brief UncheckedUnaryOperator for CAST.
- */
-template <class SourceType, bool source_nullability, class TargetType, bool target_nullability>
-class UncheckedNumericCastOperator : public UncheckedUnaryOperator {
- public:
-  UncheckedNumericCastOperator()
-      : UncheckedUnaryOperator(),
-        target_type_(TargetType::Instance(target_nullability)) {
-  }
-
-  TypedValue applyToTypedValue(const TypedValue &argument) const override {
-    if (source_nullability && argument.isNull()) {
-      return TypedValue(TargetType::kStaticTypeID);
-    }
-
-    return TypedValue(static_cast<typename TargetType::cpptype>(
-        argument.getLiteral<typename SourceType::cpptype>()));
-  }
-
-  TypedValue applyToDataPtr(const void *argument) const override {
-    if (source_nullability && argument == nullptr) {
-      return TypedValue(TargetType::kStaticTypeID);
-    }
-
-    return TypedValue(
-        static_cast<const typename TargetType::cpptype>(
-            *static_cast<const typename SourceType::cpptype*>(argument)));
-  }
-
-  ColumnVector* applyToColumnVector(const ColumnVector &argument) const override {
-    DCHECK(NativeColumnVector::UsableForType(target_type_));
-    DCHECK(argument.isNative());
-    const NativeColumnVector &native_argument = static_cast<const NativeColumnVector&>(argument);
-    NativeColumnVector *result = new NativeColumnVector(
-        target_type_,
-        native_argument.size());
-    for (std::size_t pos = 0;
-         pos < native_argument.size();
-         ++pos) {
-      const typename SourceType::cpptype *scalar_arg
-          = static_cast<const typename SourceType::cpptype*>(
-              native_argument.getUntypedValue<source_nullability>(pos));
-      if (source_nullability && (scalar_arg == nullptr)) {
-        result->appendNullValue();
-      } else {
-        *static_cast<typename TargetType::cpptype*>(result->getPtrForDirectWrite())
-            = static_cast<typename TargetType::cpptype>(*scalar_arg);
-      }
-    }
-    return result;
-  }
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-  ColumnVector* applyToValueAccessor(ValueAccessor *accessor,
-                                     const attribute_id argument_attr_id) const override {
-    DCHECK(NativeColumnVector::UsableForType(target_type_));
-    return InvokeOnValueAccessorMaybeTupleIdSequenceAdapter(
-        accessor,
-        [&](auto *accessor) -> ColumnVector* {  // NOLINT(build/c++11)
-      NativeColumnVector *result = new NativeColumnVector(
-          target_type_,
-          accessor->getNumTuples());
-      accessor->beginIteration();
-      while (accessor->next()) {
-        const typename SourceType::cpptype *scalar_arg
-            = static_cast<const typename SourceType::cpptype*>(
-                accessor->template getUntypedValue<source_nullability>(argument_attr_id));
-        if (source_nullability && (scalar_arg == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename TargetType::cpptype*>(result->getPtrForDirectWrite())
-              = static_cast<typename TargetType::cpptype>(*scalar_arg);
-        }
-      }
-      return result;
-    });
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
-
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  ColumnVector* applyToValueAccessorForJoin(
-      ValueAccessor *accessor,
-      const bool use_left_relation,
-      const attribute_id argument_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const override {
-    DCHECK(NativeColumnVector::UsableForType(target_type_));
-    NativeColumnVector *result = new NativeColumnVector(target_type_,
-                                                        joined_tuple_ids.size());
-    InvokeOnValueAccessorNotAdapter(
-        accessor,
-        [&](auto *accessor) -> void {  // NOLINT(build/c++11)
-      for (const std::pair<tuple_id, tuple_id> &joined_pair : joined_tuple_ids) {
-        const typename SourceType::cpptype *scalar_arg
-            = static_cast<const typename SourceType::cpptype*>(
-                accessor->template getUntypedValueAtAbsolutePosition<source_nullability>(
-                    argument_attr_id,
-                    use_left_relation ? joined_pair.first : joined_pair.second));
-        if (source_nullability && (scalar_arg == nullptr)) {
-          result->appendNullValue();
-        } else {
-          *static_cast<typename TargetType::cpptype*>(result->getPtrForDirectWrite())
-              = static_cast<typename TargetType::cpptype>(*scalar_arg);
-        }
-      }
-    });
-    return result;
-  }
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
- private:
-  const Type &target_type_;
-
-  DISALLOW_COPY_AND_ASSIGN(UncheckedNumericCastOperator);
-};
-
-/**
- * @brief UnaryOperation for CAST.
- */
-class NumericCastOperation : public UnaryOperation {
- public:
-  serialization::UnaryOperation getProto() const override {
-    serialization::UnaryOperation proto;
-    proto.set_operation_id(serialization::UnaryOperation::CAST);
-    proto.MutableExtension(serialization::CastOperation::target_type)
-        ->CopyFrom(getTargetType().getProto());
-
-    return proto;
-  }
-
-  /**
-   * @brief Get a reference to the singleton instance of this Operation.
-   *
-   * @param target_type The target type to coerce input values to.
-   * @return A reference to the singleton instance of this Operation.
-   **/
-  static const NumericCastOperation& Instance(const Type &target_type) {
-    static PtrMap<const Type*, NumericCastOperation> instance_map;
-    PtrMap<const Type*, NumericCastOperation>::iterator instance_map_it =
-        instance_map.find(&target_type);
-    if (instance_map_it == instance_map.end()) {
-      instance_map_it = instance_map.insert(&target_type,
-                                            new NumericCastOperation(target_type)).first;
-    }
-    return *(instance_map_it->second);
-  }
-
-  /**
-   * @return The target type for coercion.
-   */
-  const Type& getTargetType() const {
-    return target_type_;
-  }
-
-  std::string getName() const override {
-    return std::string(kUnaryOperationNames[static_cast<std::size_t>(operation_id_)])
-        .append("(")
-        .append(target_type_.getName())
-        .append(")");
-  }
-
-  bool canApplyToType(const Type &type) const override {
-    return target_type_.isCoercibleFrom(type);
-  }
-
-  const Type* resultTypeForArgumentType(const Type &type) const override {
-    if (canApplyToType(type)) {
-      return &target_type_;
-    } else {
-      return nullptr;
-    }
-  }
-
-  const Type* fixedNullableResultType() const override {
-    return &target_type_.getNullableVersion();
-  }
-
-  bool resultTypeIsPlausible(const Type &result_type) const override {
-    return result_type.equals(target_type_)
-           || result_type.equals(target_type_.getNullableVersion());
-  }
-
-  const Type* pushDownTypeHint(const Type *type_hint) const override {
-    return &target_type_;
-  }
-
-  TypedValue applyToChecked(const TypedValue &argument,
-                            const Type &argument_type) const override {
-    return target_type_.coerceValue(argument, argument_type);
-  }
-
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorForType(const Type &type) const override {
-    switch (type.getTypeID()) {
-      case kInt:
-        return makeUncheckedUnaryOperatorHelperForSourceNullability<IntType>(type);
-      case kLong:
-        return makeUncheckedUnaryOperatorHelperForSourceNullability<LongType>(type);
-      case kFloat:
-        return makeUncheckedUnaryOperatorHelperForSourceNullability<FloatType>(type);
-      case kDouble:
-        return makeUncheckedUnaryOperatorHelperForSourceNullability<DoubleType>(type);
-      default:
-        FATAL_ERROR("Unhandled type " << kTypeNames[type.getTypeID()]);
-    }
-  }
-
- private:
-  explicit NumericCastOperation(const Type &target_type)
-      : UnaryOperation(UnaryOperationID::kCast),
-        target_type_(target_type) {}
-
-  template <class SourceType>
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorHelperForSourceNullability(const Type &type) const {
-    if (type.isNullable()) {
-      return makeUncheckedUnaryOperatorHelperForTargetType<SourceType, true>();
-    } else {
-      return makeUncheckedUnaryOperatorHelperForTargetType<SourceType, false>();
-    }
-  }
-
-  template <class SourceType, bool source_nullability>
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorHelperForTargetType() const {
-    switch (target_type_.getTypeID()) {
-      case kInt:
-        return makeUncheckedUnaryOperatorHelperForTargetNullability<SourceType, source_nullability, IntType>();
-      case kLong:
-        return makeUncheckedUnaryOperatorHelperForTargetNullability<SourceType, source_nullability, LongType>();
-      case kFloat:
-        return makeUncheckedUnaryOperatorHelperForTargetNullability<SourceType, source_nullability, FloatType>();
-      case kDouble:
-        return makeUncheckedUnaryOperatorHelperForTargetNullability<SourceType, source_nullability, DoubleType>();
-      default:
-        FATAL_ERROR("Unhandled type " << kTypeNames[target_type_.getTypeID()]);
-    }
-  }
-
-  template <class SourceType, bool source_nullability, class TargetType>
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorHelperForTargetNullability() const {
-    if (target_type_.isNullable()) {
-      return new UncheckedNumericCastOperator<SourceType, source_nullability, TargetType, true>();
-    } else {
-      return new UncheckedNumericCastOperator<SourceType, source_nullability, TargetType, false>();
-    }
-  }
-
-  const Type& target_type_;
-
-  DISALLOW_COPY_AND_ASSIGN(NumericCastOperation);
-};
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_NUMERIC_CAST_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/SubstringOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/SubstringOperation.cpp b/types/operations/unary_operations/SubstringOperation.cpp
index 84f1c8d..408ea2d 100644
--- a/types/operations/unary_operations/SubstringOperation.cpp
+++ b/types/operations/unary_operations/SubstringOperation.cpp
@@ -40,30 +40,27 @@
 
 namespace quickstep {
 
-serialization::UnaryOperation SubstringOperation::getProto() const {
-  serialization::UnaryOperation proto;
-  proto.set_operation_id(serialization::UnaryOperation::SUBSTRING);
-  proto.SetExtension(serialization::SubstringOperation::start_position,
-                     start_position_);
-  proto.SetExtension(serialization::SubstringOperation::substring_length,
-                     substring_length_);
-  return proto;
-}
+UncheckedUnaryOperator* SubstringOperation::makeUncheckedUnaryOperator(
+    const Type &type,
+    const std::vector<TypedValue> &static_arguments) const {
+  DCHECK(UnaryOperation::canApplyTo(type, static_arguments));
 
-UncheckedUnaryOperator* SubstringOperation::makeUncheckedUnaryOperatorForType(
-    const Type &type) const {
-  DCHECK(type.getSuperTypeID() == Type::kAsciiString);
+  std::size_t start_position;
+  std::size_t substring_length;
+  ExtractStaticArguments(static_arguments, &start_position, &substring_length);
 
   const std::size_t input_maximum_length =
-      static_cast<const AsciiStringSuperType&>(type).getStringLength();
+      type.getTypeID() == kChar
+          ? static_cast<const CharType&>(type).getStringLength()
+          : static_cast<const VarCharType&>(type).getStringLength();
   const bool input_null_terminated = (type.getTypeID() == TypeID::kVarChar);
 
-  const Type *result_type = resultTypeForArgumentType(type);
+  const Type *result_type = getResultType(type, static_arguments);
   DCHECK(result_type != nullptr);
 
   return CreateBoolInstantiatedInstance<SubstringUncheckedOperator, UncheckedUnaryOperator>(
-      std::forward_as_tuple(start_position_,
-                            computeMaximumSubstringLength(type),
+      std::forward_as_tuple(start_position,
+                            ComputeMaximumSubstringLength(type, start_position, substring_length),
                             input_maximum_length,
                             *result_type),
       input_null_terminated, type.isNullable());
@@ -108,23 +105,6 @@ TypedValue SubstringUncheckedOperator<null_terminated,
 }
 
 template <bool null_terminated, bool input_nullable>
-TypedValue SubstringUncheckedOperator<null_terminated,
-                                      input_nullable>
-    ::applyToDataPtr(const void *argument) const {
-  if (input_nullable && argument == nullptr) {
-    return TypedValue(result_type_.getTypeID());
-  }
-
-  char *output_ptr = static_cast<char*>(std::malloc(substring_length_));
-  computeSubstring(static_cast<const char*>(argument),
-                   output_ptr);
-
-  return TypedValue::CreateWithOwnedData(result_type_.getTypeID(),
-                                         output_ptr,
-                                         substring_length_);
-}
-
-template <bool null_terminated, bool input_nullable>
 ColumnVector* SubstringUncheckedOperator<null_terminated,
                                          input_nullable>
     ::applyToColumnVector(const ColumnVector &argument) const {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/SubstringOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/SubstringOperation.hpp b/types/operations/unary_operations/SubstringOperation.hpp
index 66f311f..fd5868f 100644
--- a/types/operations/unary_operations/SubstringOperation.hpp
+++ b/types/operations/unary_operations/SubstringOperation.hpp
@@ -22,23 +22,27 @@
 
 #include <algorithm>
 #include <cstddef>
+#include <cstdint>
 #include <cstdlib>
 #include <cstring>
+#include <limits>
 #include <memory>
 #include <unordered_map>
 #include <utility>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
+#include "types/CharType.hpp"
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
 #include "types/TypeID.hpp"
 #include "types/TypedValue.hpp"
+#include "types/VarCharType.hpp"
 #include "types/operations/Operation.pb.h"
 #include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
 #include "utility/HashPair.hpp"
 #include "utility/Macros.hpp"
+#include "utility/StringUtil.hpp"
 
 #include "glog/logging.h"
 
@@ -53,116 +57,81 @@ class ValueAccessor;
  */
 class SubstringOperation : public UnaryOperation {
  public:
-  /**
-   * @brief Get a reference to the singleton instance of this Operation for
-   *        the given (start_position, substring_length) pair.
-   **/
-  static const SubstringOperation& Instance(const std::size_t start_position,
-                                            const std::size_t substring_length) {
-    // TODO(jianqiao): This is a temporary solution that creates a new instance
-    // for each distinct pair of start_position and substring_length arguments.
-    // The number of instances may be unbounded if quickstep continuously accepts
-    // queries that call SUBSTRING with different arguments. It still remains to
-    // design a better long-term solution.
-    const auto hash = [](const auto &pair) {
-      return hash_combine_detail::HashCombiner<std::size_t>::CombineHashes(pair.first, pair.second);
-    };
-    static std::unordered_map<std::pair<std::size_t, std::size_t>,
-                              std::unique_ptr<const SubstringOperation>,
-                              decltype(hash)> instance_map(10, hash);
-
-    const std::pair<std::size_t, std::size_t> key_pair =
-        std::make_pair(start_position, substring_length);
-    auto imit = instance_map.find(key_pair);
-    if (imit != instance_map.end()) {
-      return *imit->second;
-    } else {
-      const SubstringOperation *instance =
-          new SubstringOperation(start_position, substring_length);
-      instance_map.emplace(key_pair,
-                           std::unique_ptr<const SubstringOperation>(instance));
-      return *instance;
-    }
+  SubstringOperation() {}
+
+  std::string getName() const override {
+    return "Substring";
   }
 
-  serialization::UnaryOperation getProto() const override;
+  std::string getShortName() const override {
+    return "Substring";
+  }
 
-  bool canApplyToType(const Type &type) const override {
-    return (type.getSuperTypeID() == Type::kAsciiString);
+  std::vector<OperationSignaturePtr> getSignatures() const override {
+    return {
+        OperationSignature::Create(getName(), {kChar}, {kLong, kLong}),
+        OperationSignature::Create(getName(), {kVarChar}, {kLong, kLong})
+    };
   }
 
-  const Type *resultTypeForArgumentType(const Type &type) const override {
-    if (type.getSuperTypeID() == Type::kAsciiString) {
-      // Result is a Char string.
-      return &TypeFactory::GetType(TypeID::kChar,
-                                   computeMaximumSubstringLength(type),
-                                   type.isNullable());
+  bool canApplyTo(const Type &type,
+                  const std::vector<TypedValue> &static_arguments,
+                  std::string *message) const override {
+    DCHECK(type.getTypeID() == kChar || type.getTypeID() == kVarChar);
+    DCHECK(!static_arguments.empty() && static_arguments[0].getTypeID() == kLong);
+    DCHECK(static_arguments.size() <= 2);
+
+    if (static_arguments[0].getLiteral<std::int64_t>() <= 0) {
+      *message = "The start position must be greater than 0";
+      return false;
     }
-    return nullptr;
-  }
 
-  const Type* fixedNullableResultType() const override {
-    // Result type is not fixed (i.e. can have various lengths).
-    return nullptr;
-  }
+    if (static_arguments.size() == 2) {
+      DCHECK(static_arguments[1].getTypeID() == kLong);
+      if (static_arguments[1].getLiteral<std::int64_t>() <= 0) {
+        *message = "The substring length must be greater than 0";
+        return false;
+      }
+    }
 
-  bool resultTypeIsPlausible(const Type &result_type) const override {
-    // Result can be coerced to Char or VarChar.
-    return (result_type.getSuperTypeID() == Type::kAsciiString);
+    return true;
   }
 
-  const Type* pushDownTypeHint(const Type *type_hint) const override {
-    // Input can only be a string, but we don't know the length.
-    return nullptr;
-  }
+  const Type* getResultType(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override {
+    DCHECK(UnaryOperation::canApplyTo(type, static_arguments));
 
-  TypedValue applyToChecked(const TypedValue &argument,
-                            const Type &argument_type) const override {
-    DCHECK(canApplyToType(argument_type));
-
-    const Type *result_type = resultTypeForArgumentType(argument_type);
-    DCHECK(result_type != nullptr);
-
-    if (argument_type.isNullable() && argument.isNull()) {
-      return result_type->makeNullValue();
-    } else {
-      const std::size_t result_length = computeMaximumSubstringLength(argument_type);
-      char *output_ptr = static_cast<char*>(std::malloc(result_length));
-      const char *input_ptr = static_cast<const char*>(argument.getOutOfLineData());
-
-      const std::size_t string_length = argument.getAsciiStringLength();
-      if (start_position_ >= string_length) {
-        *output_ptr = '\0';
-      } else {
-        const std::size_t actual_substring_length =
-            std::min(string_length - start_position_, substring_length_);
-        std::memcpy(output_ptr, input_ptr + start_position_, actual_substring_length);
-        if (actual_substring_length < result_length) {
-          output_ptr[actual_substring_length] = '\0';
-        }
-      }
+    std::size_t start_position;
+    std::size_t substring_length;
+    ExtractStaticArguments(static_arguments, &start_position, &substring_length);
 
-      return TypedValue::CreateWithOwnedData(result_type->getTypeID(),
-                                             output_ptr,
-                                             result_length);
-    }
+    return &TypeFactory::GetType(TypeID::kChar,
+                                 ComputeMaximumSubstringLength(type, start_position, substring_length),
+                                 type.isNullable());
   }
 
-  UncheckedUnaryOperator* makeUncheckedUnaryOperatorForType(const Type &type) const override;
+  UncheckedUnaryOperator* makeUncheckedUnaryOperator(
+      const Type &type,
+      const std::vector<TypedValue> &static_arguments) const override;
 
  private:
-  /**
-   * @brief Constructor.
-   *
-   * @param input_type The data type of the input argument for substring.
-   * @param start_position The 0-base starting position of the substring.
-   * @param substring_length The substring length.
-   */
-  SubstringOperation(const std::size_t start_position,
-                     const std::size_t substring_length)
-      : UnaryOperation(UnaryOperationID::kSubstring),
-        start_position_(start_position),
-        substring_length_(substring_length) {
+  inline static void ExtractStaticArguments(
+      const std::vector<TypedValue> &static_arguments,
+      std::size_t *start_position,
+      std::size_t *substring_length) {
+    DCHECK_LE(1u, static_arguments.size());
+    DCHECK_GE(2u, static_arguments.size());
+
+    DCHECK(static_arguments[0].getTypeID() == kLong);
+    *start_position =
+        static_cast<std::size_t>(static_arguments[0].getLiteral<std::int64_t>() - 1);
+
+    DCHECK(static_arguments.size() < 2u || static_arguments[1].getTypeID() == kLong);
+    *substring_length =
+        static_arguments.size() < 2u
+            ? std::numeric_limits<std::size_t>::max()
+            : static_cast<std::size_t>(static_arguments[1].getLiteral<std::int64_t>());
   }
 
   /**
@@ -171,19 +140,23 @@ class SubstringOperation : public UnaryOperation {
    *
    * @param type The type of the input, must be either CharType or VarCharType.
    */
-  inline std::size_t computeMaximumSubstringLength(const Type& type) const {
-      DCHECK(type.getSuperTypeID() == Type::kAsciiString);
-
-      // Substring result should have length no greater than the minimum of
-      // (1) the input string length subtract the start position, and
-      // (2) the specified substring length.
-     return std::min(static_cast<const AsciiStringSuperType&>(type).getStringLength() - start_position_,
-                     substring_length_);
+  inline static std::size_t ComputeMaximumSubstringLength(
+      const Type& type,
+      const std::size_t start_position,
+      const std::size_t substring_length) {
+    DCHECK(type.getTypeID() == kChar || type.getTypeID() == kVarChar);
+
+    const std::size_t input_maximum_length =
+        type.getTypeID() == kChar
+            ? static_cast<const CharType&>(type).getStringLength()
+            : static_cast<const VarCharType&>(type).getStringLength();
+
+    // Substring result should have length no greater than the minimum of
+    // (1) the input string length subtract the start position, and
+    // (2) the specified substring length.
+    return std::min(input_maximum_length - start_position, substring_length);
   }
 
-  const std::size_t start_position_;
-  const std::size_t substring_length_;
-
  private:
   DISALLOW_COPY_AND_ASSIGN(SubstringOperation);
 };
@@ -203,8 +176,6 @@ class SubstringUncheckedOperator : public UncheckedUnaryOperator {
 
   TypedValue applyToTypedValue(const TypedValue& argument) const override;
 
-  TypedValue applyToDataPtr(const void *argument) const override;
-
   ColumnVector* applyToColumnVector(const ColumnVector &argument) const override;
 
 #ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/UnaryOperation.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperation.cpp b/types/operations/unary_operations/UnaryOperation.cpp
index af150b3..09b27d9 100644
--- a/types/operations/unary_operations/UnaryOperation.cpp
+++ b/types/operations/unary_operations/UnaryOperation.cpp
@@ -20,28 +20,8 @@
 #include "types/operations/unary_operations/UnaryOperation.hpp"
 
 #include "types/operations/Operation.pb.h"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
 
-serialization::UnaryOperation UnaryOperation::getProto() const {
-  serialization::UnaryOperation proto;
-  switch (operation_id_) {
-    case UnaryOperationID::kNegate:
-      proto.set_operation_id(serialization::UnaryOperation::NEGATE);
-      break;
-    case UnaryOperationID::kCast:
-      FATAL_ERROR("Must use the overridden NumericCastOperation::getProto");
-    case UnaryOperationID::kDateExtract:
-      FATAL_ERROR("Must use the overridden DateExtractOperation::getProto");
-    case UnaryOperationID::kSubstring:
-      FATAL_ERROR("Must use the overridden SubstringOperation::getProto");
-    default:
-      FATAL_ERROR("Unrecognized UnaryOperationID in UnaryOperation::getProto");
-  }
-
-  return proto;
-}
-
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/UnaryOperation.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperation.hpp b/types/operations/unary_operations/UnaryOperation.hpp
index 30a2961..68dc993 100644
--- a/types/operations/unary_operations/UnaryOperation.hpp
+++ b/types/operations/unary_operations/UnaryOperation.hpp
@@ -24,18 +24,10 @@
 #include <string>
 #include <type_traits>
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-#include <utility>
-#include <vector>
-
-#include "storage/StorageBlockInfo.hpp"
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
 #include "catalog/CatalogTypedefs.hpp"
-#include "types/operations/Operation.hpp"
-#include "types/operations/Operation.pb.h"
 #include "types/TypedValue.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
+#include "types/operations/Operation.hpp"
+#include "types/operations/OperationSignature.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -48,6 +40,9 @@ class ValueAccessor;
  *  @{
  */
 
+class UnaryOperation;
+typedef std::shared_ptr<const UnaryOperation> UnaryOperationPtr;
+
 /**
  * @brief A unary operator which can be quickly applied to data items WITHOUT
  *        checking their type.
@@ -69,14 +64,6 @@ class UncheckedUnaryOperator {
   virtual TypedValue applyToTypedValue(const TypedValue &argument) const = 0;
 
   /**
-   * @brief Apply to a data item via a pointer without type-checking.
-   *
-   * @param argument The data item to apply to.
-   * @return The literal result of the operation.
-   **/
-  virtual TypedValue applyToDataPtr(const void *argument) const = 0;
-
-  /**
    * @brief Apply to a vector of values without type-checking.
    *
    * @param argument The argument ColumnVector to apply to.
@@ -96,27 +83,6 @@ class UncheckedUnaryOperator {
                                              const attribute_id argument_attr_id) const = 0;
 #endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_SELECTION
 
-#ifdef QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-  /**
-   * @brief Apply to an attribute of a list of joined tuples in a
-   *        ValueAccessor.
-   *
-   * @param accessor The ValueAccessor to apply to.
-   * @param use_left_relation If true, this UnaryOperation's argument is
-   *        assumed to be taken from the left relation in the pairs of
-   *        joined_tuple_ids. If false, the right relation.
-   * @param argument_attr_id The attribute ID of the argument in accessor.
-   * @param joined_tuple_ids A series of pairs of tuple ids from the left and
-   *        right relations in a join.
-   * @return A ColumnVector of literal results of the operation.
-   **/
-  virtual ColumnVector* applyToValueAccessorForJoin(
-      ValueAccessor *accessor,
-      const bool use_left_relation,
-      const attribute_id argument_attr_id,
-      const std::vector<std::pair<tuple_id, tuple_id>> &joined_tuple_ids) const = 0;
-#endif  // QUICKSTEP_ENABLE_VECTOR_COPY_ELISION_JOIN
-
  protected:
   UncheckedUnaryOperator() {
   }
@@ -134,30 +100,11 @@ class UncheckedUnaryOperator {
 class UnaryOperation : public Operation {
  public:
   /**
-   * @brief Generate a serialized Protocol Buffer representation of
-   *        this UnaryOperation.
-   *
-   * @return The serialized Protocol Buffer representation of this UnaryOperation.
-   **/
-  virtual serialization::UnaryOperation getProto() const;
-
-  /**
-   * @brief Determine the ID of this UnaryOperation.
-   *
-   * @return The ID of this UnaryOperation.
-   **/
-  inline UnaryOperationID getUnaryOperationID() const {
-    return operation_id_;
-  }
-
-  /**
    * @brief Get a human-readable name for this UnaryOperation.
    *
    * @return A human-readable name for this UnaryOperation.
    **/
-  virtual std::string getName() const {
-    return kUnaryOperationNames[static_cast<std::size_t>(operation_id_)];
-  }
+  virtual std::string getName() const = 0;
 
   /**
    * @brief Get a human-readable short name (e.g. "-") for this UnaryOperation.
@@ -165,113 +112,35 @@ class UnaryOperation : public Operation {
    * @return A human-readable short name for this BinaryOperation.
    **/
   virtual std::string getShortName() const {
-    return kUnaryOperationShortNames[static_cast<std::size_t>(operation_id_)];
+    return getName();
   }
 
-  /**
-   * @brief Determine whether this UnaryOperation can apply to the specified
-   *        Type.
-   *
-   * @param type The argument Type to check.
-   * @return Whether this UnaryOperation can apply to type.
-   **/
-  virtual bool canApplyToType(const Type &type) const = 0;
+  virtual std::vector<OperationSignaturePtr> getSignatures() const = 0;
 
-  /**
-   * @brief Determine the Type of the result from applying this UnaryOperation
-   *        to an argument of the specified Type.
-   *
-   * @param type The argument Type to check.
-   * @return The Type of the result from applying this UnaryOperation to type
-   *         (NULL if not applicable).
-   **/
-  virtual const Type* resultTypeForArgumentType(const Type &type) const = 0;
+  virtual bool canApplyTo(const Type &argument_type,
+                          const std::vector<TypedValue> &static_arguments,
+                          std::string *message) const = 0;
 
-  /**
-   * @brief If this UnaryOperation always yields the same Type (or if the ONLY
-   *        difference between 2 possible return Types is nullability), return
-   *        that Type, otherwise return NULL.
-   *
-   * @return The nullable version of this UnaryOperation's fixed result Type,
-   *         if applicable.
-   **/
-  virtual const Type* fixedNullableResultType() const = 0;
+  virtual const Type* getResultType(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments) const = 0;
 
-  /**
-   * @brief Check if a particular Type might possibly be returned by this
-   *        UnaryOperation, assuming an appropriate argument type.
-   * @note A nullable result type may be considered plausible even if a
-   *       particular UnaryOperation never actually returns NULL values, so
-   *       long as the non-nullable version of the type would otherwise be
-   *       plausible.
-   *
-   * @param result_type Check whether this Type can possibly be returned by
-   *        this UnaryOperation.
-   * @return true if result_type can be returned by this UnaryOperation, false
-   *         otherwise.
-   **/
-  virtual bool resultTypeIsPlausible(const Type &result_type) const = 0;
+  virtual UncheckedUnaryOperator* makeUncheckedUnaryOperator(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments) const = 0;
 
-  /**
-   * @brief Get a "hint" Type for the argument to this UnaryOperation based on
-   *        a hint for this UnaryOperation's result type. If possible, returns
-   *        a pointer to a Type that, when given to this UnaryOperation as an
-   *        argument, yields values of the desired type (i.e. calling
-   *        resultTypeForArgumentType() on the returned type should return the
-   *        original type_hint).
-   * @note In some cases (e.g. NumericCastOperation) there may be multiple
-   *       types that can be used as arguments to this UnaryOperation that will
-   *       all yield the desired type_hint. In such cases, this method will
-   *       pick one Type based on its own implementation-specific preference.
-   *
-   * @param type_hint A hint about what Type the result of this UnaryOperation
-   *        should have. May be NULL to indicate no preference.
-   * @return A type hint for the argument to this UnaryOperation based on
-   *         type_hint, or NULL if no suitable Type exists.
-   **/
-  virtual const Type* pushDownTypeHint(const Type *type_hint) const = 0;
-
-  /**
-   * @brief Apply this UnaryOperation to a TypedValue.
-   * @warning It is an error to call this method if this UnaryOperation can not
-   *          be applied to argument_type. If in doubt, check canApplyToType()
-   *          first.
-   *
-   * @param argument The TypedValue to apply to.
-   * @param argument_type The Type that argument belongs to.
-   * @return The literal result of the operation.
-   **/
-  virtual TypedValue applyToChecked(const TypedValue &argument,
-                                    const Type &argument_type) const = 0;
-
-  /**
-   * @brief Create an UncheckedUnaryOperator which can apply to items of the
-   *        specified type.
-   * @warning The resulting UncheckedUnaryOperator performs no type-checking
-   *          whatsoever. Nonetheless, it is useful in situations where many
-   *          data items of the same, known type are to be operated on (for
-   *          example, over many tuples in the same relation).
-   *
-   * @param type The Type of argument to apply to.
-   * @return An UncheckedUnaryOperator which applies this UnaryOperation to
-   *         the specified Type.
-   * @exception OperationInapplicableToType This UnaryOperation is not
-   *            applicable to type.
-   **/
-  virtual UncheckedUnaryOperator* makeUncheckedUnaryOperatorForType(const Type &type) const = 0;
+  bool canApplyTo(
+      const Type &argument_type,
+      const std::vector<TypedValue> &static_arguments) const {
+    std::string message;
+    return canApplyTo(argument_type, static_arguments, &message);
+  }
 
  protected:
-  explicit UnaryOperation(const UnaryOperationID operation_id)
-      : Operation(Operation::kUnaryOperation,
-                  kUnaryOperationNames[
-                      static_cast<typename std::underlying_type<UnaryOperationID>::type>(operation_id)],
-                  kUnaryOperationShortNames[
-                      static_cast<typename std::underlying_type<UnaryOperationID>::type>(operation_id)]),
-        operation_id_(operation_id) {
+  UnaryOperation()
+      : Operation(Operation::kUnaryOperation) {
   }
 
-  const UnaryOperationID operation_id_;
-
  private:
   DISALLOW_COPY_AND_ASSIGN(UnaryOperation);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/UnaryOperationFactory.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationFactory.cpp b/types/operations/unary_operations/UnaryOperationFactory.cpp
index b306061..cebf9e2 100644
--- a/types/operations/unary_operations/UnaryOperationFactory.cpp
+++ b/types/operations/unary_operations/UnaryOperationFactory.cpp
@@ -21,100 +21,54 @@
 
 #include <string>
 
-#include "types/TypeFactory.hpp"
-#include "types/operations/Operation.pb.h"
+#include "types/operations/OperationSignature.hpp"
 #include "types/operations/unary_operations/ArithmeticUnaryOperations.hpp"
-#include "types/operations/unary_operations/NumericCastOperation.hpp"
+#include "types/operations/unary_operations/CMathUnaryOperations.hpp"
+#include "types/operations/unary_operations/CastOperation.hpp"
 #include "types/operations/unary_operations/DateExtractOperation.hpp"
 #include "types/operations/unary_operations/SubstringOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
+#include "types/operations/unary_operations/UnaryOperationWrapper.hpp"
 #include "utility/Macros.hpp"
-
-#include "glog/logging.h"
+#include "utility/StringUtil.hpp"
 
 namespace quickstep {
 
-const UnaryOperation& UnaryOperationFactory::GetUnaryOperation(const UnaryOperationID id) {
-  switch (id) {
-    case UnaryOperationID::kNegate:
-      return NegateUnaryOperation::Instance();
-    case UnaryOperationID::kCast:
-      FATAL_ERROR("Getting a CastOperation through GetUnaryOperation is not supported");
-    case UnaryOperationID::kDateExtract:
-      FATAL_ERROR("Getting a DateExtractOperation through GetUnaryOperation is not supported");
-    case UnaryOperationID::kSubstring:
-      FATAL_ERROR("Getting a SubstringOperation through GetUnaryOperation is not supported");
-    default:
-      FATAL_ERROR("Unknown UnaryOperationID");
-  }
+UnaryOperationFactory::UnaryOperationFactory() {
+  registerUnaryOperation(UnaryOperationPtr(new CastOperation()));
+  registerUnaryOperation(UnaryOperationPtr(new DateExtractOperation()));
+  registerUnaryOperation(UnaryOperationPtr(new SubstringOperation()));
+
+  registerUnaryOperationPack<CMathUnaryFunctorPack>();
+  registerUnaryOperationPack<ArithmeticUnaryFunctorPack>();
 }
 
-bool UnaryOperationFactory::ProtoIsValid(const serialization::UnaryOperation &proto) {
-  // Check that UnaryOperation is fully initialized.
-  if (!proto.IsInitialized()) {
-    return false;
-  }
+void UnaryOperationFactory::registerUnaryOperation(
+    const UnaryOperationPtr &operation) {
+  for (const OperationSignaturePtr op_sig : operation->getSignatures()) {
+    const OperationSignaturePtr normalized_op_sig =
+        OperationSignature::Create(ToLower(op_sig->getName()),
+                                   op_sig->getArgumentTypeIDs(),
+                                   op_sig->getNumStaticArguments());
 
-  // Check that the operation_id is a valid UnaryOperation.
-  if (!proto.UnaryOperationID_IsValid(proto.operation_id())) {
-    return false;
-  }
+    // TODO: print error message for collision
+    unary_operations_.emplace(normalized_op_sig, operation);
 
-  switch (proto.operation_id()) {
-    case serialization::UnaryOperation::NEGATE:
-      return true;
-    case serialization::UnaryOperation::CAST:
-      return proto.HasExtension(serialization::CastOperation::target_type)
-          && TypeFactory::ProtoIsValid(proto.GetExtension(serialization::CastOperation::target_type));
-    case serialization::UnaryOperation::DATE_EXTRACT:
-      return proto.HasExtension(serialization::DateExtractOperation::unit)
-          && DateExtractOperation_Unit_IsValid(proto.GetExtension(serialization::DateExtractOperation::unit));
-    case serialization::UnaryOperation::SUBSTRING:
-      return proto.HasExtension(serialization::SubstringOperation::start_position)
-          && proto.HasExtension(serialization::SubstringOperation::substring_length);
-    default:
-      return false;
+    const auto name_arity_pair = std::make_pair(normalized_op_sig->getName(),
+                                                normalized_op_sig->getArity());
+    name_arity_index_[name_arity_pair].emplace(normalized_op_sig);
   }
 }
 
-const UnaryOperation& UnaryOperationFactory::ReconstructFromProto(
-    const serialization::UnaryOperation &proto) {
-  DCHECK(ProtoIsValid(proto))
-      << "Attempted to create UnaryOperation from an invalid proto description:\n"
-      << proto.DebugString();
-
-  switch (proto.operation_id()) {
-    case serialization::UnaryOperation::NEGATE:
-      return GetUnaryOperation(UnaryOperationID::kNegate);
-    case serialization::UnaryOperation::CAST:
-      return NumericCastOperation::Instance(
-          TypeFactory::ReconstructFromProto(
-              proto.GetExtension(
-                  serialization::CastOperation::target_type)));
-    case serialization::UnaryOperation::DATE_EXTRACT:
-      switch (proto.GetExtension(serialization::DateExtractOperation::unit)) {
-        case serialization::DateExtractOperation::YEAR:
-          return DateExtractOperation::Instance(DateExtractUnit::kYear);
-        case serialization::DateExtractOperation::MONTH:
-          return DateExtractOperation::Instance(DateExtractUnit::kMonth);
-        case serialization::DateExtractOperation::DAY:
-          return DateExtractOperation::Instance(DateExtractUnit::kDay);
-        case serialization::DateExtractOperation::HOUR:
-          return DateExtractOperation::Instance(DateExtractUnit::kHour);
-        case serialization::DateExtractOperation::MINUTE:
-          return DateExtractOperation::Instance(DateExtractUnit::kMinute);
-        case serialization::DateExtractOperation::SECOND:
-          return DateExtractOperation::Instance(DateExtractUnit::kSecond);
-        default:
-          FATAL_ERROR("Unrecognized DateExtractOperation unit in UnaryOperation::ReconstructFromProto");
-      }
-    case serialization::UnaryOperation::SUBSTRING:
-      return SubstringOperation::Instance(
-          proto.GetExtension(serialization::SubstringOperation::start_position),
-          proto.GetExtension(serialization::SubstringOperation::substring_length));
-    default:
-      FATAL_ERROR("Unrecognized UnaryOperationID in UnaryOperation::ReconstructFromProto");
+template <typename UnaryOperationPackT>
+void UnaryOperationFactory::registerUnaryOperationPack() {
+  for (const UnaryOperationPtr &operation : UnaryOperationPackT::GenerateAll()) {
+    registerUnaryOperation(operation);
   }
 }
 
+const UnaryOperationFactory& UnaryOperationFactory::Instance() {
+  static UnaryOperationFactory instance;
+  return instance;
+}
+
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/UnaryOperationFactory.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationFactory.hpp b/types/operations/unary_operations/UnaryOperationFactory.hpp
index 2ce83d4..c586b62 100644
--- a/types/operations/unary_operations/UnaryOperationFactory.hpp
+++ b/types/operations/unary_operations/UnaryOperationFactory.hpp
@@ -20,13 +20,18 @@
 #ifndef QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_FACTORY_HPP_
 #define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_FACTORY_HPP_
 
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
+#include <set>
+#include <unordered_map>
+#include <utility>
+
+#include "types/operations/OperationSignature.hpp"
+#include "types/operations/unary_operations/UnaryOperation.hpp"
+#include "utility/HashPair.hpp"
 #include "utility/Macros.hpp"
 
-namespace quickstep {
+#include "glog/logging.h"
 
-class UnaryOperation;
-namespace serialization { class UnaryOperation; }
+namespace quickstep {
 
 /** \addtogroup Types
  *  @{
@@ -37,38 +42,51 @@ namespace serialization { class UnaryOperation; }
  **/
 class UnaryOperationFactory {
  public:
-  /**
-   * @brief Convenience factory method to get a pointer to a UnaryOperation
-   *        from that UnaryOperation's ID.
-   *
-   * @param id The ID of the desired UnaryOperation.
-   * @return The UnaryOperation corresponding to id.
-   **/
-  static const UnaryOperation& GetUnaryOperation(const UnaryOperationID id);
-
-  /**
-   * @brief Get a reference to a UnaryOperation from that UnaryOperation's
-   *        serialized Protocol Buffer representation.
-   *
-   * @param proto A serialized Protocol Buffer representation of a UnaryOperation,
-   *        originally generated by getProto().
-   * @return The UnaryOperation described by proto.
-   **/
-  static const UnaryOperation& ReconstructFromProto(const serialization::UnaryOperation &proto);
-
-  /**
-   * @brief Check whether a serialization::UnaryOperation is fully-formed and
-   *        all parts are valid.
-   *
-   * @param proto A serialized Protocol Buffer representation of a UnaryOperation,
-   *        originally generated by getProto().
-   * @return Whether proto is fully-formed and valid.
-   **/
-  static bool ProtoIsValid(const serialization::UnaryOperation &proto);
+  static const UnaryOperationFactory& Instance();
+
+  inline bool hasUnaryOperation(
+      const OperationSignaturePtr &op_signature) const {
+    return unary_operations_.find(op_signature) != unary_operations_.end();
+  }
+
+  inline bool hasUnaryOperation(const std::string &name,
+                                const std::size_t arity) const {
+    const auto it = name_arity_index_.find(std::make_pair(name, arity));
+    return it != name_arity_index_.end();
+  }
+
+  inline const UnaryOperationPtr getUnaryOperation(
+      const OperationSignaturePtr &op_signature) const {
+    DCHECK(hasUnaryOperation(op_signature));
+    return unary_operations_.at(op_signature);
+  }
+
+  inline const std::set<OperationSignaturePtr,
+                        OperationSignatureNumStaticArgumentsGreater>&
+      getUnaryOperations(const std::string &name,
+                         const std::size_t arity) const {
+    DCHECK(hasUnaryOperation(name, arity));
+    return name_arity_index_.at(std::make_pair(name, arity));
+  }
 
  private:
   UnaryOperationFactory();
 
+  void registerUnaryOperation(const UnaryOperationPtr &operation);
+
+  template <typename UnaryOperationPackT>
+  void registerUnaryOperationPack();
+
+  std::unordered_map<OperationSignaturePtr,
+                     UnaryOperationPtr,
+                     OperationSignatureHash,
+                     OperationSignatureEqual> unary_operations_;
+
+  std::unordered_map<
+      std::pair<std::string, std::size_t>,
+      std::set<OperationSignaturePtr,
+               OperationSignatureNumStaticArgumentsGreater>> name_arity_index_;
+
   DISALLOW_COPY_AND_ASSIGN(UnaryOperationFactory);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/UnaryOperationID.cpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationID.cpp b/types/operations/unary_operations/UnaryOperationID.cpp
deleted file mode 100644
index b47a848..0000000
--- a/types/operations/unary_operations/UnaryOperationID.cpp
+++ /dev/null
@@ -1,32 +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/unary_operations/UnaryOperationID.hpp"
-
-namespace quickstep {
-
-const char *kUnaryOperationNames[] = {
-  "Negate", "Cast", "DateExtract", "Substring"
-};
-
-const char *kUnaryOperationShortNames[] = {
-  "-", "Cast", "DateExtract", "Substring"
-};
-
-}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/types/operations/unary_operations/UnaryOperationID.hpp
----------------------------------------------------------------------
diff --git a/types/operations/unary_operations/UnaryOperationID.hpp b/types/operations/unary_operations/UnaryOperationID.hpp
deleted file mode 100644
index fa50f50..0000000
--- a/types/operations/unary_operations/UnaryOperationID.hpp
+++ /dev/null
@@ -1,63 +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_UNARY_OPERATIONS_UNARY_OPERATION_ID_HPP_
-#define QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_ID_HPP_
-
-#include <type_traits>
-
-namespace quickstep {
-
-/** \addtogroup Types
- *  @{
- */
-
-/**
- * @brief Concrete UnaryOperations.
- **/
-enum class UnaryOperationID {
-  kNegate = 0,
-  kCast,
-  kDateExtract,
-  kSubstring,
-  kNumUnaryOperationIDs  // Not a real UnaryOperationID, exists for counting purposes.
-};
-
-/**
- * @brief Names of comparisons in the same order as UnaryOperationID.
- * @note Defined out-of-line in UnaryOperation.cpp
- **/
-extern const char *kUnaryOperationNames[
-    static_cast<typename std::underlying_type<UnaryOperationID>::type>(
-        UnaryOperationID::kNumUnaryOperationIDs)];
-
-/**
- * @brief Short names (i.e. mathematical symbols) of comparisons in the same
- *        order as UnaryOperationID.
- * @note Defined out-of-line in UnaryOperation.cpp
- **/
-extern const char *kUnaryOperationShortNames[
-    static_cast<typename std::underlying_type<UnaryOperationID>::type>(
-        UnaryOperationID::kNumUnaryOperationIDs)];
-
-/** @} */
-
-}  // namespace quickstep
-
-#endif  // QUICKSTEP_TYPES_OPERATIONS_UNARY_OPERATIONS_UNARY_OPERATION_ID_HPP_


[5/8] incubator-quickstep git commit: Initial commit

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/27c41625/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index 23cbf1b..085decb 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -143,16 +143,13 @@ typedef struct YYLTYPE {
 #include "types/operations/comparisons/Comparison.hpp"
 #include "types/operations/comparisons/ComparisonFactory.hpp"
 #include "types/operations/comparisons/ComparisonID.hpp"
-#include "types/operations/unary_operations/UnaryOperation.hpp"
-#include "types/operations/unary_operations/UnaryOperationFactory.hpp"
-#include "types/operations/unary_operations/UnaryOperationID.hpp"
 #include "utility/PtrList.hpp"
 #include "utility/PtrVector.hpp"
 
 // Needed for Bison 2.6 and higher, which do not automatically provide this typedef.
 typedef void* yyscan_t;
 
-#line 156 "SqlParser_gen.cpp" /* yacc.c:339  */
+#line 153 "SqlParser_gen.cpp" /* yacc.c:339  */
 
 # ifndef YY_NULLPTR
 #  if defined __cplusplus && 201103L <= __cplusplus
@@ -221,103 +218,104 @@ extern int quickstep_yydebug;
     TOKEN_CSB_TREE = 289,
     TOKEN_BY = 290,
     TOKEN_CASE = 291,
-    TOKEN_CHARACTER = 292,
-    TOKEN_CHECK = 293,
-    TOKEN_COLUMN = 294,
-    TOKEN_CONSTRAINT = 295,
-    TOKEN_COPY = 296,
-    TOKEN_CREATE = 297,
-    TOKEN_CURRENT = 298,
-    TOKEN_DATE = 299,
-    TOKEN_DATETIME = 300,
-    TOKEN_DAY = 301,
-    TOKEN_DECIMAL = 302,
-    TOKEN_DEFAULT = 303,
-    TOKEN_DELETE = 304,
-    TOKEN_DELIMITER = 305,
-    TOKEN_DESC = 306,
-    TOKEN_DISTINCT = 307,
-    TOKEN_DOUBLE = 308,
-    TOKEN_DROP = 309,
-    TOKEN_ELSE = 310,
-    TOKEN_END = 311,
-    TOKEN_ESCAPE_STRINGS = 312,
-    TOKEN_EXISTS = 313,
-    TOKEN_EXTRACT = 314,
-    TOKEN_FALSE = 315,
-    TOKEN_FIRST = 316,
-    TOKEN_FLOAT = 317,
-    TOKEN_FOLLOWING = 318,
-    TOKEN_FOR = 319,
-    TOKEN_FOREIGN = 320,
-    TOKEN_FROM = 321,
-    TOKEN_FULL = 322,
-    TOKEN_GROUP = 323,
-    TOKEN_HASH = 324,
-    TOKEN_HAVING = 325,
-    TOKEN_HOUR = 326,
-    TOKEN_IN = 327,
-    TOKEN_INDEX = 328,
-    TOKEN_INNER = 329,
-    TOKEN_INSERT = 330,
-    TOKEN_INTEGER = 331,
-    TOKEN_INTERVAL = 332,
-    TOKEN_INTO = 333,
-    TOKEN_JOIN = 334,
-    TOKEN_KEY = 335,
-    TOKEN_LAST = 336,
-    TOKEN_LEFT = 337,
-    TOKEN_LIMIT = 338,
-    TOKEN_LONG = 339,
-    TOKEN_MINUTE = 340,
-    TOKEN_MONTH = 341,
-    TOKEN_NULL = 342,
-    TOKEN_NULLS = 343,
-    TOKEN_OFF = 344,
-    TOKEN_ON = 345,
-    TOKEN_ORDER = 346,
-    TOKEN_OUTER = 347,
-    TOKEN_OVER = 348,
-    TOKEN_PARTITION = 349,
-    TOKEN_PARTITIONS = 350,
-    TOKEN_PERCENT = 351,
-    TOKEN_PRECEDING = 352,
-    TOKEN_PRIMARY = 353,
-    TOKEN_PRIORITY = 354,
-    TOKEN_QUIT = 355,
-    TOKEN_RANGE = 356,
-    TOKEN_REAL = 357,
-    TOKEN_REFERENCES = 358,
-    TOKEN_RIGHT = 359,
-    TOKEN_ROW = 360,
-    TOKEN_ROW_DELIMITER = 361,
-    TOKEN_ROWS = 362,
-    TOKEN_SECOND = 363,
-    TOKEN_SELECT = 364,
-    TOKEN_SET = 365,
-    TOKEN_SMA = 366,
-    TOKEN_SMALLINT = 367,
-    TOKEN_SUBSTRING = 368,
-    TOKEN_TABLE = 369,
-    TOKEN_THEN = 370,
-    TOKEN_TIME = 371,
-    TOKEN_TIMESTAMP = 372,
-    TOKEN_TRUE = 373,
-    TOKEN_TUPLESAMPLE = 374,
-    TOKEN_UNBOUNDED = 375,
-    TOKEN_UNIQUE = 376,
-    TOKEN_UPDATE = 377,
-    TOKEN_USING = 378,
-    TOKEN_VALUES = 379,
-    TOKEN_VARCHAR = 380,
-    TOKEN_WHEN = 381,
-    TOKEN_WHERE = 382,
-    TOKEN_WINDOW = 383,
-    TOKEN_WITH = 384,
-    TOKEN_YEAR = 385,
-    TOKEN_YEARMONTH = 386,
-    TOKEN_EOF = 387,
-    TOKEN_LEX_ERROR = 388
+    TOKEN_CAST = 292,
+    TOKEN_CHARACTER = 293,
+    TOKEN_CHECK = 294,
+    TOKEN_COLUMN = 295,
+    TOKEN_CONSTRAINT = 296,
+    TOKEN_COPY = 297,
+    TOKEN_CREATE = 298,
+    TOKEN_CURRENT = 299,
+    TOKEN_DATE = 300,
+    TOKEN_DATETIME = 301,
+    TOKEN_DAY = 302,
+    TOKEN_DECIMAL = 303,
+    TOKEN_DEFAULT = 304,
+    TOKEN_DELETE = 305,
+    TOKEN_DELIMITER = 306,
+    TOKEN_DESC = 307,
+    TOKEN_DISTINCT = 308,
+    TOKEN_DOUBLE = 309,
+    TOKEN_DROP = 310,
+    TOKEN_ELSE = 311,
+    TOKEN_END = 312,
+    TOKEN_ESCAPE_STRINGS = 313,
+    TOKEN_EXISTS = 314,
+    TOKEN_EXTRACT = 315,
+    TOKEN_FALSE = 316,
+    TOKEN_FIRST = 317,
+    TOKEN_FLOAT = 318,
+    TOKEN_FOLLOWING = 319,
+    TOKEN_FOR = 320,
+    TOKEN_FOREIGN = 321,
+    TOKEN_FROM = 322,
+    TOKEN_FULL = 323,
+    TOKEN_GROUP = 324,
+    TOKEN_HASH = 325,
+    TOKEN_HAVING = 326,
+    TOKEN_HOUR = 327,
+    TOKEN_IN = 328,
+    TOKEN_INDEX = 329,
+    TOKEN_INNER = 330,
+    TOKEN_INSERT = 331,
+    TOKEN_INTEGER = 332,
+    TOKEN_INTERVAL = 333,
+    TOKEN_INTO = 334,
+    TOKEN_JOIN = 335,
+    TOKEN_KEY = 336,
+    TOKEN_LAST = 337,
+    TOKEN_LEFT = 338,
+    TOKEN_LIMIT = 339,
+    TOKEN_LONG = 340,
+    TOKEN_MINUTE = 341,
+    TOKEN_MONTH = 342,
+    TOKEN_NULL = 343,
+    TOKEN_NULLS = 344,
+    TOKEN_OFF = 345,
+    TOKEN_ON = 346,
+    TOKEN_ORDER = 347,
+    TOKEN_OUTER = 348,
+    TOKEN_OVER = 349,
+    TOKEN_PARTITION = 350,
+    TOKEN_PARTITIONS = 351,
+    TOKEN_PERCENT = 352,
+    TOKEN_PRECEDING = 353,
+    TOKEN_PRIMARY = 354,
+    TOKEN_PRIORITY = 355,
+    TOKEN_QUIT = 356,
+    TOKEN_RANGE = 357,
+    TOKEN_REAL = 358,
+    TOKEN_REFERENCES = 359,
+    TOKEN_RIGHT = 360,
+    TOKEN_ROW = 361,
+    TOKEN_ROW_DELIMITER = 362,
+    TOKEN_ROWS = 363,
+    TOKEN_SECOND = 364,
+    TOKEN_SELECT = 365,
+    TOKEN_SET = 366,
+    TOKEN_SMA = 367,
+    TOKEN_SMALLINT = 368,
+    TOKEN_SUBSTRING = 369,
+    TOKEN_TABLE = 370,
+    TOKEN_THEN = 371,
+    TOKEN_TIME = 372,
+    TOKEN_TIMESTAMP = 373,
+    TOKEN_TRUE = 374,
+    TOKEN_TUPLESAMPLE = 375,
+    TOKEN_UNBOUNDED = 376,
+    TOKEN_UNIQUE = 377,
+    TOKEN_UPDATE = 378,
+    TOKEN_USING = 379,
+    TOKEN_VALUES = 380,
+    TOKEN_VARCHAR = 381,
+    TOKEN_WHEN = 382,
+    TOKEN_WHERE = 383,
+    TOKEN_WINDOW = 384,
+    TOKEN_WITH = 385,
+    TOKEN_YEAR = 386,
+    TOKEN_YEARMONTH = 387,
+    TOKEN_EOF = 388,
+    TOKEN_LEX_ERROR = 389
   };
 #endif
 
@@ -326,7 +324,7 @@ extern int quickstep_yydebug;
 
 union YYSTYPE
 {
-#line 120 "../SqlParser.ypp" /* yacc.c:355  */
+#line 117 "../SqlParser.ypp" /* yacc.c:355  */
 
   quickstep::ParseString *string_value_;
 
@@ -397,7 +395,7 @@ union YYSTYPE
   quickstep::ParseStatementQuit *quit_statement_;
 
   const quickstep::Comparison *comparison_;
-  const quickstep::UnaryOperation *unary_operation_;
+  quickstep::ParseString *unary_operation_;
   const quickstep::BinaryOperation *binary_operation_;
 
   quickstep::ParseFunctionCall *function_call_;
@@ -426,7 +424,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 430 "SqlParser_gen.cpp" /* yacc.c:355  */
+#line 428 "SqlParser_gen.cpp" /* yacc.c:355  */
 };
 
 typedef union YYSTYPE YYSTYPE;
@@ -455,13 +453,13 @@ int quickstep_yyparse (yyscan_t yyscanner, quickstep::ParseStatement **parsedSta
 #endif /* !YY_QUICKSTEP_YY_SQLPARSER_GEN_HPP_INCLUDED  */
 
 /* Copy the second part of user declarations.  */
-#line 220 "../SqlParser.ypp" /* yacc.c:358  */
+#line 217 "../SqlParser.ypp" /* yacc.c:358  */
 
 /* This header needs YYSTYPE, which is defined by the %union directive above */
 #include "SqlLexer_gen.hpp"
 void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string &feature);
 
-#line 465 "SqlParser_gen.cpp" /* yacc.c:358  */
+#line 463 "SqlParser_gen.cpp" /* yacc.c:358  */
 
 #ifdef short
 # undef short
@@ -705,21 +703,21 @@ union yyalloc
 /* YYFINAL -- State number of the termination state.  */
 #define YYFINAL  47
 /* YYLAST -- Last index in YYTABLE.  */
-#define YYLAST   1346
+#define YYLAST   1446
 
 /* YYNTOKENS -- Number of terminals.  */
-#define YYNTOKENS  145
+#define YYNTOKENS  146
 /* YYNNTS -- Number of nonterminals.  */
-#define YYNNTS  107
+#define YYNNTS  108
 /* YYNRULES -- Number of rules.  */
-#define YYNRULES  289
+#define YYNRULES  291
 /* YYNSTATES -- Number of states.  */
-#define YYNSTATES  536
+#define YYNSTATES  543
 
 /* YYTRANSLATE[YYX] -- Symbol number corresponding to YYX as returned
    by yylex, with out-of-bounds checking.  */
 #define YYUNDEFTOK  2
-#define YYMAXUTOK   388
+#define YYMAXUTOK   389
 
 #define YYTRANSLATE(YYX)                                                \
   ((unsigned int) (YYX) <= YYMAXUTOK ? yytranslate[YYX] : YYUNDEFTOK)
@@ -729,11 +727,11 @@ union yyalloc
 static const yytype_uint8 yytranslate[] =
 {
        0,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-     140,     2,     2,     2,     2,     2,     2,     2,     2,     2,
+     141,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,   144,     2,     2,
-     141,   142,    23,    21,   143,    22,    27,    24,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,     2,     2,   139,
+       2,     2,     2,     2,     2,     2,     2,   145,     2,     2,
+     142,   143,    23,    21,   144,    22,    27,    24,     2,     2,
+       2,     2,     2,     2,     2,     2,     2,     2,     2,   140,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
@@ -766,42 +764,43 @@ static const yytype_uint8 yytranslate[] =
      100,   101,   102,   103,   104,   105,   106,   107,   108,   109,
      110,   111,   112,   113,   114,   115,   116,   117,   118,   119,
      120,   121,   122,   123,   124,   125,   126,   127,   128,   129,
-     130,   131,   132,   133,   134,   135,   136,   137,   138
+     130,   131,   132,   133,   134,   135,   136,   137,   138,   139
 };
 
 #if YYDEBUG
   /* YYRLINE[YYN] -- Source line where rule number YYN was defined.  */
 static const yytype_uint16 yyrline[] =
 {
-       0,   629,   629,   633,   637,   641,   645,   648,   655,   658,
-     661,   664,   667,   670,   673,   676,   679,   682,   688,   694,
-     701,   707,   714,   723,   728,   737,   742,   747,   751,   757,
-     762,   765,   768,   773,   776,   779,   782,   785,   788,   791,
-     794,   797,   800,   812,   815,   818,   836,   856,   859,   862,
-     867,   872,   878,   884,   893,   897,   903,   906,   911,   916,
-     921,   928,   935,   939,   945,   948,   953,   956,   961,   964,
-     969,   972,   991,   994,   999,  1003,  1009,  1012,  1015,  1020,
-    1023,  1030,  1035,  1046,  1051,  1055,  1059,  1065,  1068,  1074,
-    1082,  1085,  1088,  1094,  1099,  1102,  1107,  1111,  1115,  1119,
-    1125,  1130,  1135,  1139,  1145,  1151,  1154,  1159,  1162,  1181,
-    1186,  1190,  1196,  1202,  1208,  1211,  1215,  1221,  1224,  1229,
-    1233,  1239,  1242,  1245,  1250,  1255,  1260,  1263,  1266,  1271,
-    1274,  1277,  1280,  1283,  1286,  1289,  1292,  1297,  1300,  1305,
-    1309,  1313,  1316,  1320,  1323,  1328,  1331,  1336,  1339,  1344,
-    1348,  1354,  1357,  1362,  1365,  1370,  1373,  1378,  1381,  1400,
-    1403,  1408,  1412,  1418,  1424,  1429,  1432,  1437,  1440,  1445,
-    1448,  1453,  1456,  1461,  1462,  1465,  1470,  1471,  1474,  1479,
-    1483,  1489,  1496,  1499,  1502,  1507,  1510,  1513,  1519,  1522,
-    1527,  1532,  1541,  1546,  1555,  1560,  1563,  1568,  1571,  1576,
-    1582,  1588,  1591,  1594,  1597,  1600,  1603,  1609,  1618,  1621,
-    1626,  1629,  1634,  1637,  1642,  1645,  1648,  1651,  1655,  1659,
-    1662,  1665,  1668,  1671,  1676,  1680,  1684,  1687,  1692,  1697,
-    1701,  1707,  1710,  1715,  1719,  1725,  1730,  1734,  1740,  1745,
-    1748,  1753,  1757,  1763,  1766,  1769,  1772,  1784,  1788,  1807,
-    1820,  1835,  1838,  1841,  1844,  1847,  1850,  1855,  1859,  1865,
-    1868,  1873,  1877,  1884,  1887,  1890,  1893,  1896,  1899,  1902,
-    1905,  1908,  1911,  1916,  1927,  1930,  1935,  1938,  1941,  1947,
-    1951,  1957,  1960,  1968,  1971,  1974,  1977,  1983,  1988,  1993
+       0,   627,   627,   631,   635,   639,   643,   646,   653,   656,
+     659,   662,   665,   668,   671,   674,   677,   680,   686,   692,
+     699,   705,   712,   721,   726,   735,   740,   745,   749,   755,
+     760,   763,   766,   771,   774,   777,   780,   783,   786,   789,
+     792,   795,   798,   810,   813,   816,   834,   854,   857,   860,
+     865,   870,   876,   882,   891,   895,   901,   904,   909,   914,
+     919,   926,   933,   937,   943,   946,   951,   954,   959,   962,
+     967,   970,   989,   992,   997,  1001,  1007,  1010,  1013,  1018,
+    1021,  1028,  1033,  1044,  1049,  1053,  1057,  1063,  1066,  1072,
+    1080,  1083,  1086,  1092,  1097,  1100,  1105,  1109,  1113,  1117,
+    1123,  1128,  1133,  1137,  1143,  1149,  1152,  1157,  1160,  1179,
+    1184,  1188,  1194,  1200,  1206,  1209,  1213,  1219,  1222,  1227,
+    1231,  1237,  1240,  1243,  1248,  1253,  1258,  1261,  1264,  1269,
+    1272,  1275,  1278,  1281,  1284,  1287,  1290,  1295,  1298,  1303,
+    1307,  1311,  1314,  1318,  1321,  1326,  1329,  1334,  1337,  1342,
+    1346,  1352,  1355,  1360,  1363,  1368,  1371,  1376,  1379,  1398,
+    1401,  1406,  1410,  1416,  1422,  1427,  1430,  1435,  1438,  1443,
+    1446,  1451,  1454,  1459,  1460,  1463,  1468,  1469,  1472,  1477,
+    1481,  1487,  1494,  1497,  1500,  1505,  1508,  1511,  1517,  1520,
+    1525,  1530,  1539,  1544,  1553,  1558,  1561,  1566,  1569,  1574,
+    1580,  1586,  1589,  1592,  1595,  1598,  1601,  1607,  1616,  1619,
+    1624,  1627,  1632,  1638,  1643,  1646,  1649,  1652,  1656,  1660,
+    1663,  1666,  1669,  1672,  1675,  1680,  1684,  1688,  1691,  1696,
+    1712,  1723,  1731,  1742,  1745,  1750,  1754,  1760,  1765,  1769,
+    1775,  1780,  1783,  1788,  1792,  1798,  1801,  1804,  1807,  1819,
+    1823,  1842,  1855,  1870,  1873,  1876,  1879,  1882,  1885,  1890,
+    1894,  1900,  1903,  1908,  1912,  1919,  1922,  1925,  1928,  1931,
+    1934,  1937,  1940,  1943,  1946,  1951,  1962,  1965,  1970,  1973,
+    1976,  1982,  1986,  1992,  1995,  2003,  2006,  2009,  2012,  2018,
+    2023,  2028
 };
 #endif
 
@@ -819,20 +818,21 @@ static const char *const yytname[] =
   "TOKEN_ADD", "TOKEN_ALL", "TOKEN_ALTER", "TOKEN_AS", "TOKEN_ASC",
   "TOKEN_BIGINT", "TOKEN_BIT", "TOKEN_BITWEAVING", "TOKEN_BLOCKPROPERTIES",
   "TOKEN_BLOCKSAMPLE", "TOKEN_BLOOM_FILTER", "TOKEN_CSB_TREE", "TOKEN_BY",
-  "TOKEN_CASE", "TOKEN_CHARACTER", "TOKEN_CHECK", "TOKEN_COLUMN",
-  "TOKEN_CONSTRAINT", "TOKEN_COPY", "TOKEN_CREATE", "TOKEN_CURRENT",
-  "TOKEN_DATE", "TOKEN_DATETIME", "TOKEN_DAY", "TOKEN_DECIMAL",
-  "TOKEN_DEFAULT", "TOKEN_DELETE", "TOKEN_DELIMITER", "TOKEN_DESC",
-  "TOKEN_DISTINCT", "TOKEN_DOUBLE", "TOKEN_DROP", "TOKEN_ELSE",
-  "TOKEN_END", "TOKEN_ESCAPE_STRINGS", "TOKEN_EXISTS", "TOKEN_EXTRACT",
-  "TOKEN_FALSE", "TOKEN_FIRST", "TOKEN_FLOAT", "TOKEN_FOLLOWING",
-  "TOKEN_FOR", "TOKEN_FOREIGN", "TOKEN_FROM", "TOKEN_FULL", "TOKEN_GROUP",
-  "TOKEN_HASH", "TOKEN_HAVING", "TOKEN_HOUR", "TOKEN_IN", "TOKEN_INDEX",
-  "TOKEN_INNER", "TOKEN_INSERT", "TOKEN_INTEGER", "TOKEN_INTERVAL",
-  "TOKEN_INTO", "TOKEN_JOIN", "TOKEN_KEY", "TOKEN_LAST", "TOKEN_LEFT",
-  "TOKEN_LIMIT", "TOKEN_LONG", "TOKEN_MINUTE", "TOKEN_MONTH", "TOKEN_NULL",
-  "TOKEN_NULLS", "TOKEN_OFF", "TOKEN_ON", "TOKEN_ORDER", "TOKEN_OUTER",
-  "TOKEN_OVER", "TOKEN_PARTITION", "TOKEN_PARTITIONS", "TOKEN_PERCENT",
+  "TOKEN_CASE", "TOKEN_CAST", "TOKEN_CHARACTER", "TOKEN_CHECK",
+  "TOKEN_COLUMN", "TOKEN_CONSTRAINT", "TOKEN_COPY", "TOKEN_CREATE",
+  "TOKEN_CURRENT", "TOKEN_DATE", "TOKEN_DATETIME", "TOKEN_DAY",
+  "TOKEN_DECIMAL", "TOKEN_DEFAULT", "TOKEN_DELETE", "TOKEN_DELIMITER",
+  "TOKEN_DESC", "TOKEN_DISTINCT", "TOKEN_DOUBLE", "TOKEN_DROP",
+  "TOKEN_ELSE", "TOKEN_END", "TOKEN_ESCAPE_STRINGS", "TOKEN_EXISTS",
+  "TOKEN_EXTRACT", "TOKEN_FALSE", "TOKEN_FIRST", "TOKEN_FLOAT",
+  "TOKEN_FOLLOWING", "TOKEN_FOR", "TOKEN_FOREIGN", "TOKEN_FROM",
+  "TOKEN_FULL", "TOKEN_GROUP", "TOKEN_HASH", "TOKEN_HAVING", "TOKEN_HOUR",
+  "TOKEN_IN", "TOKEN_INDEX", "TOKEN_INNER", "TOKEN_INSERT",
+  "TOKEN_INTEGER", "TOKEN_INTERVAL", "TOKEN_INTO", "TOKEN_JOIN",
+  "TOKEN_KEY", "TOKEN_LAST", "TOKEN_LEFT", "TOKEN_LIMIT", "TOKEN_LONG",
+  "TOKEN_MINUTE", "TOKEN_MONTH", "TOKEN_NULL", "TOKEN_NULLS", "TOKEN_OFF",
+  "TOKEN_ON", "TOKEN_ORDER", "TOKEN_OUTER", "TOKEN_OVER",
+  "TOKEN_PARTITION", "TOKEN_PARTITIONS", "TOKEN_PERCENT",
   "TOKEN_PRECEDING", "TOKEN_PRIMARY", "TOKEN_PRIORITY", "TOKEN_QUIT",
   "TOKEN_RANGE", "TOKEN_REAL", "TOKEN_REFERENCES", "TOKEN_RIGHT",
   "TOKEN_ROW", "TOKEN_ROW_DELIMITER", "TOKEN_ROWS", "TOKEN_SECOND",
@@ -869,7 +869,7 @@ static const char *const yytname[] =
   "opt_order_direction", "opt_nulls_first", "opt_where_clause",
   "where_clause", "or_expression", "and_expression", "not_expression",
   "predicate_expression_base", "add_expression", "multiply_expression",
-  "unary_expression", "expression_base", "function_call",
+  "unary_expression", "expression_base", "function_call", "cast_function",
   "extract_function", "substr_function", "case_expression",
   "simple_when_clause_list", "simple_when_clause",
   "searched_when_clause_list", "searched_when_clause", "opt_else_clause",
@@ -899,15 +899,15 @@ static const yytype_uint16 yytoknum[] =
      350,   351,   352,   353,   354,   355,   356,   357,   358,   359,
      360,   361,   362,   363,   364,   365,   366,   367,   368,   369,
      370,   371,   372,   373,   374,   375,   376,   377,   378,   379,
-     380,   381,   382,   383,   384,   385,   386,   387,   388,    59,
-      10,    40,    41,    44,    37
+     380,   381,   382,   383,   384,   385,   386,   387,   388,   389,
+      59,    10,    40,    41,    44,    37
 };
 # endif
 
-#define YYPACT_NINF -234
+#define YYPACT_NINF -240
 
 #define yypact_value_is_default(Yystate) \
-  (!!((Yystate) == (-234)))
+  (!!((Yystate) == (-240)))
 
 #define YYTABLE_NINF -130
 
@@ -918,60 +918,61 @@ static const yytype_uint16 yytoknum[] =
      STATE-NUM.  */
 static const yytype_int16 yypact[] =
 {
-     168,  -234,  -234,   -58,   181,   -19,    40,   -37,    59,  -234,
-     123,   181,   181,  -234,   208,   127,  -234,  -234,  -234,  -234,
-    -234,  -234,  -234,  -234,  -234,  -234,    -2,    95,   -46,   233,
-     181,  -234,  -234,   113,   181,   181,   181,   181,   181,  -234,
-    -234,   667,   139,   114,  -234,   236,   128,  -234,  -234,  -234,
-     189,    95,   183,  -234,  -234,  -234,  -234,    60,   272,   196,
-     138,   191,  -234,    80,  -234,  -234,   296,   299,  -234,  -234,
-    -234,   757,   187,  -234,   242,  -234,  -234,   190,  -234,  -234,
-     310,  -234,  -234,  -234,  -234,   205,  -234,  -234,   214,   277,
-     850,   355,   290,   219,  -234,  -234,   182,    20,  -234,  -234,
-     269,  -234,  -234,  -234,  -234,  -234,  1030,    -9,   181,   181,
-     228,   181,   181,  -234,   366,    89,   174,   241,   181,   181,
-     577,  -234,  -234,   235,   181,  -234,  -234,  -234,   577,    42,
-      17,  -234,   370,  -234,   126,   126,  1120,   371,  -234,   243,
-      30,  -234,    34,   191,  1120,  -234,  -234,   181,  1120,  -234,
-    -234,  -234,  -234,  1120,    35,   299,  -234,   181,   359,   -56,
-    -234,   368,  -234,   270,  -234,   110,  -234,   270,  -234,   181,
-      -1,   181,   181,   247,  -234,   248,  -234,   141,  1158,   940,
-     228,   487,   378,   381,  -234,  -234,   545,   375,  1210,   143,
-       8,  1120,    58,  -234,  1120,  -234,   330,   254,  -234,  -234,
-    -234,  -234,  -234,  -234,   327,  -234,    50,   255,  -234,  -234,
-      16,   245,   153,  -234,   256,   245,   -14,   331,  -234,  -234,
-      20,  -234,   306,  -234,  -234,   264,  1120,  -234,   268,   151,
-     181,  -234,  1120,  -234,   181,  -234,  -234,   271,   325,   328,
-     274,  -234,  -234,  -234,    31,   181,   291,    -1,   181,  -234,
-      14,  -234,  -234,     2,   326,   577,   577,    62,  -234,  -234,
-    -234,  -234,  -234,  -234,  -234,  -234,  1120,   279,  1120,    39,
-    -234,   154,   289,  1120,    25,  -234,   360,   268,  -234,  -234,
-    1120,   415,  -234,   133,   181,  -234,  -234,   332,  -234,   334,
-     335,   340,    34,  -234,   418,   420,   245,   388,   358,   394,
-     293,   342,  -234,   156,  -234,  1120,  -234,   268,  -234,   577,
-     295,   298,   181,   437,     1,   158,  -234,   167,   416,   144,
-    -234,   301,   311,  -234,   357,   309,  1210,  -234,   362,   181,
-    -234,  -234,    14,  -234,  -234,   381,  -234,  -234,  -234,  1120,
-     312,    76,   850,  -234,   268,   363,  -234,  -234,  1210,   313,
-     268,  1120,  -234,    32,   -13,  -234,  -234,  -234,  -234,  -234,
-      34,   153,   351,   356,  -234,  1120,   577,   364,  1120,  -234,
-     419,   159,  -234,   268,     7,   181,   181,   169,  -234,  -234,
-    -234,  -234,  -234,  -234,  -234,   173,  -234,   181,  -234,  -234,
-    -234,  -234,   315,    -1,   422,   365,  -234,   577,  -234,  -234,
-     322,  -234,   237,   850,  -234,  1120,   171,  -234,  -234,  1210,
-     268,  -234,   458,  -234,   372,  -234,  -234,   329,   378,   429,
-     382,   329,  1120,  -234,  -234,  -234,   452,  -234,   175,   177,
-    -234,   469,     1,  -234,   181,  -234,  -234,   337,   435,  -234,
-       9,   181,  1120,   179,   268,  -234,   184,   341,   577,  1120,
-     472,   349,   343,  -234,   178,    22,   376,  -234,  -234,  -234,
-     192,  -234,  -234,  -234,  -234,    21,   181,   -16,  -234,   346,
-     268,  -234,  -234,  -234,   378,   343,  -234,   181,  -234,   349,
-    -234,  1120,  -234,  -234,   403,   396,   389,   400,   494,   181,
-    -234,   181,  -234,  -234,   181,  -234,   207,  -234,  -234,   373,
-    -234,   473,  -234,  -234,   115,  -234,  -234,  -234,  -234,    41,
-     374,  -234,   209,  -234,   181,   383,  -234,  -234,   438,   395,
-     439,  -234,   181,  -234,   211,   306,  -234,  -234,  -234,   215,
-     411,   377,  -234,   505,  -234,  -234
+     217,  -240,  -240,   -47,   249,   -25,    41,   -38,    54,  -240,
+      45,   249,   249,  -240,   121,   119,  -240,  -240,  -240,  -240,
+    -240,  -240,  -240,  -240,  -240,  -240,   -21,    51,   113,   194,
+     249,  -240,  -240,   109,   249,   249,   249,   249,   249,  -240,
+    -240,   727,    86,    75,  -240,   198,    91,  -240,  -240,  -240,
+     154,    51,   153,  -240,  -240,  -240,  -240,    37,   266,   180,
+     137,   163,  -240,   -44,  -240,  -240,   277,   313,  -240,  -240,
+    -240,   820,   189,   191,  -240,   239,  -240,  -240,   192,  -240,
+    -240,   332,  -240,  -240,  -240,  -240,   202,  -240,  -240,   226,
+     259,   913,   355,   297,   227,  -240,  -240,   200,    -3,  -240,
+    -240,   271,  -240,  -240,  -240,  -240,  -240,  -240,  1111,    25,
+     249,   249,   230,   249,   249,  -240,   366,   178,   224,   240,
+     249,   249,   634,  -240,  -240,   237,   249,  -240,  -240,  -240,
+     634,    46,    -8,  -240,  1204,   374,  -240,   100,   100,  1204,
+     375,  -240,   242,    21,  -240,    23,   163,  1204,  -240,  -240,
+     249,  1204,  -240,  -240,  -240,  -240,  1204,    33,   313,  -240,
+     249,   405,     4,  -240,   372,  -240,   269,  -240,   138,  -240,
+     269,  -240,   249,    76,   249,   249,   244,  -240,   245,  -240,
+     143,  1309,  1018,   230,   528,   380,   381,  -240,  -240,   489,
+     371,  1257,   146,    53,  1204,    50,  -240,  1204,  -240,   330,
+     195,   250,  -240,  -240,  -240,  -240,  -240,  -240,   323,  -240,
+      36,   253,  -240,  -240,    19,    83,   -10,  -240,   254,    83,
+       3,   325,  -240,  -240,    -3,  -240,   301,  -240,  -240,   260,
+    1204,  -240,   241,   148,   249,  -240,  1204,  -240,   249,  -240,
+    -240,   255,   316,   318,   263,  -240,  -240,  -240,    27,   249,
+     278,    76,   249,  -240,   234,  -240,  -240,     7,   289,   634,
+     634,   190,  -240,  -240,  -240,  -240,  -240,  -240,  -240,  -240,
+    1204,   264,  1204,    34,  -240,   151,   283,  1204,    55,  -240,
+     353,   241,  -240,  1309,  -240,  1204,   409,  -240,     8,   249,
+    -240,  -240,   319,  -240,   320,   321,   335,    23,  -240,   415,
+     416,    83,   384,   349,   389,   287,   334,  -240,   171,  -240,
+    1204,  -240,   241,  -240,   634,   291,   292,   249,   430,   117,
+     173,  -240,   175,   413,   149,  -240,   298,   306,  -240,   343,
+     303,  1257,  -240,   357,   249,  -240,  -240,   234,  -240,  -240,
+     381,  -240,  -240,  -240,  1204,   307,    70,   913,  -240,   241,
+     358,  -240,  -240,  1257,   308,   241,  1204,  -240,   309,    26,
+     -20,  -240,  -240,  -240,  -240,  -240,    23,   -10,   352,   359,
+    -240,  1204,   634,   360,  1204,  -240,   419,    92,  -240,   241,
+      14,   249,   249,   181,  -240,  -240,  -240,  -240,  -240,  -240,
+    -240,   193,  -240,   249,  -240,  -240,  -240,  -240,   324,    76,
+     424,   362,  -240,   634,  -240,  -240,   329,  -240,   218,   913,
+    -240,  1204,   185,  -240,  -240,  1257,   241,  -240,  -240,   458,
+    -240,   376,  -240,  -240,   331,   380,   427,   385,   331,  1204,
+    -240,  -240,  -240,   457,  -240,   197,   203,  -240,   472,   117,
+    -240,   249,  -240,  -240,   336,   439,  -240,    30,   249,  1204,
+     205,   241,  -240,   207,   338,   634,  1204,   475,   350,   339,
+    -240,   168,    10,   377,  -240,  -240,  -240,   210,  -240,  -240,
+    -240,  -240,    12,   249,    -6,  -240,   342,   241,  -240,  -240,
+    -240,   380,   339,  -240,   249,  -240,   350,  -240,  1204,  -240,
+    -240,   395,   387,   382,   388,   487,   249,  -240,   249,  -240,
+    -240,   249,  -240,   213,  -240,  -240,   367,  -240,   481,  -240,
+    -240,    18,  -240,  -240,  -240,  -240,    17,   373,  -240,   215,
+    -240,   249,   378,  -240,  -240,   445,   407,   447,  -240,   249,
+    -240,   220,   301,  -240,  -240,  -240,   222,   418,   379,  -240,
+     514,  -240,  -240
 };
 
   /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM.
@@ -979,92 +980,93 @@ static const yytype_int16 yypact[] =
      means the default is an error.  */
 static const yytype_uint16 yydefact[] =
 {
-       0,     6,   289,     0,     0,     0,     0,     0,     0,    18,
+       0,     6,   291,     0,     0,     0,     0,     0,     0,    18,
      114,     0,     0,     7,     0,     0,    15,     8,    10,    11,
-      13,    14,     9,    17,    12,    16,     0,   107,     0,   287,
-       0,   281,   282,     0,     0,     0,     0,     0,     0,   115,
+      13,    14,     9,    17,    12,    16,     0,   107,     0,   289,
+       0,   283,   284,     0,     0,     0,     0,     0,     0,   115,
      116,     0,     0,   109,   110,     0,   147,     1,     3,     2,
-       0,   107,     0,   105,     5,     4,   288,     0,     0,     0,
-       0,   188,    25,     0,   247,   244,     0,   273,   117,    40,
-      29,     0,     0,    30,    31,    34,    36,     0,    37,    39,
-       0,    41,   243,    35,    38,     0,    32,    33,     0,     0,
-       0,     0,     0,   118,   119,   223,   123,   209,   211,   213,
-     216,   219,   220,   221,   215,   214,     0,   259,     0,     0,
-       0,     0,     0,   106,     0,     0,     0,    94,     0,     0,
-       0,   101,   189,     0,     0,    91,   245,   246,     0,     0,
-     239,   236,     0,    43,     0,   248,     0,     0,    44,     0,
-       0,   250,     0,   188,     0,   274,   275,     0,     0,   122,
-     277,   278,   276,     0,     0,     0,   212,     0,     0,   188,
-     103,     0,   111,     0,   112,     0,   279,     0,   108,     0,
-       0,     0,     0,     0,    93,    66,    27,     0,     0,     0,
-       0,     0,   190,   192,   194,   196,     0,   214,     0,     0,
-       0,     0,   239,   233,     0,   237,     0,     0,   253,   254,
-     255,   252,   256,   251,     0,   249,     0,     0,   125,   222,
-       0,     0,   149,   138,   124,   143,   126,   151,   120,   121,
-     208,   210,   165,   217,   260,     0,     0,   224,   241,     0,
-       0,   100,     0,   148,     0,    92,    19,     0,     0,     0,
-       0,    20,    21,    22,     0,     0,     0,    64,     0,    42,
-      56,   195,   203,     0,     0,     0,     0,     0,   263,   265,
-     266,   267,   268,   264,   269,   271,     0,     0,     0,     0,
-     257,     0,     0,     0,     0,   234,     0,   240,   232,    45,
-       0,     0,    46,   129,     0,   139,   145,   135,   130,   131,
-     133,     0,     0,   142,     0,     0,   141,     0,   153,     0,
-       0,   167,   225,     0,   226,     0,   102,   104,   280,     0,
-       0,     0,     0,     0,     0,     0,   261,     0,   259,     0,
-      63,    65,    68,    28,     0,     0,     0,    47,     0,     0,
-      49,    55,    57,    26,   202,   191,   193,   270,   272,     0,
-       0,     0,     0,   204,   201,     0,   200,    90,     0,     0,
-     238,     0,   231,     0,     0,   144,   146,   136,   132,   134,
-       0,   150,     0,     0,   140,     0,     0,   155,     0,   218,
-       0,   169,   227,   242,     0,     0,     0,     0,    96,   285,
-     286,   284,   283,    97,    95,     0,    67,     0,    83,    84,
-      85,    86,    87,     0,     0,    70,    48,     0,    51,    50,
-       0,    54,     0,     0,   206,     0,     0,   199,   258,     0,
-     235,   228,     0,   229,     0,   127,   128,   152,   154,     0,
-     157,   166,     0,   172,   171,   164,     0,    61,     0,     0,
-      58,     0,     0,   262,     0,    24,    62,     0,     0,    23,
-       0,     0,     0,     0,   197,   205,     0,     0,     0,     0,
-       0,   159,   168,   179,   182,     0,     0,    59,    98,    99,
-       0,    74,    76,    77,    78,     0,     0,     0,    52,     0,
-     198,   207,    89,   230,   137,   156,   158,     0,   113,   160,
-     161,     0,   183,   184,   185,     0,     0,     0,     0,     0,
-      88,     0,    82,    80,     0,    79,     0,    72,    73,     0,
-      53,     0,   162,   180,     0,   181,   173,   175,   174,     0,
-       0,    75,     0,    69,     0,     0,   186,   187,     0,     0,
-       0,   170,     0,    81,     0,   165,   176,   178,   177,     0,
-       0,     0,    60,     0,   163,    71
+       0,   107,     0,   105,     5,     4,   290,     0,     0,     0,
+       0,   188,    25,     0,   249,   246,     0,   275,   117,    40,
+      29,     0,     0,     0,    30,    31,    34,    36,     0,    37,
+      39,     0,    41,   245,    35,    38,     0,    32,    33,     0,
+       0,     0,     0,     0,   118,   119,   224,   123,   209,   211,
+     213,   216,   219,   220,   221,   222,   215,   214,     0,   261,
+       0,     0,     0,     0,     0,   106,     0,     0,     0,    94,
+       0,     0,     0,   101,   189,     0,     0,    91,   247,   248,
+       0,     0,   241,   238,     0,     0,    43,     0,   250,     0,
+       0,    44,     0,     0,   252,     0,   188,     0,   276,   277,
+       0,     0,   122,   279,   280,   278,     0,     0,     0,   212,
+       0,     0,   188,   103,     0,   111,     0,   112,     0,   281,
+       0,   108,     0,     0,     0,     0,     0,    93,    66,    27,
+       0,     0,     0,     0,     0,   190,   192,   194,   196,     0,
+     214,     0,     0,     0,     0,   241,   235,     0,   239,     0,
+       0,     0,   255,   256,   257,   254,   258,   253,     0,   251,
+       0,     0,   125,   223,     0,     0,   149,   138,   124,   143,
+     126,   151,   120,   121,   208,   210,   165,   217,   262,     0,
+       0,   225,   243,     0,     0,   100,     0,   148,     0,    92,
+      19,     0,     0,     0,     0,    20,    21,    22,     0,     0,
+       0,    64,     0,    42,    56,   195,   203,     0,     0,     0,
+       0,     0,   265,   267,   268,   269,   270,   266,   271,   273,
+       0,     0,     0,     0,   259,     0,     0,     0,     0,   236,
+       0,   242,   234,     0,    45,     0,     0,    46,   129,     0,
+     139,   145,   135,   130,   131,   133,     0,     0,   142,     0,
+       0,   141,     0,   153,     0,     0,   167,   226,     0,   227,
+       0,   102,   104,   282,     0,     0,     0,     0,     0,     0,
+       0,   263,     0,   261,     0,    63,    65,    68,    28,     0,
+       0,     0,    47,     0,     0,    49,    55,    57,    26,   202,
+     191,   193,   272,   274,     0,     0,     0,     0,   204,   201,
+       0,   200,    90,     0,     0,   240,     0,   233,     0,     0,
+       0,   144,   146,   136,   132,   134,     0,   150,     0,     0,
+     140,     0,     0,   155,     0,   218,     0,   169,   228,   244,
+       0,     0,     0,     0,    96,   287,   288,   286,   285,    97,
+      95,     0,    67,     0,    83,    84,    85,    86,    87,     0,
+       0,    70,    48,     0,    51,    50,     0,    54,     0,     0,
+     206,     0,     0,   199,   260,     0,   237,   229,   230,     0,
+     231,     0,   127,   128,   152,   154,     0,   157,   166,     0,
+     172,   171,   164,     0,    61,     0,     0,    58,     0,     0,
+     264,     0,    24,    62,     0,     0,    23,     0,     0,     0,
+       0,   197,   205,     0,     0,     0,     0,     0,   159,   168,
+     179,   182,     0,     0,    59,    98,    99,     0,    74,    76,
+      77,    78,     0,     0,     0,    52,     0,   198,   207,    89,
+     232,   137,   156,   158,     0,   113,   160,   161,     0,   183,
+     184,   185,     0,     0,     0,     0,     0,    88,     0,    82,
+      80,     0,    79,     0,    72,    73,     0,    53,     0,   162,
+     180,     0,   181,   173,   175,   174,     0,     0,    75,     0,
+      69,     0,     0,   186,   187,     0,     0,     0,   170,     0,
+      81,     0,   165,   176,   178,   177,     0,     0,     0,    60,
+       0,   163,    71
 };
 
   /* YYPGOTO[NTERM-NUM].  */
 static const yytype_int16 yypgoto[] =
 {
-    -234,  -234,  -234,  -234,  -234,  -234,  -234,  -234,   -94,  -234,
-     338,   185,  -234,  -234,  -233,  -234,  -234,  -234,  -234,  -234,
-    -234,    47,    36,  -234,  -234,  -234,  -234,  -234,  -234,  -234,
-    -234,  -234,  -234,  -234,  -234,   292,  -234,   474,  -234,  -234,
-     414,    11,  -234,  -234,  -234,   386,  -234,   -97,  -234,  -234,
-    -189,   166,  -180,   -10,  -234,  -234,  -234,  -234,  -234,  -234,
-    -234,    52,    10,  -234,  -234,  -234,  -234,  -234,  -234,    83,
-      53,  -234,  -234,   -75,  -234,  -116,   278,   282,   361,   -35,
-     393,   390,   436,  -137,  -234,  -234,  -234,  -234,   352,  -234,
-     417,   354,  -207,  -185,   413,   140,  -119,  -234,  -234,  -234,
-    -234,  -234,  -120,    -4,   120,  -234,  -234
+    -240,  -240,  -240,  -240,  -240,  -240,  -240,  -240,  -127,  -240,
+    -153,   204,  -240,  -240,  -239,  -240,  -240,  -240,  -240,  -240,
+    -240,    56,    28,  -240,  -240,  -240,  -240,  -240,  -240,  -240,
+    -240,  -240,  -240,  -240,  -240,   296,  -240,   480,  -240,  -240,
+     426,     9,  -240,  -240,  -240,   392,  -240,   -99,  -240,  -240,
+    -165,   174,  -168,    -9,  -240,  -240,  -240,  -240,  -240,  -240,
+    -240,    57,    13,  -240,  -240,  -240,  -240,  -240,  -240,    88,
+      58,  -240,  -240,   -27,  -240,  -129,   293,   294,   369,   -35,
+     402,   399,   448,  -134,  -240,  -240,  -240,  -240,  -240,   363,
+    -240,   425,   364,  -216,  -187,   422,   150,  -120,  -240,  -240,
+    -240,  -240,  -240,  -121,    -4,   124,  -240,  -240
 };
 
   /* YYDEFGOTO[NTERM-NUM].  */
 static const yytype_int16 yydefgoto[] =
 {
-      -1,    14,    15,    16,    17,    18,    19,    20,   176,   177,
-      91,   331,   332,   333,   241,   321,   322,   246,   395,   439,
-     499,   460,   461,   462,   463,   464,   392,   435,    21,    22,
-     174,   315,    23,    24,   159,   160,    25,    53,    26,    43,
-      44,   139,    41,    92,    93,    94,   143,    95,   296,   291,
-     212,   213,   285,   286,   214,   298,   367,   420,   451,   478,
-     479,   480,   300,   301,   371,   425,   426,   488,   521,   452,
-     453,   484,   505,   121,   122,   182,   183,   184,   185,   186,
-      97,    98,    99,   100,   101,   102,   103,   192,   193,   130,
-     131,   196,   229,   104,   204,   271,   105,   317,   268,   106,
-     148,   153,   165,   107,   383,    28,    29
+      -1,    14,    15,    16,    17,    18,    19,    20,   179,   180,
+      92,   336,   337,   338,   245,   326,   327,   250,   401,   446,
+     506,   467,   468,   469,   470,   471,   398,   442,    21,    22,
+     177,   320,    23,    24,   162,   163,    25,    53,    26,    43,
+      44,   142,    41,    93,    94,    95,   146,    96,   301,   296,
+     216,   217,   290,   291,   218,   303,   373,   427,   458,   485,
+     486,   487,   305,   306,   377,   432,   433,   495,   528,   459,
+     460,   491,   512,   123,   124,   185,   186,   187,   188,   189,
+      98,    99,   100,   101,   102,   103,   104,   105,   195,   196,
+     132,   133,   199,   233,   106,   208,   275,   107,   322,   272,
+     108,   151,   156,   168,   109,   389,    28,    29
 };
 
   /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM.  If
@@ -1072,374 +1074,396 @@ static const yytype_int16 yydefgoto[] =
      number is the opposite.  If YYTABLE_NINF, syntax error.  */
 static const yytype_int16 yytable[] =
 {
-      33,   187,    45,   270,   189,   215,    96,    42,    46,   187,
-     255,    27,   190,   164,   320,   255,   255,   255,   157,   303,
-      31,   283,    32,   294,   324,    31,    57,    32,   492,   485,
-      59,    60,    61,    62,    63,   293,   129,    51,    31,    31,
-      32,    32,   237,   150,   151,   211,   145,   146,   518,   345,
-     493,   145,   146,   145,   146,   140,   412,   325,   497,    34,
-     187,    30,   187,   145,   146,   253,   379,   326,   217,   238,
-     486,   145,   146,   215,   125,   236,   120,   194,    50,   337,
-     338,   339,    37,   252,   231,   405,   313,   230,   115,   519,
-     498,    54,   149,   314,    55,   380,   381,   145,   146,    45,
-      35,   206,   239,   361,   161,    46,   327,   166,   167,    96,
-     295,    36,    10,   211,   175,   178,   364,   328,   194,   116,
-     166,   281,   329,   228,   382,   240,   316,   158,   273,   413,
-      10,   346,   158,   169,   170,   406,   187,   187,   216,   340,
-     330,   398,    38,   219,   334,   351,   254,   487,   128,   427,
-     223,   468,    39,   224,   323,   215,   274,   210,   417,   277,
-     436,   421,   494,   408,   152,   178,   520,   242,   243,     1,
-     343,     2,   209,   191,   411,   210,   222,   198,   235,   388,
-      40,   516,   389,   390,    58,    31,    31,    32,    32,   191,
-     187,   228,   377,   374,    10,   211,   443,   307,     3,   145,
-     146,   517,   199,   145,   146,   287,   216,    46,    47,   123,
-     482,    46,   288,   147,     4,     5,   200,   201,   171,   172,
-     289,   124,     6,   215,   270,   287,   161,     7,   431,    52,
-     308,   341,   288,   344,   483,   432,    56,  -129,   350,   202,
-     289,   318,   290,   404,   178,   353,   442,   187,     8,    31,
-     418,    32,   233,   234,   108,   428,   429,   109,   145,   146,
-     391,   203,   290,   211,    48,   423,    49,   110,   433,   111,
-     373,   424,   112,     9,   356,   355,   284,   117,   187,   119,
-      46,   440,    10,   247,   248,   272,   234,   114,   216,   145,
-     146,   118,    46,   304,   305,    11,   347,   348,   372,   305,
-     384,   385,    12,   126,   402,    13,   127,   228,   166,   386,
-     387,   430,   234,   445,   305,   135,   410,   456,   234,   457,
-     234,   471,   305,   120,   133,   400,   472,   348,   132,   187,
-     228,   134,   474,   228,   490,   491,   257,   258,   259,   260,
-     261,   262,   263,   264,   265,   266,   136,   145,   146,   513,
-     491,   523,   234,   530,   234,   137,   216,   532,   234,   138,
-     141,   142,   144,    31,    64,    32,    65,   154,   228,   163,
-     444,   166,   166,   168,   512,   173,   188,   197,   207,   232,
-      66,    67,   225,   318,    10,   208,   255,   454,   244,   245,
-     256,   278,    69,    70,   524,   269,   279,   282,   280,   292,
-      71,    72,   529,   267,   297,   299,   302,   470,    73,    74,
-     310,    75,   309,   311,   454,   312,   226,    76,   349,   319,
-     342,   352,   354,    77,   360,   362,    78,   363,   365,   357,
-     465,   358,   359,   366,   368,   369,   375,   469,   370,   376,
-      79,    80,   378,   157,   393,   394,   454,   399,    81,   396,
-     397,    82,   415,   403,   409,   407,   434,   416,   437,   422,
-     419,   495,   465,   441,   438,   447,    83,   448,   209,   449,
-     450,   455,   305,   501,   458,   467,    84,    85,   466,   476,
-      86,    87,   477,   473,   489,   510,   481,   465,   500,    88,
-     166,    31,    64,    32,    65,    89,   504,   179,   506,   507,
-      90,   227,   508,   509,   515,   527,   526,   528,    66,    67,
-     166,   533,   535,   496,   514,   522,   250,   401,   166,   534,
-      69,    70,   306,   162,   525,   113,   414,   511,    71,    72,
-     218,   502,   475,   335,   503,   531,    73,    74,   336,    75,
-     251,   220,   156,   221,   275,    76,   276,   195,   205,   446,
-     180,    77,   459,     0,    78,   257,   258,   259,   260,   261,
-     262,   263,   264,   265,   266,     0,   145,   146,    79,    80,
-       0,     0,     0,     0,     0,     0,    81,     0,     0,    82,
-       0,    31,    64,    32,    65,     0,     0,   179,     0,     0,
-       0,     0,     0,     0,    83,     0,     0,     0,    66,    67,
-       0,    10,     0,     0,    84,    85,     0,     0,    86,    87,
-      69,    70,     0,     0,     0,     0,     0,    88,    71,    72,
-       0,     0,   267,    89,     0,     0,    73,    74,   181,    75,
-       0,     0,     0,     0,     0,    76,     0,     0,     0,     0,
-     180,    77,     0,     0,    78,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,    79,    80,
-       0,     0,     0,     0,     0,     0,    81,     0,     0,    82,
-       0,    31,    64,    32,    65,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    83,     0,     0,     0,    66,    67,
-      68,     0,     0,     0,    84,    85,     0,     0,    86,    87,
-      69,    70,     0,     0,     0,     0,     0,    88,    71,    72,
-       0,     0,     0,    89,     0,     0,    73,    74,   181,    75,
-       0,     0,     0,     0,     0,    76,     0,     0,     0,     0,
-       0,    77,     0,     0,    78,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,    79,    80,
-       0,     0,     0,     0,     0,     0,    81,     0,     0,    82,
+      33,   193,   190,    45,   274,   192,    97,    42,    46,    27,
+     190,   219,   325,   167,   308,   259,    31,   492,    32,   499,
+     153,   154,   259,    31,   525,    32,    57,    31,   254,    32,
+      59,    60,    61,    62,    63,    51,   131,    31,   259,    32,
+     299,   500,   148,   149,   350,   240,   215,   148,   149,   288,
+     419,   298,   160,   197,    34,   257,   143,   148,   149,   493,
+      50,   259,   190,   292,   190,   117,   526,   148,   149,   504,
+     293,    10,   127,    30,    39,  -129,   148,   149,   294,   411,
+     219,   292,    37,   318,   256,   523,   125,    31,   293,    32,
+     319,   148,   149,   152,    10,    35,   294,   118,   126,   200,
+     295,   505,    45,    40,   210,   524,   164,    46,   286,   169,
+     170,   197,    97,    36,   289,   215,   178,   181,   295,   221,
+     241,    47,   169,   420,   130,   328,   232,   351,   300,   321,
+     358,   412,   367,   370,    10,   235,   494,   122,    38,   190,
+     190,   220,   155,   527,   404,   161,   223,   242,   234,   258,
+     339,   361,   202,   227,   501,   424,   228,   434,   428,   278,
+     443,   214,   281,   219,   213,   214,   414,   161,   181,   418,
+     246,   247,   348,   475,   277,   226,   356,   203,   194,   239,
+     243,    58,   194,   385,   394,   380,    52,   395,   396,   148,
+     149,   204,   205,   450,   190,   232,   383,    56,   215,   430,
+     489,   312,   110,   244,    31,   431,    32,   342,   343,   344,
+     220,    46,   386,   387,   206,    46,   148,   149,     1,   111,
+       2,   148,   149,   172,   173,   490,   283,   449,   274,   112,
+     164,   150,   219,   113,   313,   346,   207,   349,   114,   148,
+     149,   388,   355,   425,   329,   323,   410,     3,   181,   438,
+     359,    54,   190,    31,    55,    32,   439,    48,   116,    49,
+     435,   436,   148,   149,     4,     5,   397,   215,   345,   174,
+     175,   119,     6,   440,   447,   379,   120,     7,   330,   121,
+     362,   237,   238,   190,   128,    46,   251,   252,   331,   276,
+     238,   309,   310,   220,   352,   353,   122,    46,     8,   261,
+     262,   263,   264,   265,   266,   267,   268,   269,   270,   408,
+     148,   149,   232,   169,   378,   310,   390,   391,   392,   393,
+     129,   416,   136,     9,   437,   238,   481,   332,   452,   310,
+     406,   134,    10,   135,   137,   190,   232,   138,   333,   232,
+     463,   238,   141,   334,   139,    11,   464,   238,   478,   310,
+     479,   353,    12,   497,   498,    13,   520,   498,   530,   238,
+     144,   335,   220,   537,   238,   539,   238,   271,   140,   145,
+     157,   147,   166,   171,   232,   176,   451,   169,   169,   191,
+     519,   201,   211,   236,    10,   212,   248,   249,   259,   323,
+     260,   273,   282,   284,   461,   285,   287,   314,   297,   302,
+     531,   304,   315,   307,   316,   317,   347,   324,   536,    31,
+      64,    32,    65,   354,   477,   357,   360,   363,   364,   365,
+     366,   461,   368,   369,   371,   372,    66,    67,   229,   374,
+     375,   376,   213,   381,   382,   384,   402,   472,    69,    70,
+     160,   400,   399,   405,   476,   403,    71,    72,    73,   409,
+     415,   413,   417,   461,   422,    74,    75,   426,    76,   429,
+     444,   423,   445,   230,    77,   454,   441,   456,   502,   472,
+      78,   448,   455,    79,   457,   310,   462,   465,   473,   474,
+     508,   480,   483,   488,   484,   507,   496,    80,    81,   511,
+     513,   515,   517,   514,   472,    82,   516,   169,    83,   261,
+     262,   263,   264,   265,   266,   267,   268,   269,   270,   521,
+     148,   149,   522,    84,   533,   529,   535,   169,   534,   540,
+     532,   542,   541,    85,    86,   169,   518,    87,    88,   503,
+     311,   115,    31,    64,    32,    65,    89,   165,   182,   222,
+     421,   407,    90,   509,   482,   538,   510,    91,   231,    66,
+      67,   255,   340,   224,   341,   225,   159,   198,   279,   280,
+     209,    69,    70,   466,     0,   453,     0,   271,     0,    71,
+      72,    73,     0,     0,     0,     0,     0,     0,    74,    75,
+       0,    76,     0,     0,     0,     0,     0,    77,     0,     0,
+       0,     0,   183,    78,     0,     0,    79,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+      80,    81,     0,     0,     0,     0,     0,     0,    82,     0,
+       0,    83,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,    84,     0,    31,    64,
+      32,    65,     0,    10,   182,     0,    85,    86,     0,     0,
+      87,    88,     0,     0,     0,    66,    67,     0,     0,    89,
+       0,     0,     0,     0,     0,    90,     0,    69,    70,     0,
+     184,     0,     0,     0,     0,    71,    72,    73,     0,     0,
+       0,     0,     0,     0,    74,    75,     0,    76,     0,     0,
+       0,     0,     0,    77,     0,     0,     0,     0,   183,    78,
+       0,     0,    79,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,    80,    81,     0,     0,
+       0,     0,     0,     0,    82,     0,     0,    83,     0,     0,
        0,    31,    64,    32,    65,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    83,     0,     0,     0,    66,    67,
-       0,     0,     0,     0,    84,    85,     0,     0,    86,    87,
-      69,    70,     0,     0,     0,     0,     0,    88,    71,    72,
-       0,     0,     0,    89,     0,     0,    73,    74,    90,    75,
-       0,     0,     0,     0,     0,    76,     0,     0,     0,     0,
-       0,    77,     0,     0,    78,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,    79,    80,
-       0,     0,     0,     0,     0,     0,    81,     0,     0,    82,
-       0,     0,     0,     0,    31,    64,    32,    65,     0,     0,
-       0,     0,     0,     0,    83,     0,     0,     0,     0,     0,
-       0,    66,    67,     0,    84,    85,     0,     0,    86,    87,
-       0,     0,     0,    69,    70,     0,     0,    88,   128,     0,
-       0,    71,    72,    89,     0,     0,     0,     0,    90,    73,
-      74,     0,    75,     0,     0,     0,     0,     0,    76,     0,
-       0,     0,     0,     0,    77,     0,     0,    78,     0,     0,
+       0,     0,    84,     0,     0,     0,     0,     0,    66,    67,
+      68,     0,    85,    86,     0,     0,    87,    88,     0,     0,
+      69,    70,     0,     0,     0,    89,     0,     0,    71,    72,
+      73,    90,     0,     0,     0,     0,   184,    74,    75,     0,
+      76,     0,     0,     0,     0,     0,    77,     0,     0,     0,
+       0,     0,    78,     0,     0,    79,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,    80,
+      81,     0,     0,     0,     0,     0,     0,    82,     0,     0,
+      83,     0,     0,     0,    31,    64,    32,    65,     0,     0,
+       0,     0,     0,     0,     0,    84,     0,     0,     0,     0,
+       0,    66,    67,     0,     0,    85,    86,     0,     0,    87,
+      88,     0,     0,    69,    70,     0,     0,     0,    89,     0,
+       0,    71,    72,    73,    90,     0,     0,     0,     0,    91,
+      74,    75,     0,    76,     0,     0,     0,     0,     0,    77,
+       0,     0,     0,     0,     0,    78,     0,     0,    79,     0,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,    80,    81,     0,     0,     0,     0,     0,     0,
+      82,     0,     0,    83,     0,     0,     0,    31,    64,    32,
+      65,     0,     0,     0,     0,     0,     0,     0,    84,     0,
+       0,     0,     0,     0,    66,    67,     0,     0,    85,    86,
+       0,     0,    87,    88,     0,     0,    69,    70,     0,     0,
+       0,    89,   130,     0,    71,    72,    73,    90,     0,     0,
+       0,     0,    91,    74,    75,     0,    76,     0,     0,     0,
+       0,     0,    77,     0,     0,     0,     0,     0,    78,     0,
+       0,    79,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,     0,    80,    81,     0,     0,     0,
+       0,     0,     0,    82,     0,     0,    83,     0,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,    79,    80,     0,     0,     0,     0,     0,     0,    81,
-       0,     0,    82,     0,    31,    64,    32,    65,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    83,     0,     0,
-       0,    66,    67,     0,    10,     0,     0,    84,    85,     0,
-       0,    86,    87,    69,    70,     0,     0,     0,     0,     0,
-      88,    71,    72,     0,     0,     0,    89,     0,     0,    73,
-      74,    90,    75,     0,     0,     0,     0,     0,    76,     0,
-       0,     0,     0,   180,    77,     0,     0,    78,     0,     0,
+       0,    84,    31,    64,    32,    65,     0,     0,    10,     0,
+       0,    85,    86,     0,     0,    87,    88,     0,     0,    66,
+      67,     0,     0,     0,    89,     0,     0,     0,     0,     0,
+      90,    69,    70,     0,     0,    91,     0,     0,     0,    71,
+      72,    73,     0,     0,     0,     0,     0,     0,    74,    75,
+       0,    76,     0,     0,     0,     0,     0,    77,     0,     0,
+       0,     0,   183,    78,     0,     0,    79,     0,     0,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,    79,    80,     0,     0,     0,     0,     0,     0,    81,
-       0,     0,    82,     0,    31,    64,    32,    65,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    83,     0,     0,
-       0,    66,   155,     0,     0,     0,     0,    84,    85,     0,
-       0,    86,    87,    69,    70,     0,     0,     0,     0,     0,
-      88,    71,    72,     0,     0,     0,    89,     0,     0,    73,
-      74,   181,    75,     0,     0,     0,     0,     0,    76,     0,
-       0,     0,     0,     0,    77,     0,     0,    78,     0,     0,
+      80,    81,     0,     0,     0,     0,     0,     0,    82,     0,
+       0,    83,     0,     0,     0,    31,    64,    32,    65,     0,
+       0,     0,     0,     0,     0,     0,    84,     0,     0,     0,
+       0,     0,    66,   158,     0,     0,    85,    86,     0,     0,
+      87,    88,     0,     0,    69,    70,     0,     0,     0,    89,
+       0,     0,    71,    72,    73,    90,     0,     0,     0,     0,
+     184,    74,    75,     0,    76,     0,     0,     0,     0,     0,
+      77,     0,     0,     0,     0,     0,    78,     0,     0,    79,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,    80,    81,     0,     0,     0,     0,     0,
+       0,    82,     0,     0,    83,     0,     0,     0,    31,    64,
+      32,    65,     0,     0,     0,     0,     0,     0,     0,    84,
+       0,     0,     0,     0,     0,    66,    67,     0,     0,    85,
+      86,     0,     0,    87,    88,     0,     0,    69,    70,     0,
+       0,     0,    89,     0,     0,    71,    72,    73,    90,     0,
+       0,     0,     0,    91,    74,    75,     0,    76,     0,     0,
+       0,     0,    64,    77,    65,     0,     0,     0,     0,    78,
+       0,     0,    79,     0,     0,     0,     0,     0,    66,   158,
+       0,     0,     0,     0,     0,     0,    80,    81,     0,     0,
+      69,    70,     0,     0,    82,     0,     0,    83,     0,     0,
+      73,     0,     0,     0,     0,     0,     0,    74,    75,     0,
+      76,     0,    84,     0,     0,     0,    77,     0,     0,     0,
+       0,     0,    85,    86,     0,    79,    87,    88,     0,     0,
+       0,     0,     0,     0,     0,    89,     0,     0,     0,    80,
+      81,    90,    69,    70,     0,     0,    91,    82,     0,     0,
+      83,     0,    73,     0,     0,     0,     0,     0,     0,    74,
+      75,     0,    76,     0,     0,    84,     0,     0,    77,     0,
+       0,     0,     0,     0,     0,    85,     0,    79,     0,    87,
+      88,     0,     0,     0,     0,     0,     0,     0,    89,     0,
+       0,    80,   253,     0,    90,     0,     0,     0,     0,    82,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,    79,    80,     0,     0,     0,     0,     0,     0,    81,
-       0,     0,    82,     0,    31,    64,    32,    65,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    83,     0,     0,
-       0,    66,    67,     0,     0,     0,     0,    84,    85,     0,
-       0,    86,    87,    69,    70,     0,     0,     0,     0,     0,
-      88,    71,    72,     0,     0,     0,    89,     0,     0,    73,
-      74,    90,    75,     0,     0,     0,     0,     0,    76,     0,
-       0,     0,     0,     0,    77,     0,     0,    78,     0,     0,
-       0,    69,    70,     0,     0,     0,     0,     0,     0,     0,
-      72,    79,    80,     0,     0,     0,     0,    73,    74,    81,
-      75,     0,    82,     0,     0,    64,    76,    65,     0,     0,
-       0,     0,     0,     0,     0,    78,     0,    83,     0,     0,
-       0,    66,   155,     0,     0,     0,     0,    84,    85,    79,
-     249,    86,    87,    69,    70,     0,     0,    81,     0,     0,
-      88,     0,    72,     0,     0,     0,    89,     0,     0,    73,
-      74,    90,    75,     0,     0,    83,     0,     0,    76,     0,
-       0,     0,     0,     0,     0,    84,     0,    78,     0,    86,
-      87,     0,     0,     0,     0,     0,     0,     0,    88,     0,
-       0,    79,    80,     0,    89,     0,     0,     0,     0,    81,
-       0,     0,    82,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    83,     0,     0,
        0,     0,     0,     0,     0,     0,     0,    84,     0,     0,
-       0,    86,    87,     0,     0,     0,     0,     0,     0,     0,
-      88,     0,     0,     0,     0,     0,    89
+       0,     0,     0,     0,     0,     0,     0,    85,     0,     0,
+       0,    87,    88,     0,     0,     0,     0,     0,     0,     0,
+      89,     0,     0,     0,     0,     0,    90
 };
 
 static const yytype_int16 yycheck[] =
 {
-       4,   120,    12,   188,   124,   142,    41,    11,    12,   128,
-       8,     0,   128,   110,   247,     8,     8,     8,    27,   226,
-       4,   210,     6,    37,    10,     4,    30,     6,     7,     7,
-      34,    35,    36,    37,    38,   215,    71,    26,     4,     4,
-       6,     6,    43,    23,    24,   142,    21,    22,     7,    10,
-      29,    21,    22,    21,    22,    90,    69,    43,    74,    78,
-     179,   119,   181,    21,    22,   181,    65,    53,   143,    70,
-      48,    21,    22,   210,    63,   169,   132,    60,    80,    17,
-      18,    19,   119,   180,   159,     9,    55,   143,    28,    48,
-     106,   137,    96,    62,   140,    94,    95,    21,    22,   109,
-     119,   136,   103,   292,   108,   109,    92,   111,   112,   144,
-     124,    71,   114,   210,   118,   119,   296,   103,    60,    59,
-     124,    71,   108,   158,   123,   126,   245,   141,   120,   142,
-     114,    92,   141,    44,    45,   342,   255,   256,   142,    77,
-     126,   326,    83,   147,   142,   120,   181,   125,   131,   142,
-     154,   142,    29,   157,   248,   292,   191,   141,   365,   194,
-     393,   368,   141,   348,   144,   169,   125,   171,   172,     1,
-     267,     3,   142,   131,   142,   141,   141,    51,   167,    35,
-      57,    66,    38,    39,    71,     4,     4,     6,     6,   131,
-     309,   226,   312,   309,   114,   292,   403,   232,    30,    21,
-      22,    86,    76,    21,    22,    72,   210,   211,     0,   129,
-      32,   215,    79,    31,    46,    47,    90,    91,    44,    45,
-      87,   141,    54,   360,   409,    72,   230,    59,    55,   134,
-     234,   266,    79,   268,    56,    62,     3,    84,   273,   113,
-      87,   245,   109,   340,   248,   280,     9,   366,    80,     4,
-     366,     6,   142,   143,   115,   375,   376,   143,    21,    22,
-     116,   135,   109,   360,   137,   106,   139,    31,   387,   141,
-     305,   112,    83,   105,   284,   142,    31,     5,   397,   141,
-     284,   397,   114,   142,   143,   142,   143,   104,   292,    21,
-      22,    95,   296,   142,   143,   127,   142,   143,   142,   143,
-     142,   143,   134,     7,   339,   137,     7,   342,   312,   142,
-     143,   142,   143,   142,   143,     5,   351,   142,   143,   142,
-     143,   142,   143,   132,    82,   329,   142,   143,   141,   448,
-     365,   141,   448,   368,   142,   143,    10,    11,    12,    13,
-      14,    15,    16,    17,    18,    19,   141,    21,    22,   142,
-     143,   142,   143,   142,   143,   141,   360,   142,   143,    82,
-       5,    71,   143,     4,     5,     6,     7,    98,   403,   141,
-     405,   375,   376,     7,   494,   134,   141,     7,     7,    11,
-      21,    22,    23,   387,   114,   142,     8,   422,   141,   141,
-       9,    61,    33,    34,   514,    20,   142,   142,    71,   143,
-      41,    42,   522,    77,    73,    99,   142,   442,    49,    50,
-      85,    52,   141,    85,   449,   141,    57,    58,   129,   128,
-     141,    61,     7,    64,    84,     7,    67,     7,    40,    97,
-     434,    97,    97,    75,    40,   142,   141,   441,    96,   141,
-      81,    82,     5,    27,   143,   134,   481,    85,    89,    92,
-     141,    92,   101,   141,   141,    92,   141,   101,    36,    40,
-      96,   465,   466,   141,    99,     7,   107,    95,   142,    40,
-      88,    19,   143,   477,     5,    40,   117,   118,   141,     7,
-     121,   122,   133,   142,   108,   489,   143,   491,   142,   130,
-     494,     4,     5,     6,     7,   136,    93,    10,   102,   110,
-     141,   142,   102,     9,    31,   110,    68,    68,    21,    22,
-     514,   100,     7,   466,   141,   141,   178,   332,   522,   142,
-      33,    34,   230,   109,   141,    51,   360,   491,    41,    42,
-     144,   479,   449,   255,   481,   525,    49,    50,   256,    52,
-     179,   148,   106,   153,   192,    58,   192,   130,   135,   409,
-      63,    64,   432,    -1,    67,    10,    11,    12,    13,    14,
-      15,    16,    17,    18,    19,    -1,    21,    22,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
-      -1,     4,     5,     6,     7,    -1,    -1,    10,    -1,    -1,
-      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    21,    22,
-      -1,   114,    -1,    -1,   117,   118,    -1,    -1,   121,   122,
-      33,    34,    -1,    -1,    -1,    -1,    -1,   130,    41,    42,
-      -1,    -1,    77,   136,    -1,    -1,    49,    50,   141,    52,
-      -1,    -1,    -1,    -1,    -1,    58,    -1,    -1,    -1,    -1,
-      63,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
-      -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    21,    22,
-      23,    -1,    -1,    -1,   117,   118,    -1,    -1,   121,   122,
-      33,    34,    -1,    -1,    -1,    -1,    -1,   130,    41,    42,
-      -1,    -1,    -1,   136,    -1,    -1,    49,    50,   141,    52,
-      -1,    -1,    -1,    -1,    -1,    58,    -1,    -1,    -1,    -1,
-      -1,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
+       4,   130,   122,    12,   191,   126,    41,    11,    12,     0,
+     130,   145,   251,   112,   230,     8,     4,     7,     6,     7,
+      23,    24,     8,     4,     7,     6,    30,     4,   181,     6,
+      34,    35,    36,    37,    38,    26,    71,     4,     8,     6,
+      37,    29,    21,    22,    10,   172,   145,    21,    22,   214,
+      70,   219,    27,    61,    79,   184,    91,    21,    22,    49,
+      81,     8,   182,    73,   184,    28,    49,    21,    22,    75,
+      80,   115,    63,   120,    29,    85,    21,    22,    88,     9,
+     214,    73,   120,    56,   183,    67,   130,     4,    80,     6,
+      63,    21,    22,    97,   115,   120,    88,    60,   142,   134,
+     110,   107,   111,    58,   139,    87,   110,   111,    72,   113,
+     114,    61,   147,    72,    31,   214,   120,   121,   110,   146,
+      44,     0,   126,   143,   132,   252,   161,    93,   125,   249,
+     283,   347,   297,   301,   115,   162,   126,   133,    84,   259,
+     260,   145,   145,   126,   331,   142,   150,    71,   144,   184,
+     143,   143,    52,   157,   142,   371,   160,   143,   374,   194,
+     399,   142,   197,   297,   143,   142,   353,   142,   172,   143,
+     174,   175,   271,   143,   121,   142,   121,    77,   132,   170,
+     104,    72,   132,    66,    35,   314,   135,    38,    39,    21,
+      22,    91,    92,   409,   314,   230,   317,     3,   297,   107,
+      32,   236,   116,   127,     4,   113,     6,    17,    18,    19,
+     214,   215,    95,    96,   114,   219,    21,    22,     1,   144,
+       3,    21,    22,    45,    46,    57,    31,     9,   415,    31,
+     234,    31,   366,   142,   238,   270,   136,   272,    84,    21,
+      22,   124,   277,   372,    10,   249,   345,    30,   252,    56,
+     285,   138,   372,     4,   141,     6,    63,   138,   105,   140,
+     381,   382,    21,    22,    47,    48,   117,   366,    78,    45,
+      46,     5,    55,   393,   403,   310,    96,    60,    44,   142,
+     289,   143,   144,   403,     7,   289,   143,   144,    54,   143,
+     144,   143,   144,   297,   143,   144,   133,   301,    81,    10,
+      11,    12,    13,    14,    15,    16,    17,    18,    19,   344,
+      21,    22,   347,   317,   143,   144,   143,   144,   143,   144,
+       7,   356,    83,   106,   143,   144,   455,    93,   143,   144,
+     334,   142,   115,   142,   142,   455,   371,     5,   104,   374,
+     143,   144,    83,   109,   142,   128,   143,   144,   143,   144,
+     143,   144,   135,   143,   144,   138,   143,   144,   143,   144,
+       5,   127,   366,   143,   144,   143,   144,    78,   142,    72,
+      99,   144,   142,     7,   409,   135,   411,   381,   382,   142,
+     501,     7,     7,    11,   115,   143,   142,   142,     8,   393,
+       9,    20,    62,   143,   429,    72,   143,   142,   144,    74,
+     521,   100,    86,   143,    86,   142,   142,   129,   529,     4,
+       5,     6,     7,   130,   449,    62,     7,    98,    98,    98,
+      85,   456,     7,     7,    40,    76,    21,    22,    23,    40,
+     143,    97,   143,   142,   142,     5,    93,   441,    33,    34,
+      27,   135,   144,    86,   448,   142,    41,    42,    43,   142,
+     142,    93,   143,   488,   102,    50,    51,    97,    53,    40,
+      36,   102,   100,    58,    59,     7,   142,    40,   472,   473,
+      65,   142,    96,    68,    89,   144,    19,     5,   142,    40,
+     484,   143,     7,   144,   134,   143,   109,    82,    83,    94,
+     103,   103,   496,   111,   498,    90,     9,   501,    93,    10,
+      11,    12,    13,    14,    15,    16,    17,    18,    19,   142,
+      21,    22,    31,   108,    69,   142,    69,   521,   111,   101,
+     142,     7,   143,   118,   119,   529,   498,   122,   123,   473,
+     234,    51,     4,     5,     6,     7,   131,   111,    10,   147,
+     366,   337,   137,   486,   456,   532,   488,   142,   143,    21,
+      22,   182,   259,   151,   260,   156,   108,   132,   195,   195,
+     138,    33,    34,   439,    -1,   415,    -1,    78,    -1,    41,
+      42,    43,    -1,    -1,    -1,    -1,    -1,    -1,    50,    51,
+      -1,    53,    -1,    -1,    -1,    -1,    -1,    59,    -1,    -1,
+      -1,    -1,    64,    65,    -1,    -1,    68,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      82,    83,    -1,    -1,    -1,    -1,    -1,    -1,    90,    -1,
+      -1,    93,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,   108,    -1,     4,     5,
+       6,     7,    -1,   115,    10,    -1,   118,   119,    -1,    -1,
+     122,   123,    -1,    -1,    -1,    21,    22,    -1,    -1,   131,
+      -1,    -1,    -1,    -1,    -1,   137,    -1,    33,    34,    -1,
+     142,    -1,    -1,    -1,    -1,    41,    42,    43,    -1,    -1,
+      -1,    -1,    -1,    -1,    50,    51,    -1,    53,    -1,    -1,
+      -1,    -1,    -1,    59,    -1,    -1,    -1,    -1,    64,    65,
+      -1,    -1,    68,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    82,    83,    -1,    -1,
+      -1,    -1,    -1,    -1,    90,    -1,    -1,    93,    -1,    -1,
       -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    21,    22,
-      -1,    -1,    -1,    -1,   117,   118,    -1,    -1,   121,   122,
-      33,    34,    -1,    -1,    -1,    -1,    -1,   130,    41,    42,
-      -1,    -1,    -1,   136,    -1,    -1,    49,    50,   141,    52,
-      -1,    -1,    -1,    -1,    -1,    58,    -1,    -1,    -1,    -1,
-      -1,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
-      -1,    -1,    -1,    -1,     4,     5,     6,     7,    -1,    -1,
-      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    -1,
-      -1,    21,    22,    -1,   117,   118,    -1,    -1,   121,   122,
-      -1,    -1,    -1,    33,    34,    -1,    -1,   130,   131,    -1,
-      -1,    41,    42,   136,    -1,    -1,    -1,    -1,   141,    49,
-      50,    -1,    52,    -1,    -1,    -1,    -1,    -1,    58,    -1,
-      -1,    -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,
+      -1,    -1,   108,    -1,    -1,    -1,    -1,    -1,    21,    22,
+      23,    -1,   118,   119,    -1,    -1,   122,   123,    -1,    -1,
+      33,    34,    -1,    -1,    -1,   131,    -1,    -1,    41,    42,
+      43,   137,    -1,    -1,    -1,    -1,   142,    50,    51,    -1,
+      53,    -1,    -1,    -1,    -1,    -1,    59,    -1,    -1,    -1,
+      -1,    -1,    65,    -1,    -1,    68,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    82,
+      83,    -1,    -1,    -1,    -1,    -1,    -1,    90,    -1,    -1,
+      93,    -1,    -1,    -1,     4,     5,     6,     7,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,   108,    -1,    -1,    -1,    -1,
+      -1,    21,    22,    -1,    -1,   118,   119,    -1,    -1,   122,
+     123,    -1,    -1,    33,    34,    -1,    -1,    -1,   131,    -1,
+      -1,    41,    42,    43,   137,    -1,    -1,    -1,    -1,   142,
+      50,    51,    -1,    53,    -1,    -1,    -1,    -1,    -1,    59,
+      -1,    -1,    -1,    -1,    -1,    65,    -1,    -1,    68,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,    89,
-      -1,    -1,    92,    -1,     4,     5,     6,     7,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,
-      -1,    21,    22,    -1,   114,    -1,    -1,   117,   118,    -1,
-      -1,   121,   122,    33,    34,    -1,    -1,    -1,    -1,    -1,
-     130,    41,    42,    -1,    -1,    -1,   136,    -1,    -1,    49,
-      50,   141,    52,    -1,    -1,    -1,    -1,    -1,    58,    -1,
-      -1,    -1,    -1,    63,    64,    -1,    -1,    67,    -1,    -1,
+      -1,    -1,    82,    83,    -1,    -1,    -1,    -1,    -1,    -1,
+      90,    -1,    -1,    93,    -1,    -1,    -1,     4,     5,     6,
+       7,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   108,    -1,
+      -1,    -1,    -1,    -1,    21,    22,    -1,    -1,   118,   119,
+      -1,    -1,   122,   123,    -1,    -1,    33,    34,    -1,    -1,
+      -1,   131,   132,    -1,    41,    42,    43,   137,    -1,    -1,
+      -1,    -1,   142,    50,    51,    -1,    53,    -1,    -1,    -1,
+      -1,    -1,    59,    -1,    -1,    -1,    -1,    -1,    65,    -1,
+      -1,    68,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    82,    83,    -1,    -1,    -1,
+      -1,    -1,    -1,    90,    -1,    -1,    93,    -1,    -1,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,    89,
-      -1,    -1,    92,    -1,     4,     5,     6,     7,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,
-      -1,    21,    22,    -1,    -1,    -1,    -1,   117,   118,    -1,
-      -1,   121,   122,    33,    34,    -1,    -1,    -1,    -1,    -1,
-     130,    41,    42,    -1,    -1,    -1,   136,    -1,    -1,    49,
-      50,   141,    52,    -1,    -1,    -1,    -1,    -1,    58,    -1,
-      -1,    -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,
+      -1,   108,     4,     5,     6,     7,    -1,    -1,   115,    -1,
+      -1,   118,   119,    -1,    -1,   122,   123,    -1,    -1,    21,
+      22,    -1,    -1,    -1,   131,    -1,    -1,    -1,    -1,    -1,
+     137,    33,    34,    -1,    -1,   142,    -1,    -1,    -1,    41,
+      42,    43,    -1,    -1,    -1,    -1,    -1,    -1,    50,    51,
+      -1,    53,    -1,    -1,    -1,    -1,    -1,    59,    -1,    -1,
+      -1,    -1,    64,    65,    -1,    -1,    68,    -1,    -1,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,    89,
-      -1,    -1,    92,    -1,     4,     5,     6,     7,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,
-      -1,    21,    22,    -1,    -1,    -1,    -1,   117,   118,    -1,
-      -1,   121,   122,    33,    34,    -1,    -1,    -1,    -1,    -1,
-     130,    41,    42,    -1,    -1,    -1,   136,    -1,    -1,    49,
-      50,   141,    52,    -1,    -1,    -1,    -1,    -1,    58,    -1,
-      -1,    -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,
-      -1,    33,    34,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      42,    81,    82,    -1,    -1,    -1,    -1,    49,    50,    89,
-      52,    -1,    92,    -1,    -1,     5,    58,     7,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    67,    -1,   107,    -1,    -1,
-      -1,    21,    22,    -1,    -1,    -1,    -1,   117,   118,    81,
-      82,   121,   122,    33,    34,    -1,    -1,    89,    -1,    -1,
-     130,    -1,    42,    -1,    -1,    -1,   136,    -1,    -1,    49,
-      50,   141,    52,    -1,    -1,   107,    -1,    -1,    58,    -1,
-      -1,    -1,    -1,    -1,    -1,   117,    -1,    67,    -1,   121,
-     122,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   130,    -1,
-      -1,    81,    82,    -1,   136,    -1,    -1,    -1,    -1,    89,
-      -1,    -1,    92,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,   117,    -1,    -1,
-      -1,   121,   122,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-     130,    -1,    -1,    -1,    -1,    -1,   136
+      82,    83,    -1,    -1,    -1,    -1,    -1,    -1,    90,    -1,
+      -1,    93,    -1,    -1,    -1,     4,     5,     6,     7,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,   108,    -1,    -1,    -1,
+      -1,    -1,    21,    22,    -1,    -1,   118,   119,    -1,    -1,
+     122,   123,    -1,    -1,    33,    34,    -1,    -1,    -1,   131,
+      -1,    -1,    41,    42,    43,   137,    -1,    -1,    -1,    -1,
+     142,    50,    51,    -1,    53,    -1,    -1,    -1,    -1,    -1,
+      59,    -1,    -1,    -1,    -1,    -1,    65,    -1,    -1,    68,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    82,    83,    -1,    -1,    -1,    -1,    -1,
+      -1,    90,    -1,    -1,    93,    -1,    -1,    -1,     4,     5,
+       6,     7,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   108,
+      -1,    -1,    -1,    -1,    -1,    21,    22,    -1,    -1,   118,
+     119,    -1,    -1,   122,   123,    -1,    -1,    33,    34,    -1,
+      -1,    -1,   131,    -1,    -1,    41,    42,    43,   137,    -1,
+      -1,    -1,    -1,   142,    50,    51,    -1,    53,    -1,    -1,
+      -1,    -1,     5,    59,     7,    -1,    -1,    -1,    -1,    65,
+      -1,    -1,    68,    -1,    -1,    -1,    -1,    -1,    21,    22,
+      -1,    -1,    -1,    -1,    -1,    -1,    82,    83,    -1,    -1,
+      33,    34,    -1,    -1,    90,    -1,    -1,    93,    -1,    -1,
+      43,    -1,    -1,    -1,    -1,    -1,    -1,    50,    51,    -1,
+      53,    -1,   108,    -1,    -1,    -1,    59,    -1,    -1,    -1,
+      -1,    -1,   118,   119,    -1,    68,   122,   123,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,   131,    -1,    -1,    -1,    82,
+      83,   137,    33,    34,    -1,    -1,   142,    90,    -1,    -1,
+      93,    -1,    43,    -1,    -1,    -1,    -1,    -1,    -1,    50,
+      51,    -1,    53,    -1,    -1,   108,    -1,    -1,    59,    -1,
+      -1,    -1,    -1,    -1,    -1,   118,    -1,    68,    -1,   122,
+     123,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   131,    -1,
+      -1,    82,    83,    -1,   137,    -1,    -1,    -1,    -1,    90,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,   108,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,   118,    -1,    -1,
+      -1,   122,   123,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+     131,    -1,    -1,    -1,    -1,    -1,   137
 };
 
   /* YYSTOS[STATE-NUM] -- The (internal number of the) accessing
      symbol of state STATE-NUM.  */
 static const yytype_uint8 yystos[] =
 {
-       0,     1,     3,    30,    46,    47,    54,    59,    80,   105,
-     114,   127,   134,   137,   146,   147,   148,   149,   150,   151,
-     152,   173,   174,   177,   178,   181,   183,   186,   250,   251,
-     119,     4,     6,   248,    78,   119,    71,   119,    83,    29,
-      57,   187,   248,   184,   185,   198,   248,     0,   137,   139,
-      80,   186,   134,   182,   137,   140,     3,   248,    71,   248,
-     248,   248,   248,   248,     5,     7,    21,    22,    23,    33,
-      34,    41,    42,    49,    50,    52,    58,    64,    67,    81,
-      82,    89,    92,   107,   117,   118,   121,   122,   130,   136,
-     141,   155,   188,   189,   190,   192,   224,   225,   226,   227,
-     228,   229,   230,   231,   238,   241,   244,   248,   115,   143,
-      31,   141,    83,   182,   104,    28,    59,     5,    95,   141,
-     132,   218,   219,   129,   141,   186,     7,     7,   131,   224,
-     234,   235,   141,    82,   141,     5,   141,   141,    82,   186,
-     224,     5,    71,   191,   143,    21,    22,    31,   245,   248,
-      23,    24,   144,   246,    98,    22,   227,    27,   141,   179,
-     180,   248,   185,   141,   192,   247,   248,   248,     7,    44,
-      45,    44,    45,   134,   175,   248,   153,   154,   248,    10,
-      63,   141,   220,   221,   222,   223,   224,   241,   141,   247,
-     220,   131,   232,   233,    60,   235,   236,     7,    51,    76,
-      90,    91,   113,   135,   239,   239,   224,     7,   142,   142,
-     141,   192,   195,   196,   199,   228,   248,   218,   190,   248,
-     225,   226,   141,   248,   248,    23,    57,   142,   224,   237,
-     143,   218,    11,   142,   143,   186,   153,    43,    70,   103,
-     126,   159,   248,   248,   141,   141,   162,   142,   143,    82,
-     155,   223,   192,   220,   224,     8,     9,    10,    11,    12,
-      13,    14,    15,    16,    17,    18,    19,    77,   243,    20,
-     238,   240,   142,   120,   224,   233,   236,   224,    61,   142,
-      71,    71,   142,   195,    31,   197,   198,    72,    79,    87,
-     109,   194,   143,   197,    37,   124,   193,    73,   200,    99,
-     207,   208,   142,   237,   142,   143,   180,   224,   248,   141,
-      85,    85,   141,    55,    62,   176,   241,   242,   248,   128,
-     159,   160,   161,   153,    10,    43,    53,    92,   103,   108,
-     126,   156,   157,   158,   142,   221,   222,    17,    18,    19,
-      77,   224,   141,   192,   224,    10,    92,   142,   143,   129,
-     224,   120,    61,   224,     7,   142,   198,    97,    97,    97,
-      84,   195,     7,     7,   197,    40,    75,   201,    40,   142,
-      96,   209,   142,   224,   220,   141,   141,   247,     5,    65,
-      94,    95,   123,   249,   142,   143,   142,   143,    35,    38,
-      39,   116,   171,   143,   134,   163,    92,   141,   238,    85,
-     248,   156,   224,   141,   192,     9,   237,    92,   238,   141,
-     224,   142,    69,   142,   196,   101,   101,   237,   220,    96,
-     202,   237,    40,   106,   112,   210,   211,   142,   247,   247,
-     142,    55,    62,   241,   141,   172,   159,    36,    99,   164,
-     220,   141,     9,   237,   224,   142,   240,     7,    95,    40,
-      88,   203,   214,   215,   224,    19,   142,   142,     5,   249,
-     166,   167,   168,   169,   170,   248,   141,    40,   142,   248,
-     224,   142,   142,   142,   220,   214,     7,   133,   204,   205,
-     206,   143,    32,    56,   216,     7,    48,   125,   212,   108,
-     142,   143,     7,    29,   141,   248,   166,    74,   106,   165,
-     142,   248,   206,   215,    93,   217,   102,   110,   102,     9,
-     248,   167,   247,   142,   141,    31,    66,    86,     7,    48,
-     125,   213,   141,   142,   247,   141,    68,   110,    68,   247,
-     142,   207,   142,   100,   142,     7
+       0,     1,     3,    30,    47,    48,    55,    60,    81,   106,
+     115,   128,   135,   138,   147,   148,   149,   150,   151,   152,
+     153,   174,   175,   178,   179,   182,   184,   187,   252,   253,
+     120,     4,     6,   250,    79,   120,    72,   120,    84,    29,
+      58,   188,   250,   185,   186,   199,   250,     0,   138,   140,
+      81,   187,   135,   183,   138,   141,     3,   250,    72,   250,
+     250,   250,   250,   250,     5,     7,    21,    22,    23,    33,
+      34,    41,    42,    43,    50,    51,    53,    59,    65,    68,
+      82,    83,    90,    93,   108,   118,   119,   122,   123,   131,
+     137,   142,   156,   189,   190,   191,   193,   225,   226,   227,
+     228,   229,   230,   231,   232,   233,   240,   243,   246,   250,
+     116,   144,    31,   142,    84,   183,   105,    28,    60,     5,
+      96,   142,   133,   219,   220,   130,   142,   187,     7,     7,
+     132,   225,   236,   237,   142,   142,    83,   142,     5,   142,
+     142,    83,   187,   225,     5,    72,   192,   144,    21,    22,
+      31,   247,   250,    23,    24,   145,   248,    99,    22,   228,
+      27,   142,   180,   181,   250,   186,   142,   193,   249,   250,
+     250,     7,    45,    46,    45,    46,   135,   176,   250,   154,
+     155,   250,    10,    64,   142,   221,   222,   223,   224,   225,
+     243,   142,   249,   221,   132,   234,   235,    61,   237,   238,
+     225,     7,    52,    77,    91,    92,   114,   136,   241,   241,
+     225,     7,   143,   143,   142,   193,   196,   197,   200,   229,
+     250,   219,   191,   250,   226,   227,   142,   250,   250,    23,
+      58,   143,   225,   239,   144,   219,    11,   143,   144,   187,
+     154,    44,    71,   104,   127,   160,   250,   250,   142,   142,
+     163,   143,   144,    83,   156,   224,   193,   221,   225,     8,
+       9,    10,    11,    12,    13,    14,    15,    16,    17,    18,
+      19,    78,   245,    20,   240,   242,   143,   121,   225,   235,
+     238,   225,    62,    31,   143,    72,    72,   143,   196,    31,
+     198,   199,    73,    80,    88,   110,   195,   144,   198,    37,
+     125,   194,    74,   201,   100,   208,   209,   143,   239,   143,
+     144,   181,   225,   250,   142,    86,    86,   142,    56,    63,
+     177,   243,   244,   250,   129,   160,   161,   162,   154,    10,
+      44,    54,    93,   104,   109,   127,   157,   158,   159,   143,
+     222,   223,    17,    18,    19,    78,   225,   142,   193,   225,
+      10,    93,   143,   144,   130,   225,   121,    62,   156,   225,
+       7,   143,   199,    98,    98,    98,    85,   196,     7,     7,
+     198,    40,    76,   202,    40,   143,    97,   210,   143,   225,
+     221,   142,   142,   249,     5,    66,    95,    96,   124,   251,
+     143,   144,   143,   144,    35,    38,    39,   117,   172,   144,
+     135,   164,    93,   142,   240,    86,   250,   157,   225,   142,
+     193,     9,   239,    93,   240,   142,   225,   143,   143,    70,
+     143,   197,   102,   102,   239,   221,    97,   203,   239,    40,
+     107,   113,   211,   212,   143,   249,   249,   143,    56,    63,
+     243,   142,   173,   160,    36,   100,   165,   221,   142,     9,
+     239,   225,   143,   242,     7,    96,    40,    89,   204,   215,
+     216,   225,    19,   143,   143,     5,   251,   167,   168,   169,
+     170,   171,   250,   142,    40,   143,   250,   225,   143,   143,
+     143,   221,   215,     7,   134,   205,   206,   207,   144,    32,
+      57,   217,     7,    49,   126,   213,   109,   143,   144,     7,
+      29,   142,   250,   167,    75,   107,   166,   143,   250,   207,
+     216,    94,   218,   103,   111,   103,     9,   250,   168,   249,
+     143,   142,    31,    67,    87,     7,    49,   126,   214,   142,
+     143,   249,   142,    69,   111,    69,   249,   143,   208,   143,
+     101,   143,     7
 };
 
   /* YYR1[YYN] -- Symbol number of symbol that rule YYN derives.  */
 static const yytype_uint8 yyr1[] =
 {
-       0,   145,   146,   146,   146,   146,   146,   146,   147,   147,
-     147,   147,   147,   147,   147,   147,   147,   147,   148,   149,
-     149,   149,   149,   150,   151,   152,   153,   154,   154,   155,
-     155,   155,   155,   155,   155,   155,   155,   155,   155,   155,
-     155,   155,   155,   155,   155,   155,   155,   156,   156,   156,
-     156,   156,   156,   156,   157,   157,   158,   158,   159,   159,
-     159,   159,   160,   160,   161,   161,   162,   162,   163,   163,
-     164,   164,   165,   165,   166,   166,   167,   167,   167,   168,
-     168,   169,   170,   171,   171,   171,   171,   172,   172,   173,
-     173,   173,   173,   174,   175,   175,   176,   176,   176,   176,
-     177,   178,   179,   179,   180,   181,   181,   182,   182,   183,
-     184,   184,   185,   186,   187,   187,   187,   188,   188,   189,
-     189,   190,   190,   190,   191,   192,   193,   193,   193,   194,
-     194,   194,   194,   194,   194,   194,   194,   195,   195,   196,
-     196,   196,   196,   196,   196,   197,   197,   198,   198,   199,
-     199,   200,   200,   201,   201,   202,   202,   203,   203,   204,
-     204,   205,   205,   206,   207,   208,   208,   209,   209,   210,
-     210,   211,   211,   212,   212,   212,   213,   213,   213,   214,
-     214,   215,   216,   216,   216,   217,   217,   217,   218,   218,
-     219,   220,   220,   221,   221,   222,   222,   223,   223,   223,
-     223,   223,   223,   223,   223,   223,   223,   223,   224,   224,
-     225,   225,   226,   226,   227,   227,   227,   227,   227,   227,
-     227,   227,   227,   227,   228,   228,   228,   228,   229,   230,
-     230,   231,   231,   232,   232,   233,   234,   234,   235,   236,
-     236,   237,   237,   238,   238,   238,   238,   238,   238,   238,
-     238,   239,   239,   239,   239,   239,   239,   240,   240,   241,
-     241,   242,   242,   243,   243,   243,   243,   243,   243,   243,
-     243,   243,   243,   244,   245,   245,   246,   246,   246,   247,
-     247,   248,   248,   249,   249,   249,   249,   250,   251,   251
+       0,   146,   147,   147,   147,   147,   147,   147,   148,   148,
+     148,   148,   148,   148,   148,   148,   148,   148,   149,   150,
+     150,   150,   150,   151,   152,   153,   154,   155,   155,   156,
+     156,   156,   156,   156,   156,   156,   156,   156,   156,   156,
+     156,   156,   156,   156,   156,   156,   156,   157,   157,   157,
+     157,   157,   157,   157,   158,   158,  

<TRUNCATED>