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

[11/40] incubator-quickstep git commit: Type as first class citizen

Type as first class citizen


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

Branch: refs/heads/new-op
Commit: 65978e259341b1d520b04fd698a40d0ed41a0de7
Parents: 6c53bdd
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Wed Oct 4 00:26:36 2017 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Oct 5 17:03:11 2017 -0500

----------------------------------------------------------------------
 parser/CMakeLists.txt                 |   10 +
 parser/ParseAttributeDefinition.cpp   |    8 +-
 parser/ParseAttributeDefinition.hpp   |   48 +-
 parser/ParseBasicExpressions.cpp      |   17 +
 parser/ParseBasicExpressions.hpp      |   48 +
 parser/ParseDataType-decl.hpp         |   24 +
 parser/ParseDataType.cpp              |   71 +
 parser/ParseDataType.hpp              |  183 ++
 parser/ParseExpression.hpp            |    1 +
 parser/SqlLexer.lpp                   |   18 +-
 parser/SqlParser.ypp                  |  204 +-
 parser/preprocessed/SqlLexer_gen.cpp  | 1612 ++++++------
 parser/preprocessed/SqlLexer_gen.hpp  |    2 +-
 parser/preprocessed/SqlParser_gen.cpp | 3952 +++++++++++++---------------
 parser/preprocessed/SqlParser_gen.hpp |  212 +-
 query_optimizer/resolver/Resolver.cpp |   15 +-
 types/CMakeLists.txt                  |   39 +-
 types/MetaType-decl.hpp               |   76 +
 types/MetaType.cpp                    |   19 +-
 types/MetaType.hpp                    |    2 +-
 types/MetaTypeLite.cpp                |   47 -
 types/MetaTypeLite.hpp                |   76 -
 types/TypeFactory-decl.hpp            |  141 +
 types/TypeFactory.cpp                 |  171 ++
 types/TypeFactory.hpp                 |    3 +-
 types/TypeFactoryLite.cpp             |  170 --
 types/TypeFactoryLite.hpp             |  141 -
 types/TypeIDSelectors.hpp             |   31 -
 types/TypeUtil.hpp                    |    2 +-
 29 files changed, 3492 insertions(+), 3851 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/65978e25/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index fb7a3cd..8d6ce91 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -89,6 +89,7 @@ add_library(quickstep_parser_ParseAttributeDefinition ParseAttributeDefinition.c
 add_library(quickstep_parser_ParseBasicExpressions ParseBasicExpressions.cpp ParseBasicExpressions.hpp)
 add_library(quickstep_parser_ParseBlockProperties ParseBlockProperties.cpp ParseBlockProperties.hpp)
 add_library(quickstep_parser_ParseCaseExpressions ParseCaseExpressions.cpp ParseCaseExpressions.hpp)
+add_library(quickstep_parser_ParseDataType ParseDataType.cpp ParseDataType.hpp)
 add_library(quickstep_parser_ParseExpression ../empty_src.cpp ParseExpression.hpp)
 add_library(quickstep_parser_ParseGeneratorTableReference ParseGeneratorTableReference.cpp ParseGeneratorTableReference.hpp)
 add_library(quickstep_parser_ParseGroupBy ParseGroupBy.cpp ParseGroupBy.hpp)
@@ -128,12 +129,14 @@ target_link_libraries(quickstep_parser_ParseAssignment
                       quickstep_parser_ParseTreeNode
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_parser_ParseAttributeDefinition
+                      quickstep_parser_ParseDataType
                       quickstep_parser_ParseString
                       quickstep_parser_ParseTreeNode
                       quickstep_types_Type
                       quickstep_utility_Macros
                       quickstep_utility_PtrList)
 target_link_libraries(quickstep_parser_ParseBasicExpressions
+                      quickstep_parser_ParseDataType
                       quickstep_parser_ParseExpression
                       quickstep_parser_ParseLiteralValue
                       quickstep_parser_ParseString
@@ -157,6 +160,11 @@ target_link_libraries(quickstep_parser_ParseCaseExpressions
                       quickstep_parser_ParseTreeNode
                       quickstep_utility_Macros
                       quickstep_utility_PtrVector)
+target_link_libraries(quickstep_parser_ParseDataType
+                      quickstep_parser_ParseLiteralValue
+                      quickstep_parser_ParseString
+                      quickstep_parser_ParseTreeNode
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_parser_ParseExpression
                       quickstep_parser_ParseTreeNode
                       quickstep_utility_Macros)
@@ -336,6 +344,7 @@ target_link_libraries(quickstep_parser_SqlParser
                       quickstep_parser_ParseBasicExpressions
                       quickstep_parser_ParseBlockProperties
                       quickstep_parser_ParseCaseExpressions
+                      quickstep_parser_ParseDataType
                       quickstep_parser_ParseExpression
                       quickstep_parser_ParseGeneratorTableReference
                       quickstep_parser_ParseGroupBy
@@ -409,6 +418,7 @@ target_link_libraries(quickstep_parser
                       quickstep_parser_ParseBasicExpressions
                       quickstep_parser_ParseBlockProperties
                       quickstep_parser_ParseCaseExpressions
+                      quickstep_parser_ParseDataType
                       quickstep_parser_ParseExpression
                       quickstep_parser_ParseGeneratorTableReference
                       quickstep_parser_ParseGroupBy

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/65978e25/parser/ParseAttributeDefinition.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseAttributeDefinition.cpp b/parser/ParseAttributeDefinition.cpp
index ec6508b..0612d9e 100644
--- a/parser/ParseAttributeDefinition.cpp
+++ b/parser/ParseAttributeDefinition.cpp
@@ -54,12 +54,12 @@ void ParseAttributeDefinition::getFieldStringItems(
     std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const {
   inline_field_names->push_back("name");
   inline_field_values->push_back(name_->value());
-  inline_field_names->push_back("type");
-  inline_field_values->push_back(data_type_->getType().getName());
+  non_container_child_field_names->push_back("type");
+  non_container_child_fields->push_back(data_type_.get());
 }
 
 void ParseColumnConstraintNull::applyTo(ParseAttributeDefinition *target) const {
-  target->data_type_->type_ = &(target->data_type_->type_->getNullableVersion());
+  target->setNullable(true);
 }
 
 void ParseColumnConstraintNull::getFieldStringItems(
@@ -74,7 +74,7 @@ void ParseColumnConstraintNull::getFieldStringItems(
 }
 
 void ParseColumnConstraintNotNull::applyTo(ParseAttributeDefinition *target) const {
-  target->data_type_->type_ = &(target->data_type_->type_->getNonNullableVersion());
+  target->setNullable(false);
 }
 
 void ParseColumnConstraintNotNull::getFieldStringItems(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/65978e25/parser/ParseAttributeDefinition.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseAttributeDefinition.hpp b/parser/ParseAttributeDefinition.hpp
index 5c20d13..0e79ac4 100644
--- a/parser/ParseAttributeDefinition.hpp
+++ b/parser/ParseAttributeDefinition.hpp
@@ -24,6 +24,7 @@
 #include <string>
 #include <vector>
 
+#include "parser/ParseDataType.hpp"
 #include "parser/ParseString.hpp"
 #include "parser/ParseTreeNode.hpp"
 #include "utility/Macros.hpp"
@@ -31,7 +32,6 @@
 namespace quickstep {
 
 class ParseColumnConstraint;
-class Type;
 
 template <class T> class PtrList;
 
@@ -40,40 +40,6 @@ template <class T> class PtrList;
  */
 
 /**
- * @brief Parsed representation of a data type.
- **/
-class ParseDataType {
- public:
-  /**
-   * @brief Constructor.
-   *
-   * @param type The Type of the data.
-   **/
-  explicit ParseDataType(const Type &type)
-      : type_(&type) {
-  }
-
-  /**
-   * @brief Get the type.
-   *
-   * @return The Type.
-   **/
-  const Type& getType() const {
-    return *type_;
-  }
-
- private:
-  // Use a pointer instead of a reference so that it may be modified by column
-  // constraints.
-  const Type *type_;
-
-  friend class ParseColumnConstraintNull;
-  friend class ParseColumnConstraintNotNull;
-
-  DISALLOW_COPY_AND_ASSIGN(ParseDataType);
-};
-
-/**
  * @brief Parsed representation of an attribute definition
  **/
 class ParseAttributeDefinition : public ParseTreeNode {
@@ -122,6 +88,14 @@ class ParseAttributeDefinition : public ParseTreeNode {
     return *data_type_;
   }
 
+  const bool nullable() const {
+    return nullable_;
+  }
+
+  void setNullable(const bool nullable) {
+    nullable_ = nullable;
+  }
+
  protected:
   void getFieldStringItems(
       std::vector<std::string> *inline_field_names,
@@ -134,9 +108,7 @@ class ParseAttributeDefinition : public ParseTreeNode {
  private:
   std::unique_ptr<ParseString> name_;
   std::unique_ptr<ParseDataType> data_type_;
-
-  friend class ParseColumnConstraintNull;
-  friend class ParseColumnConstraintNotNull;
+  bool nullable_;
 
   DISALLOW_COPY_AND_ASSIGN(ParseAttributeDefinition);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/65978e25/parser/ParseBasicExpressions.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseBasicExpressions.cpp b/parser/ParseBasicExpressions.cpp
index fd9094c..6425002 100644
--- a/parser/ParseBasicExpressions.cpp
+++ b/parser/ParseBasicExpressions.cpp
@@ -127,6 +127,23 @@ void ParseFunctionCall::getFieldStringItems(
   }
 }
 
+std::string ParseTypeCast::generateName() const {
+  return operand_->generateName();
+}
+
+void ParseTypeCast::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->emplace_back("operand");
+  non_container_child_fields->emplace_back(operand_.get());
+  non_container_child_field_names->emplace_back("target_type");
+  non_container_child_fields->emplace_back(target_type_.get());
+}
+
 std::string ParseArray::generateName() const {
   string name("{");
   if (!elements_.empty()) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/65978e25/parser/ParseBasicExpressions.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseBasicExpressions.hpp b/parser/ParseBasicExpressions.hpp
index af2393f..56295e7 100644
--- a/parser/ParseBasicExpressions.hpp
+++ b/parser/ParseBasicExpressions.hpp
@@ -26,6 +26,7 @@
 #include <string>
 #include <vector>
 
+#include "parser/ParseDataType.hpp"
 #include "parser/ParseExpression.hpp"
 #include "parser/ParseLiteralValue.hpp"
 #include "parser/ParseString.hpp"
@@ -345,6 +346,53 @@ class ParseFunctionCall : public ParseExpression {
 };
 
 
+class ParseTypeCast : public ParseExpression {
+ public:
+  ParseTypeCast(const int line_number,
+                const int column_number,
+                ParseExpression *operand,
+                ParseDataType *target_type)
+      : ParseExpression(line_number, column_number),
+        operand_(operand),
+        target_type_(target_type) {}
+
+  ~ParseTypeCast() override {}
+
+  ExpressionType getExpressionType() const override {
+    return kTypeCast;
+  }
+
+  std::string getName() const override {
+    return "TypeCast";
+  }
+
+  std::string generateName() const override;
+
+  const ParseExpression& operand() const {
+    return *operand_;
+  }
+
+  const ParseDataType& target_type() const {
+    return *target_type_;
+  }
+
+ 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_;
+  std::unique_ptr<ParseDataType> target_type_;
+
+  DISALLOW_COPY_AND_ASSIGN(ParseTypeCast);
+};
+
+
 class ParseArray : public ParseExpression {
  public:
   ParseArray(const int line_number, const int column_number)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/65978e25/parser/ParseDataType-decl.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseDataType-decl.hpp b/parser/ParseDataType-decl.hpp
new file mode 100644
index 0000000..f85ab55
--- /dev/null
+++ b/parser/ParseDataType-decl.hpp
@@ -0,0 +1,24 @@
+/**
+ * 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_PARSER_PARSE_DATA_TYPE_DECL_HPP_
+#define QUICKSTEP_PARSER_PARSE_DATA_TYPE_DECL_HPP_
+
+
+#endif  // QUICKSTEP_PARSER_PARSE_DATA_TYPE_DECL_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/65978e25/parser/ParseDataType.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseDataType.cpp b/parser/ParseDataType.cpp
new file mode 100644
index 0000000..b960961
--- /dev/null
+++ b/parser/ParseDataType.cpp
@@ -0,0 +1,71 @@
+/**
+ * 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 "parser/ParseDataType.hpp"
+
+#include <string>
+#include <vector>
+
+#include "parser/ParseTreeNode.hpp"
+
+namespace quickstep {
+
+void ParseDataType::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->emplace_back("name");
+  inline_field_values->emplace_back(name_->value());
+
+  inline_field_names->emplace_back("nullable");
+
+
+  container_child_field_names->emplace_back("parameters");
+  container_child_fields->emplace_back();
+  for (const auto &parameter : parameters_) {
+    container_child_fields->back().emplace_back(parameter.get());
+  }
+}
+
+void ParseDataTypeParameterLiteralValue::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->emplace_back("literal_value");
+  non_container_child_fields->emplace_back(literal_value_.get());
+}
+
+void ParseDataTypeParameterDataType::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->emplace_back("data_type");
+  non_container_child_fields->emplace_back(data_type_.get());
+}
+
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/65978e25/parser/ParseDataType.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseDataType.hpp b/parser/ParseDataType.hpp
new file mode 100644
index 0000000..96b7b96
--- /dev/null
+++ b/parser/ParseDataType.hpp
@@ -0,0 +1,183 @@
+/**
+ * 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_PARSER_PARSE_DATA_TYPE_HPP_
+#define QUICKSTEP_PARSER_PARSE_DATA_TYPE_HPP_
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parser/ParseLiteralValue.hpp"
+#include "parser/ParseString.hpp"
+#include "parser/ParseTreeNode.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+class Type;
+
+/** \addtogroup Parser
+ *  @{
+ */
+
+class ParseDataTypeParameter : public ParseTreeNode {
+ public:
+  enum class ParameterType {
+    kDataType,
+    kLiteralValue
+  };
+
+  virtual ParameterType getParameterType() const = 0;
+
+ protected:
+  ParseDataTypeParameter(const int line_number,
+                         const int column_number)
+      : ParseTreeNode(line_number, column_number) {}
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(ParseDataTypeParameter);
+};
+
+/**
+ * @brief Parsed representation of a data type.
+ **/
+class ParseDataType : public ParseTreeNode {
+ public:
+  ParseDataType(const int line_number,
+                const int column_number,
+                ParseString *name)
+      : ParseTreeNode(line_number, column_number),
+        name_(name),
+        nullable_(false) {}
+
+  std::string getName() const override {
+    return "DataType";
+  }
+
+  const std::vector<std::unique_ptr<ParseDataTypeParameter>>& parameters() const {
+    return parameters_;
+  }
+
+  bool nullable() const {
+    return nullable_;
+  }
+
+  void addParameter(ParseDataTypeParameter *parameter) {
+    parameters_.emplace_back(std::unique_ptr<ParseDataTypeParameter>(parameter));
+  }
+
+  void setNullable(const bool nullable) {
+    nullable_ = nullable;
+  }
+
+ 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 std::unique_ptr<ParseString> name_;
+  std::vector<std::unique_ptr<ParseDataTypeParameter>> parameters_;
+  bool nullable_;
+
+  DISALLOW_COPY_AND_ASSIGN(ParseDataType);
+};
+
+class ParseDataTypeParameterLiteralValue : public ParseDataTypeParameter {
+ public:
+  ParseDataTypeParameterLiteralValue(const int line_number,
+                                     const int column_number,
+                                     ParseLiteralValue *literal_value)
+      : ParseDataTypeParameter(line_number, column_number),
+        literal_value_(literal_value) {}
+
+  std::string getName() const override {
+    return "DataTypeParameterLiteralValue";
+  }
+
+  ParameterType getParameterType() const override {
+    return ParameterType::kLiteralValue;
+  }
+
+  const ParseLiteralValue& literal_value() const {
+    return *literal_value_;
+  }
+
+ 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<ParseLiteralValue> literal_value_;
+
+  DISALLOW_COPY_AND_ASSIGN(ParseDataTypeParameterLiteralValue);
+};
+
+class ParseDataTypeParameterDataType : public ParseDataTypeParameter {
+ public:
+  ParseDataTypeParameterDataType(const int line_number,
+                                 const int column_number,
+                                 ParseDataType *data_type)
+      : ParseDataTypeParameter(line_number, column_number),
+        data_type_(data_type) {}
+
+  std::string getName() const override {
+    return "DataTypeParameterDataType";
+  }
+
+  ParameterType getParameterType() const override {
+    return ParameterType::kDataType;
+  }
+
+  const ParseDataType& data_type() const {
+    return *data_type_;
+  }
+
+ 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<ParseDataType> data_type_;
+
+  DISALLOW_COPY_AND_ASSIGN(ParseDataTypeParameterDataType);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_PARSER_PARSE_DATA_TYPE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/65978e25/parser/ParseExpression.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseExpression.hpp b/parser/ParseExpression.hpp
index f9a33a2..01da16a 100644
--- a/parser/ParseExpression.hpp
+++ b/parser/ParseExpression.hpp
@@ -45,6 +45,7 @@ class ParseExpression : public ParseTreeNode {
     kSearchedCaseExpression,
     kSimpleCaseExpression,
     kSubqueryExpression,
+    kTypeCast,
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/65978e25/parser/SqlLexer.lpp
----------------------------------------------------------------------
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index b45d8ab..4d4d171 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -90,6 +90,7 @@ class ParseSubqueryExpression;
 class ParseSubqueryTableReference;
 class ParseTableReference;
 class ParseTableReferenceSignature;
+class ParseTypeCast;
 class ParseWindow;
 class Type;
 class UnaryOperation;
@@ -178,8 +179,6 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "asc"              return TOKEN_ASC;
   "ascending"        return TOKEN_ASC;
   "between"          return TOKEN_BETWEEN;
-  "bigint"           return TOKEN_BIGINT;
-  "bit"              return TOKEN_BIT;
   "bitweaving"       return TOKEN_BITWEAVING;
   "blockproperties"  return TOKEN_BLOCKPROPERTIES;
   "blocksample"      return TOKEN_BLOCKSAMPLE;
@@ -188,24 +187,18 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "cast"             return TOKEN_CAST;
   "csbtree"          return TOKEN_CSB_TREE;
   "by"               return TOKEN_BY;
-  "char"             return TOKEN_CHARACTER;
-  "character"        return TOKEN_CHARACTER;
   "check"            return TOKEN_CHECK;
   "column"           return TOKEN_COLUMN;
   "constraint"       return TOKEN_CONSTRAINT;
   "copy"             return TOKEN_COPY;
   "create"           return TOKEN_CREATE;
   "current"          return TOKEN_CURRENT;
-  "date"             return TOKEN_DATE;
-  "datetime"         return TOKEN_DATETIME;
   "day"              return TOKEN_DAY;
-  "decimal"          return TOKEN_DECIMAL;
   "default"          return TOKEN_DEFAULT;
   "delete"           return TOKEN_DELETE;
   "desc"             return TOKEN_DESC;
   "descending"       return TOKEN_DESC;
   "distinct"         return TOKEN_DISTINCT;
-  "double"           return TOKEN_DOUBLE;
   "drop"             return TOKEN_DROP;
   "else"             return TOKEN_ELSE;
   "end"              return TOKEN_END;
@@ -213,7 +206,6 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "extract"          return TOKEN_EXTRACT;
   "false"            return TOKEN_FALSE;
   "first"            return TOKEN_FIRST;
-  "float"            return TOKEN_FLOAT;
   "following"        return TOKEN_FOLLOWING;
   "for"              return TOKEN_FOR;
   "foreign"          return TOKEN_FOREIGN;
@@ -227,8 +219,6 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "index"            return TOKEN_INDEX;
   "inner"            return TOKEN_INNER;
   "insert"           return TOKEN_INSERT;
-  "int"              return TOKEN_INTEGER;
-  "integer"          return TOKEN_INTEGER;
   "intersect"        return TOKEN_INTERSECT;
   "interval"         return TOKEN_INTERVAL;
   "into"             return TOKEN_INTO;
@@ -239,7 +229,6 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "left"             return TOKEN_LEFT;
   "like"             return TOKEN_LIKE;
   "limit"            return TOKEN_LIMIT;
-  "long"             return TOKEN_LONG;
   "minute"           return TOKEN_MINUTE;
   "month"            return TOKEN_MONTH;
   "not"              return TOKEN_NOT;
@@ -270,14 +259,11 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "select"           return TOKEN_SELECT;
   "set"              return TOKEN_SET;
   "sma"              return TOKEN_SMA;
-  "smallint"         return TOKEN_SMALLINT;
   "stderr"           return TOKEN_STDERR;
   "stdout"           return TOKEN_STDOUT;
   "substring"        return TOKEN_SUBSTRING;
   "table"            return TOKEN_TABLE;
   "then"             return TOKEN_THEN;
-  "time"             return TOKEN_TIME;
-  "timestamp"        return TOKEN_TIMESTAMP;
   "to"               return TOKEN_TO;
   "true"             return TOKEN_TRUE;
   "tuplesample"      return TOKEN_TUPLESAMPLE;
@@ -287,13 +273,11 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "update"           return TOKEN_UPDATE;
   "using"            return TOKEN_USING;
   "values"           return TOKEN_VALUES;
-  "varchar"          return TOKEN_VARCHAR;
   "when"             return TOKEN_WHEN;
   "where"            return TOKEN_WHERE;
   "window"           return TOKEN_WINDOW;
   "with"             return TOKEN_WITH;
   "year"             return TOKEN_YEAR;
-  "yearmonth"        return TOKEN_YEARMONTH;
 
   "="                return TOKEN_EQ;
   "!="               return TOKEN_NEQ;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/65978e25/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index 01be345..16e30bc 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -30,7 +30,7 @@
  * occurs, Bison resolves it by preferring to shift rather than reduce, which
  * is the correct, expected behavior for both cases in this parser.
  **/
-%expect 2
+%expect 1
 
 %{
 
@@ -72,6 +72,7 @@ typedef struct YYLTYPE {
 #include "parser/ParseBasicExpressions.hpp"
 #include "parser/ParseBlockProperties.hpp"
 #include "parser/ParseCaseExpressions.hpp"
+#include "parser/ParseDataType.hpp"
 #include "parser/ParseExpression.hpp"
 #include "parser/ParseGeneratorTableReference.hpp"
 #include "parser/ParseGroupBy.hpp"
@@ -247,7 +248,6 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_AND;
 %token TOKEN_AS;
 %token TOKEN_ASC;
-%token TOKEN_BIGINT;
 %token TOKEN_BIT;
 %token TOKEN_BITWEAVING;
 %token TOKEN_BLOCKPROPERTIES;
@@ -257,22 +257,17 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_BY;
 %token TOKEN_CASE;
 %token TOKEN_CAST;
-%token TOKEN_CHARACTER;
 %token TOKEN_CHECK;
 %token TOKEN_COLUMN;
 %token TOKEN_CONSTRAINT;
 %token TOKEN_COPY;
 %token TOKEN_CREATE;
 %token TOKEN_CURRENT;
-%token TOKEN_DATE;
-%token TOKEN_DATETIME;
 %token TOKEN_DAY;
-%token TOKEN_DECIMAL;
 %token TOKEN_DEFAULT;
 %token TOKEN_DELETE;
 %token TOKEN_DESC;
 %token TOKEN_DISTINCT;
-%token TOKEN_DOUBLE;
 %token TOKEN_DOUBLECOLON;
 %token TOKEN_DROP;
 %token TOKEN_ELSE;
@@ -281,7 +276,6 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_EXTRACT;
 %token TOKEN_FALSE;
 %token TOKEN_FIRST;
-%token TOKEN_FLOAT;
 %token TOKEN_FOLLOWING;
 %token TOKEN_FOR;
 %token TOKEN_FOREIGN;
@@ -295,7 +289,6 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_INDEX;
 %token TOKEN_INNER;
 %token TOKEN_INSERT;
-%token TOKEN_INTEGER;
 %token TOKEN_INTERSECT;
 %token TOKEN_INTERVAL;
 %token TOKEN_INTO;
@@ -305,7 +298,6 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_LBRACE;
 %token TOKEN_LEFT;
 %token TOKEN_LIMIT;
-%token TOKEN_LONG;
 %token TOKEN_MINUTE;
 %token TOKEN_MONTH;
 %token TOKEN_NOT;
@@ -337,14 +329,11 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_SELECT;
 %token TOKEN_SET;
 %token TOKEN_SMA;
-%token TOKEN_SMALLINT;
 %token TOKEN_STDERR;
 %token TOKEN_STDOUT;
 %token TOKEN_SUBSTRING;
 %token TOKEN_TABLE;
 %token TOKEN_THEN;
-%token TOKEN_TIME;
-%token TOKEN_TIMESTAMP;
 %token TOKEN_TO;
 %token TOKEN_TRUE;
 %token TOKEN_TUPLESAMPLE;
@@ -354,13 +343,11 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_UPDATE;
 %token TOKEN_USING;
 %token TOKEN_VALUES;
-%token TOKEN_VARCHAR;
 %token TOKEN_WHEN;
 %token TOKEN_WHERE;
 %token TOKEN_WINDOW;
 %token TOKEN_WITH;
 %token TOKEN_YEAR;
-%token TOKEN_YEARMONTH;
 %token TOKEN_EOF;
 %token TOKEN_LEX_ERROR;
 
@@ -375,6 +362,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
   boolean_value
   frame_mode
   opt_all_distinct
+  opt_nullable
 
 %type <literal_value_>
   literal_value
@@ -458,6 +446,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 
 %type <data_type_>
   data_type
+  data_type_parameter_commalist
 
 %type <attribute_definition_>
   column_def
@@ -768,102 +757,46 @@ column_def_commalist:
   };
 
 data_type:
-  TOKEN_BIT {
-    $$ = nullptr;
-    NotSupported(&@1, yyscanner, "BIT data type");
-    YYERROR;
-  }
-  | TOKEN_DATE {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDate));
-  }
-  | TOKEN_DATETIME {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetime));
-  }
-  | TOKEN_TIME {
-    $$ = nullptr;
-    NotSupported(&@1, yyscanner, "TIME data type");
-    YYERROR;
-  }
-  | TOKEN_TIMESTAMP {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetime));
-  }
-  | TOKEN_DECIMAL {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDouble));
-  }
-  | TOKEN_REAL {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDouble));
-  }
-  | TOKEN_DOUBLE {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDouble));
-  }
-  | TOKEN_FLOAT {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kFloat));
-  }
-  | TOKEN_SMALLINT {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kInt));
-  }
-  | TOKEN_INTEGER {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kInt));
-  }
-  | TOKEN_BIGINT {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kLong));
+  any_name {
+    $$ = new quickstep::ParseDataType(@1.first_line, @1.first_column, $1);
   }
-  | TOKEN_LONG {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kLong));
+  | data_type_parameter_commalist ')' {
+    $$ = $1;
+  };
+
+data_type_parameter_commalist:
+  any_name '(' data_type opt_nullable {
+    $$ = new quickstep::ParseDataType(@1.first_line, @1.first_column, $1);
+    $3->setNullable($4);
+    $$->addParameter(
+        new quickstep::ParseDataTypeParameterDataType($3->line_number(), $3->column_number(), $3));
   }
-  | TOKEN_INTERVAL {
-    /**
-     * NOTE(chasseur): This pattern exhibits a shift/reduce conflict with the
-     * TOKEN_INTERVAL case in 'literal_value'. Bison prefers to shift rather
-     * than reduce, so the case in 'literal_value' has precedence over this.
-     **/
-    $$ = nullptr;
-    quickstep_yyerror(&@1, yyscanner, nullptr,
-        "INTERVAL is ambiguous as a column type. Specify either DATETIME INTERVAL "
-        "or YEARMONTH INTERVAL");
-    YYERROR;
+  | data_type_parameter_commalist ',' data_type opt_nullable {
+    $$ = $1;
+    $3->setNullable($4);
+    $$->addParameter(
+        new quickstep::ParseDataTypeParameterDataType($3->line_number(), $3->column_number(), $3));
   }
-  | TOKEN_DATETIME TOKEN_INTERVAL {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kDatetimeInterval));
+  | any_name '(' literal_value {
+    $$ = new quickstep::ParseDataType(@1.first_line, @1.first_column, $1);
+    $$->addParameter(
+        new quickstep::ParseDataTypeParameterLiteralValue($3->line_number(), $3->column_number(), $3));
   }
-  | TOKEN_YEARMONTH TOKEN_INTERVAL {
-    $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kYearMonthInterval));
+  | data_type_parameter_commalist ',' literal_value {
+    $$ = $1;
+    $$->addParameter(
+        new quickstep::ParseDataTypeParameterLiteralValue($3->line_number(), $3->column_number(), $3));
+  };
+
+opt_nullable:
+  {
+    $$ = false;  // NOT NULL
   }
-  | TOKEN_CHARACTER '(' TOKEN_UNSIGNED_NUMVAL ')' {
-    if ($3->float_like()) {
-      delete $3;
-      $$ = NULL;
-      quickstep_yyerror(&@3, yyscanner, nullptr, "Non-integer length supplied for CHAR type");
-      YYERROR;
-    } else {
-      if ($3->long_value() <= 0) {
-        delete $3;
-        $$ = NULL;
-        quickstep_yyerror(&@3, yyscanner, nullptr, "Length for CHAR type must be at least 1");
-        YYERROR;
-      } else {
-        $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kChar, $3->long_value(), false));
-        delete $3;
-      }
-    }
+  | TOKEN_NULL {
+    $$ = true;  // NULL
   }
-  | TOKEN_VARCHAR '(' TOKEN_UNSIGNED_NUMVAL ')' {
-    if ($3->float_like()) {
-      delete $3;
-      $$ = NULL;
-      quickstep_yyerror(&@3, yyscanner, nullptr, "Non-integer length supplied for VARCHAR type");
-      YYERROR;
-    } else {
-      if ($3->long_value() < 0) {
-        delete $3;
-        $$ = NULL;
-        quickstep_yyerror(&@3, yyscanner, nullptr, "Negative length supplied for VARCHAR type");
-        YYERROR;
-      } else {
-        $$ = new quickstep::ParseDataType(quickstep::TypeFactory::GetType(quickstep::kVarChar, $3->long_value(), false));
-        delete $3;
-      }
-    }
+  | TOKEN_NOT TOKEN_NULL {
+    $$ = true;  // NOT NULL
   };
 
 column_constraint_def:
@@ -1787,51 +1720,11 @@ function_call:
 
 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);
-  }
-  | TOKEN_CAST '(' add_expression TOKEN_AS any_name ')' {
-    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
-    arguments->push_back($3);
-    arguments->push_back(new quickstep::ParseScalarLiteral(
-        new quickstep::StringParseLiteralValue($5, nullptr)));
-    auto *name = new quickstep::ParseString(@1.first_line, @1.first_column, "cast");
-    $$ = new quickstep::ParseFunctionCall(
-        @1.first_line, @1.first_column, false, name, arguments);
+    $$ = new quickstep::ParseTypeCast(@1.first_line, @1.first_column, $3, $5);
   }
   | expression_base TOKEN_DOUBLECOLON data_type {
-    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
-    arguments->push_back($1);
-    arguments->push_back(new quickstep::ParseScalarLiteral(
-        new quickstep::StringParseLiteralValue(
-            new quickstep::ParseString(@3.first_line,
-                                       @3.first_column,
-                                       $3->getType().getName()),
-            nullptr)));
-    delete $3;
-    auto *name = new quickstep::ParseString(@2.first_line, @2.first_column, "cast");
-    $$ = new quickstep::ParseFunctionCall(
-        @2.first_line, @2.first_column, false, name, arguments);
-  }
-  | expression_base TOKEN_DOUBLECOLON any_name {
-    auto *arguments = new quickstep::PtrList<quickstep::ParseExpression>();
-    arguments->push_back($1);
-    arguments->push_back(new quickstep::ParseScalarLiteral(
-        new quickstep::StringParseLiteralValue($3, nullptr)));
-    auto *name = new quickstep::ParseString(@2.first_line, @2.first_column, "cast");
-    $$ = new quickstep::ParseFunctionCall(
-        @2.first_line, @2.first_column, false, name, arguments);
-  }
+    $$ = new quickstep::ParseTypeCast(@2.first_line, @2.first_column, $1, $3);
+  };
 
 extract_function:
   TOKEN_EXTRACT '(' datetime_unit TOKEN_FROM add_expression ')' {
@@ -1976,20 +1869,7 @@ literal_value:
       quickstep_yyerror(&@3, yyscanner, nullptr, "Failed to parse literal as specified type");
       YYERROR;
     }
-  }
-  | data_type TOKEN_STRING_SINGLE_QUOTED {
-    quickstep::StringParseLiteralValue *parse_value
-        = new quickstep::StringParseLiteralValue($2, &($1->getType()));
-    delete $1;
-    if (!parse_value->tryExplicitTypeParse()) {
-      delete parse_value;
-      $$ = nullptr;
-      quickstep_yyerror(&@2, yyscanner, nullptr, "Failed to parse literal as specified type");
-      YYERROR;
-    } else {
-      $$ = parse_value;
-    }
-  }
+  };
 
 datetime_unit:
   TOKEN_YEAR {