You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by sh...@apache.org on 2016/06/17 20:35:19 UTC

[7/7] incubator-quickstep git commit: QUICKSTEP-20: Added parser support for SQL window aggregation function

QUICKSTEP-20: Added parser support for SQL window aggregation function


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

Branch: refs/heads/SQL-window-aggregation
Commit: 00ca1e4b3a9c9838dcb9509058b8a40b0f573617
Parents: 8e825f1
Author: shixuan <sh...@wisc.edu>
Authored: Tue Jun 14 23:07:32 2016 +0000
Committer: shixuan <sh...@wisc.edu>
Committed: Fri Jun 17 20:34:24 2016 +0000

----------------------------------------------------------------------
 parser/CMakeLists.txt                   |   11 +
 parser/ParseBasicExpressions.cpp        |   10 +
 parser/ParseBasicExpressions.hpp        |   45 +
 parser/ParseSelect.hpp                  |   27 +-
 parser/ParseWindow.hpp                  |  201 ++
 parser/SqlLexer.lpp                     |   10 +
 parser/SqlParser.ypp                    |  128 +-
 parser/preprocessed/SqlLexer_gen.cpp    | 1671 ++++++-------
 parser/preprocessed/SqlLexer_gen.hpp    |   16 +-
 parser/preprocessed/SqlParser_gen.cpp   | 3367 ++++++++++++++------------
 parser/preprocessed/SqlParser_gen.hpp   |  182 +-
 parser/tests/Select.test                |  120 +
 query_optimizer/resolver/CMakeLists.txt |    1 +
 query_optimizer/resolver/Resolver.cpp   |    7 +
 14 files changed, 3368 insertions(+), 2428 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index 2488d30..55c4a67 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -113,6 +113,7 @@ add_library(quickstep_parser_ParseSubqueryExpression ParseSubqueryExpression.cpp
 add_library(quickstep_parser_ParseSubqueryTableReference ParseSubqueryTableReference.cpp ParseSubqueryTableReference.hpp)
 add_library(quickstep_parser_ParseTableReference ParseTableReference.cpp ParseTableReference.hpp)
 add_library(quickstep_parser_ParseTreeNode ../empty_src.cpp ParseTreeNode.hpp)
+add_library(quickstep_parser_ParseWindow ../empty_src.cpp ParseWindow.hpp)
 add_library(quickstep_parser_ParserUtil ParserUtil.cpp ParserUtil.hpp)
 add_library(quickstep_parser_SqlParserWrapper SqlParserWrapper.cpp SqlParserWrapper.hpp)
 add_library(quickstep_parser_SqlParser ${BISON_SqlParser_OUTPUTS})
@@ -135,6 +136,7 @@ target_link_libraries(quickstep_parser_ParseBasicExpressions
                       quickstep_parser_ParseLiteralValue
                       quickstep_parser_ParseString
                       quickstep_parser_ParseTreeNode
+                      quickstep_parser_ParseWindow
                       quickstep_types_operations_binaryoperations_BinaryOperation
                       quickstep_types_operations_unaryoperations_UnaryOperation
                       quickstep_utility_Macros
@@ -241,6 +243,7 @@ target_link_libraries(quickstep_parser_ParseSelect
                       quickstep_parser_ParseSelectionClause
                       quickstep_parser_ParseTableReference
                       quickstep_parser_ParseTreeNode
+                      quickstep_parser_ParseWindow
                       quickstep_utility_Macros
                       quickstep_utility_PtrList)
 target_link_libraries(quickstep_parser_ParseSelectionClause
@@ -300,6 +303,12 @@ target_link_libraries(quickstep_parser_ParseTreeNode
                       quickstep_utility_TreeStringSerializable)
 target_link_libraries(quickstep_parser_ParserUtil
                       quickstep_utility_SqlError)
+target_link_libraries(quickstep_parser_ParseWindow
+                      quickstep_parser_ParseExpression
+                      quickstep_parser_ParseOrderBy
+                      quickstep_parser_ParseString
+                      quickstep_parser_ParseTreeNode
+                      quickstep_utility_PtrList)
 target_link_libraries(quickstep_parser_SqlLexer
                       quickstep_parser_ParseJoinedTableReference
                       quickstep_parser_ParseLiteralValue
@@ -337,6 +346,7 @@ target_link_libraries(quickstep_parser_SqlParser
                       quickstep_parser_ParseSubqueryExpression
                       quickstep_parser_ParseSubqueryTableReference
                       quickstep_parser_ParseTableReference
+                      quickstep_parser_ParseWindow
                       quickstep_parser_ParserUtil
                       quickstep_storage_StorageBlockInfo
                       quickstep_types_Type
@@ -419,6 +429,7 @@ target_link_libraries(quickstep_parser
                       quickstep_parser_ParseSubqueryTableReference
                       quickstep_parser_ParseTableReference
                       quickstep_parser_ParseTreeNode
+                      quickstep_parser_ParseWindow
                       quickstep_parser_SqlLexer
                       quickstep_parser_SqlParser
                       quickstep_parser_SqlParserWrapper)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/ParseBasicExpressions.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseBasicExpressions.cpp b/parser/ParseBasicExpressions.cpp
index a9d84ea..bbb6801 100644
--- a/parser/ParseBasicExpressions.cpp
+++ b/parser/ParseBasicExpressions.cpp
@@ -162,6 +162,16 @@ void ParseFunctionCall::getFieldStringItems(
       non_container_child_field_names->push_back("");
       non_container_child_fields->push_back(&argument);
     }
+
+    if (window_name_ != nullptr) {
+      inline_field_names->push_back("window_name");
+      inline_field_values->push_back(window_name_->value());
+    }
+
+    if (window_ != nullptr) {
+      non_container_child_field_names->push_back("window");
+      non_container_child_fields->push_back(window_.get());
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/ParseBasicExpressions.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseBasicExpressions.hpp b/parser/ParseBasicExpressions.hpp
index dea25d7..64ac119 100644
--- a/parser/ParseBasicExpressions.hpp
+++ b/parser/ParseBasicExpressions.hpp
@@ -30,6 +30,7 @@
 #include "parser/ParseLiteralValue.hpp"
 #include "parser/ParseString.hpp"
 #include "parser/ParseTreeNode.hpp"
+#include "parser/ParseWindow.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrList.hpp"
 
@@ -429,6 +430,46 @@ class ParseFunctionCall : public ParseExpression {
     return star_.get();
   }
 
+  /**
+   * @return The window name.
+   **/
+  const ParseString* window_name() const {
+    return window_name_.get();
+  }
+
+  /**
+   * @return The window.
+   **/
+  const ParseWindow* window() const {
+    return window_.get();
+  }
+
+  /**
+   * @brief Check if this function is a window aggregation function
+   *
+   * @return True if this function is a window aggregation function; false
+   *         otherwise.
+   **/
+  bool isWindow() const {
+    return window_name_ != nullptr || window_ != nullptr;
+  }
+
+  /**
+   * @brief Set the window name.
+   * @param window_name The window name.
+   **/
+  void setWindowName(ParseString *window_name) {
+    window_name_.reset(window_name);
+  }
+
+  /**
+   * @brief Set the window.
+   * @param window The window.
+   **/
+  void setWindow(ParseWindow *window) {
+    window_.reset(window);
+  }
+
   std::string generateName() const override;
 
  protected:
@@ -446,6 +487,10 @@ class ParseFunctionCall : public ParseExpression {
   // Either <arguments_> or <star_> is NULL.
   std::unique_ptr<PtrList<ParseExpression>> arguments_;
   std::unique_ptr<ParseStar> star_;
+  // A window aggregation function should have either <window_name_> or <window_> but not both.
+  // <window_name_> and <window_> will both be NULL if it is not a window function.
+  std::unique_ptr<ParseString> window_name_;
+  std::unique_ptr<ParseWindow> window_;
 
   DISALLOW_COPY_AND_ASSIGN(ParseFunctionCall);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/ParseSelect.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseSelect.hpp b/parser/ParseSelect.hpp
index 81e9ab8..930a215 100644
--- a/parser/ParseSelect.hpp
+++ b/parser/ParseSelect.hpp
@@ -1,6 +1,8 @@
 /**
  *   Copyright 2011-2015 Quickstep Technologies LLC.
  *   Copyright 2015 Pivotal Software, Inc.
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
  *
  *   Licensed under the Apache License, Version 2.0 (the "License");
  *   you may not use this file except in compliance with the License.
@@ -30,6 +32,7 @@
 #include "parser/ParseSelectionClause.hpp"
 #include "parser/ParseTableReference.hpp"
 #include "parser/ParseTreeNode.hpp"
+#include "parser/ParseWindow.hpp"
 #include "utility/Macros.hpp"
 #include "utility/PtrList.hpp"
 
@@ -68,7 +71,8 @@ class ParseSelect : public ParseTreeNode {
               ParseGroupBy *group_by,
               ParseHaving *having,
               ParseOrderBy *order_by,
-              ParseLimit *limit)
+              ParseLimit *limit,
+              PtrList<ParseWindow> *window_list)
       : ParseTreeNode(line_number, column_number),
         selection_(selection),
         from_list_(from_list),
@@ -76,7 +80,8 @@ class ParseSelect : public ParseTreeNode {
         group_by_(group_by),
         having_(having),
         order_by_(order_by),
-        limit_(limit) {
+        limit_(limit),
+        window_list_(window_list) {
   }
 
   ~ParseSelect() override {
@@ -152,6 +157,13 @@ class ParseSelect : public ParseTreeNode {
    */
   const ParseLimit* limit() const { return limit_.get(); }
 
+  /**
+   * @brief Gets the parsed WINDOW.
+   *
+   * @return The parsed WINDOW.
+   */
+  const PtrList<ParseWindow>* window_list() const { return window_list_.get(); }
+
  protected:
   void getFieldStringItems(
       std::vector<std::string> *inline_field_names,
@@ -171,7 +183,7 @@ class ParseSelect : public ParseTreeNode {
     if (from_list_ != nullptr) {
       container_child_field_names->push_back("from_clause");
       container_child_fields->emplace_back();
-      for (const ParseTableReference& from_item : *from_list_) {
+      for (const ParseTableReference &from_item : *from_list_) {
         container_child_fields->back().push_back(&from_item);
       }
     }
@@ -195,6 +207,14 @@ class ParseSelect : public ParseTreeNode {
       non_container_child_field_names->push_back("limit");
       non_container_child_fields->push_back(limit_.get());
     }
+
+    if (window_list_ != nullptr) {
+      container_child_field_names->push_back("window_list");
+      container_child_fields->emplace_back();
+      for (const ParseWindow &window : *window_list_) {
+        container_child_fields->back().push_back(&window);
+      }
+    }
   }
 
  private:
@@ -205,6 +225,7 @@ class ParseSelect : public ParseTreeNode {
   std::unique_ptr<ParseHaving> having_;
   std::unique_ptr<ParseOrderBy> order_by_;
   std::unique_ptr<ParseLimit> limit_;
+  std::unique_ptr<PtrList<ParseWindow>> window_list_;
 
   DISALLOW_COPY_AND_ASSIGN(ParseSelect);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/ParseWindow.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseWindow.hpp b/parser/ParseWindow.hpp
new file mode 100644
index 0000000..7b41683
--- /dev/null
+++ b/parser/ParseWindow.hpp
@@ -0,0 +1,201 @@
+/**
+ *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
+ *     University of Wisconsin\u2014Madison.
+ *
+ *   Licensed 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_WINDOW_HPP_
+#define QUICKSTEP_PARSER_PARSE_WINDOW_HPP_
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parser/ParseExpression.hpp"
+#include "parser/ParseOrderBy.hpp"
+#include "parser/ParseString.hpp"
+#include "parser/ParseTreeNode.hpp"
+#include "utility/PtrList.hpp"
+
+namespace quickstep {
+
+/**
+ * @brief The information of the how the framing in the window is defined
+ **/
+struct ParseFrameInfo : ParseTreeNode {
+  /**
+   * @brief Constructor.
+   * @param row True if the frame mode is ROW, false if it is RANGE.
+   * @param num_pre The number of rows/value of range that is preceding
+   *                the current row in the frame.
+   * @param num_follow The number of rows/value of range that is following
+   *                   the current row in the frame.
+   **/  
+  ParseFrameInfo(const int line_number,
+                 const int column_number,
+                 const bool is_row_in,
+                 const std::int64_t num_preceding_in,
+                 const std::int64_t num_following_in)
+      : ParseTreeNode(line_number, column_number),
+        is_row(is_row_in),
+        num_preceding(num_preceding_in),
+        num_following(num_following_in) {
+  }
+
+  std::string getName() const override { return "FrameInfo"; }
+
+  const bool is_row;
+  const std::int64_t num_preceding;
+  const std::int64_t num_following;
+
+ 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 {
+    inline_field_names->push_back("frame_mode");
+    inline_field_values->push_back(is_row ? "row" : "range");
+
+    inline_field_names->push_back("num_preceding");
+    inline_field_values->push_back(std::to_string(num_preceding));
+
+    inline_field_names->push_back("num_following");
+    inline_field_values->push_back(std::to_string(num_following));
+  }
+};
+
+/**
+ * @brief The parsed representation of a WINDOW definition.
+ **/
+class ParseWindow : public ParseTreeNode {
+ public:
+  /**
+   * @brief Constructor.
+   * @param line_number The line number of the first token of this WINDOW clause
+   *                    in the SQL statement.
+   * @param column_number The column number of the first token of this WINDOW
+   *                      clause in the SQL statement.
+   * @param partition_by_expressions Optional grouping expressions that might be
+   *                                 specified in the SQL statement. Similar to
+   *                                 GROUP BY with regular aggregates.
+   * @param order_by_expressions Optional ordering expressions that might be
+   *                             specified in the SQL statement.
+   * @param frame_info The information about framing.
+   **/
+  ParseWindow(const int line_number,
+              const int column_number,
+              PtrList<ParseExpression> *partition_by_expressions,
+              PtrList<ParseOrderByItem> *order_by_expressions,
+              ParseFrameInfo *frame_info)
+      : ParseTreeNode(line_number, column_number),
+        partition_by_expressions_(partition_by_expressions),
+        order_by_expressions_(order_by_expressions),
+        frame_info_(frame_info) {
+  }
+
+  /**
+   * @brief Destructor.
+   **/
+  ~ParseWindow() override {}
+
+  std::string getName() const override {
+    return "window";
+  }
+
+  /**
+   * @brief Grouping expressions.
+   **/
+  const PtrList<ParseExpression>* partition_by_expressions() const {
+    return partition_by_expressions_.get();
+  }
+
+  /**
+   * @brief Ordering expressions.
+   **/
+  const PtrList<ParseOrderByItem>* order_by_expressions() const {
+    return order_by_expressions_.get();
+  }
+
+  /**
+   * @brief Frame information.
+   **/
+  const ParseFrameInfo* frame_info() const {
+    return frame_info_.get();
+  }
+
+  /**
+   * @return The window name.
+   */
+  const ParseString* name() const {
+    return name_.get();
+  }
+
+  /**
+   * @brief Set the name of the window.
+   * @param name The name of the window.
+   **/
+  void setName(ParseString *name) {
+    name_.reset(name);
+  }
+
+ 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 {
+    if (name_ != nullptr) {
+      inline_field_names->push_back("window_name");
+      inline_field_values->push_back(name_->value());
+    }
+
+    container_child_field_names->push_back("partition_by");
+    container_child_fields->emplace_back();
+    if (partition_by_expressions_ != nullptr) {
+      for (const auto &e : *partition_by_expressions_) {
+        container_child_fields->back().emplace_back(&e);
+      }
+    }
+
+    container_child_field_names->push_back("order_by");
+    container_child_fields->emplace_back();
+    if (order_by_expressions_ != nullptr) {
+      for (const auto &e : *order_by_expressions_) {
+        container_child_fields->back().emplace_back(&e);
+      }
+    }
+
+    if (frame_info_ != nullptr) {
+      non_container_child_field_names->push_back("frame_info");
+      non_container_child_fields->push_back(frame_info_.get());
+    }
+  }
+
+ private:
+  std::unique_ptr<PtrList<ParseExpression>> partition_by_expressions_;
+  std::unique_ptr<PtrList<ParseOrderByItem>> order_by_expressions_;
+  std::unique_ptr<ParseFrameInfo> frame_info_;
+  std::unique_ptr<ParseString> name_;
+};
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_PARSER_PARSE_WINDOW_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/SqlLexer.lpp
----------------------------------------------------------------------
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index ac1c708..ee34400 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -48,6 +48,7 @@ class ParseCommand;
 struct ParseCopyFromParams;
 class ParseDataType;
 class ParseExpression;
+struct ParseFrameInfo;
 class ParseFunctionCall;
 class ParseGroupBy;
 class ParseHaving;
@@ -86,6 +87,7 @@ class ParseSubqueryExpression;
 class ParseSubqueryTableReference;
 class ParseTableReference;
 class ParseTableReferenceSignature;
+class ParseWindow;
 class Type;
 class UnaryOperation;
 
@@ -189,6 +191,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "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;
@@ -209,6 +212,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "false"            return TOKEN_FALSE;
   "first"            return TOKEN_FIRST;
   "float"            return TOKEN_FLOAT;
+  "following"        return TOKEN_FOLLOWING;
   "for"              return TOKEN_FOR;
   "foreign"          return TOKEN_FOREIGN;
   "from"             return TOKEN_FROM;
@@ -243,9 +247,11 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "or"               return TOKEN_OR;
   "order"            return TOKEN_ORDER;
   "outer"            return TOKEN_OUTER;
+  "over"             return TOKEN_OVER;
   "partition"        return TOKEN_PARTITION;
   "partitions"       return TOKEN_PARTITIONS;
   "percent"          return TOKEN_PERCENT;
+  "preceding"        return TOKEN_PRECEDING;
   "primary"          return TOKEN_PRIMARY;
   "quit"             return TOKEN_QUIT;
   "range"            return TOKEN_RANGE;
@@ -253,7 +259,9 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "references"       return TOKEN_REFERENCES;
   "regexp"           return TOKEN_REGEXP;
   "right"            return TOKEN_RIGHT;
+  "row"              return TOKEN_ROW;
   "row_delimiter"    return TOKEN_ROW_DELIMITER;
+  "rows"             return TOKEN_ROWS;
   "second"           return TOKEN_SECOND;
   "select"           return TOKEN_SELECT;
   "set"              return TOKEN_SET;
@@ -266,6 +274,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "timestamp"        return TOKEN_TIMESTAMP;
   "true"             return TOKEN_TRUE;
   "tuplesample"      return TOKEN_TUPLESAMPLE;
+  "unbounded"        return TOKEN_UNBOUNDED;
   "unique"           return TOKEN_UNIQUE;
   "update"           return TOKEN_UPDATE;
   "using"            return TOKEN_USING;
@@ -273,6 +282,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "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;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/00ca1e4b/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index b07c48e..81fa3ae 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -95,6 +95,7 @@ typedef struct YYLTYPE {
 #include "parser/ParseSubqueryExpression.hpp"
 #include "parser/ParseSubqueryTableReference.hpp"
 #include "parser/ParseTableReference.hpp"
+#include "parser/ParseWindow.hpp"
 #include "storage/StorageBlockInfo.hpp"
 #include "types/Type.hpp"
 #include "types/TypeFactory.hpp"
@@ -200,6 +201,12 @@ typedef void* yyscan_t;
 
   quickstep::ParseSample *opt_sample_clause_;
 
+  quickstep::PtrList<quickstep::ParseWindow> *opt_window_clause_;
+  quickstep::ParseWindow *window_definition_;
+  quickstep::PtrList<quickstep::ParseExpression> *window_partition_by_list_;
+  quickstep::PtrList<quickstep::ParseOrderByItem> *window_order_by_list_;
+  quickstep::ParseFrameInfo *window_frame_info_;
+
   quickstep::PtrList<quickstep::ParseOrderByItem> *order_commalist_;
   quickstep::ParseOrderByItem *order_item_;
 
@@ -254,6 +261,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_CONSTRAINT;
 %token TOKEN_COPY;
 %token TOKEN_CREATE;
+%token TOKEN_CURRENT;
 %token TOKEN_DATE;
 %token TOKEN_DATETIME;
 %token TOKEN_DAY;
@@ -273,6 +281,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_FALSE;
 %token TOKEN_FIRST;
 %token TOKEN_FLOAT;
+%token TOKEN_FOLLOWING;
 %token TOKEN_FOR;
 %token TOKEN_FOREIGN;
 %token TOKEN_FROM;
@@ -304,9 +313,11 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_OR;
 %token TOKEN_ORDER;
 %token TOKEN_OUTER;
+%token TOKEN_OVER;
 %token TOKEN_PARTITION;
 %token TOKEN_PARTITIONS;
 %token TOKEN_PERCENT;
+%token TOKEN_PRECEDING;
 %token TOKEN_PRIMARY;
 %token TOKEN_QUIT;
 %token TOKEN_RANGE;
@@ -314,7 +325,9 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_REFERENCES;
 %token TOKEN_REGEXP;
 %token TOKEN_RIGHT;
+%token TOKEN_ROW;
 %token TOKEN_ROW_DELIMITER;
+%token TOKEN_ROWS;
 %token TOKEN_SECOND;
 %token TOKEN_SELECT;
 %token TOKEN_SET;
@@ -327,6 +340,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_TIMESTAMP;
 %token TOKEN_TRUE;
 %token TOKEN_TUPLESAMPLE;
+%token TOKEN_UNBOUNDED;
 %token TOKEN_UNIQUE;
 %token TOKEN_UPDATE;
 %token TOKEN_USING;
@@ -334,6 +348,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_VARCHAR;
 %token TOKEN_WHEN;
 %token TOKEN_WHERE;
+%token TOKEN_WINDOW;
 %token TOKEN_WITH;
 %token TOKEN_YEAR;
 %token TOKEN_YEARMONTH;
@@ -348,10 +363,15 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 
 %type <boolean_value_>
   boolean_value
+  frame_mode
 
 %type <literal_value_>
   literal_value
 
+%type <numeric_literal_value_>
+  frame_preceding
+  frame_following
+
 %type <literal_value_list_>
   literal_value_commalist
 
@@ -546,6 +566,23 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %type <opt_sample_clause_>
   opt_sample_clause
 
+%type <opt_window_clause_>
+  opt_window_clause
+  window_declaration_list
+
+%type <window_definition_>
+  window_declaration
+  window_definition
+
+%type <window_partition_by_list_>
+  opt_window_partition
+
+%type <window_order_by_list_>
+  opt_window_order
+
+%type <window_frame_info_>
+  opt_window_frame
+
 %type <with_list_>
   with_clause
   with_list
@@ -1136,8 +1173,8 @@ with_list_element:
 
 select_query:
   TOKEN_SELECT opt_all_distinct selection from_clause opt_where_clause opt_group_by_clause opt_having_clause
-      opt_order_by_clause opt_limit_clause {
-    $$ = new quickstep::ParseSelect(@1.first_line, @1.first_column, $3, $4, $5, $6, $7, $8, $9);
+      opt_order_by_clause opt_limit_clause opt_window_clause {
+    $$ = new quickstep::ParseSelect(@1.first_line, @1.first_column, $3, $4, $5, $6, $7, $8, $9, $10);
   };
 
 opt_all_distinct:
@@ -1332,6 +1369,85 @@ opt_limit_clause:
     }
   }
 
+opt_window_clause:
+  {
+    $$ = nullptr;
+  }
+  | window_declaration_list {
+    $$ = $1;
+  }
+
+window_declaration_list:
+  window_declaration {
+    $$ = new quickstep::PtrList<quickstep::ParseWindow>();
+    $$->push_back($1);
+  }
+  | window_declaration_list window_declaration {
+    $$ = $1;
+    $$->push_back($2);
+  }
+
+window_declaration:
+  TOKEN_WINDOW any_name TOKEN_AS '(' window_definition ')' {
+    $$ = $5;
+    $$->setName($2);
+  } 
+
+window_definition:
+  opt_window_partition opt_window_order opt_window_frame {
+    $$ = new quickstep::ParseWindow(@1.first_line, @1.first_column, $1, $2, $3);
+  };
+
+opt_window_partition:
+  {
+    $$ = nullptr;
+  }
+  | TOKEN_PARTITION TOKEN_BY expression_list {
+    $$ = $3;
+  };
+
+opt_window_order:
+  {
+    $$ = nullptr;
+  }
+  | TOKEN_ORDER TOKEN_BY order_commalist  {
+    $$ = $3;
+  };
+
+opt_window_frame:
+  {
+    $$ = nullptr;
+  }
+  | frame_mode TOKEN_BETWEEN frame_preceding TOKEN_AND frame_following {
+    $$ = new quickstep::ParseFrameInfo(@1.first_line, @1.first_column, $1, $3->long_value(), $5->long_value());
+  };
+
+frame_mode:
+  TOKEN_ROWS {
+    $$ = true;
+  }
+  | TOKEN_RANGE {
+    $$ = false;
+  };
+
+frame_preceding:
+  TOKEN_UNSIGNED_NUMVAL TOKEN_PRECEDING
+  | TOKEN_UNBOUNDED TOKEN_PRECEDING { 
+    $$ = new quickstep::NumericParseLiteralValue(@1.first_line, @1.first_column, "-1");
+  }
+  | TOKEN_CURRENT TOKEN_ROW {
+    $$ = new quickstep::NumericParseLiteralValue(@1.first_line, @1.first_column, "0");
+  };
+
+frame_following:
+  TOKEN_UNSIGNED_NUMVAL TOKEN_FOLLOWING
+  | TOKEN_UNBOUNDED TOKEN_FOLLOWING {
+    $$ = new quickstep::NumericParseLiteralValue(@1.first_line, @1.first_column, "-1");
+  }
+  | TOKEN_CURRENT TOKEN_ROW {
+    $$ = new quickstep::NumericParseLiteralValue(@1.first_line, @1.first_column, "0");
+  };
+
 order_commalist:
   order_item {
     $$ = new quickstep::PtrList<quickstep::ParseOrderByItem>();
@@ -1505,6 +1621,14 @@ expression_base:
   | function_call {
     $$ = $1;
   }
+  | function_call TOKEN_OVER any_name {
+    $1->setWindowName($3);
+    $$ = $1;
+  }
+  | function_call TOKEN_OVER '(' window_definition ')' {
+    $1->setWindow($4);
+    $$ = $1;
+  }
   | extract_function {
     $$ = $1;
   }