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

[13/27] incubator-quickstep git commit: Implement parser and resolver for UNION and INTERSECT.

Implement parser and resolver for UNION and INTERSECT.


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

Branch: refs/heads/new-op
Commit: 5b7b5cb8446fb44c06a05238e9229fbe9f8644bb
Parents: 4432828
Author: TianrunLi <tl...@wisc.edu>
Authored: Tue Dec 20 09:23:24 2016 -0600
Committer: Tianrun <Ti...@node-0.tianrun-qv23700.quickstep-pg0.wisc.cloudlab.us>
Committed: Sat Apr 8 18:28:15 2017 -0600

----------------------------------------------------------------------
 parser/CMakeLists.txt                      |   10 +-
 parser/ParseSetOperation.hpp               |  142 +
 parser/ParseStatement.hpp                  |   41 +-
 parser/ParseSubqueryExpression.cpp         |    2 +-
 parser/ParseSubqueryExpression.hpp         |   14 +-
 parser/SqlLexer.lpp                        |    5 +-
 parser/SqlParser.ypp                       |   75 +-
 parser/preprocessed/SqlLexer_gen.cpp       | 1579 +++++-----
 parser/preprocessed/SqlLexer_gen.hpp       |  192 +-
 parser/preprocessed/SqlParser_gen.cpp      | 3548 ++++++++++++-----------
 parser/preprocessed/SqlParser_gen.hpp      |  232 +-
 parser/tests/Aggregate.test                |  334 ++-
 parser/tests/CMakeLists.txt                |    4 +
 parser/tests/Join.test                     |  242 +-
 parser/tests/Select.test                   | 2371 ++++++++-------
 parser/tests/SetOperation.test             |  235 ++
 parser/tests/TPCH.test                     | 3292 +++++++++++----------
 query_optimizer/logical/CMakeLists.txt     |    9 +
 query_optimizer/logical/LogicalType.hpp    |    1 +
 query_optimizer/logical/PatternMatcher.hpp |    2 +
 query_optimizer/logical/SetOperation.hpp   |  182 ++
 query_optimizer/resolver/CMakeLists.txt    |    1 +
 query_optimizer/resolver/Resolver.cpp      |  208 +-
 query_optimizer/resolver/Resolver.hpp      |   28 +-
 24 files changed, 6955 insertions(+), 5794 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/CMakeLists.txt b/parser/CMakeLists.txt
index b1bd870..b3ddf30 100644
--- a/parser/CMakeLists.txt
+++ b/parser/CMakeLists.txt
@@ -107,6 +107,7 @@ add_library(quickstep_parser_ParsePriority ../empty_src.cpp ParsePriority.hpp)
 add_library(quickstep_parser_ParseSample ParseSample.cpp ParseSample.hpp)
 add_library(quickstep_parser_ParseSelect ../empty_src.cpp ParseSelect.hpp)
 add_library(quickstep_parser_ParseSelectionClause ParseSelectionClause.cpp ParseSelectionClause.hpp)
+add_library(quickstep_parser_ParseSetOperation ../empty_src.cpp ParseSetOperation.hpp)
 add_library(quickstep_parser_ParseSimpleTableReference ParseSimpleTableReference.cpp ParseSimpleTableReference.hpp)
 add_library(quickstep_parser_ParseStatement ../empty_src.cpp ParseStatement.hpp)
 add_library(quickstep_parser_ParseString ParseString.cpp ParseString.hpp)
@@ -257,6 +258,10 @@ target_link_libraries(quickstep_parser_ParseSelectionClause
                       quickstep_parser_ParseTreeNode
                       quickstep_utility_Macros
                       quickstep_utility_PtrList)
+target_link_libraries(quickstep_parser_ParseSetOperation
+                      quickstep_parser_ParseTreeNode
+                      quickstep_utility_Macros
+                      quickstep_utility_PtrList)
 target_link_libraries(quickstep_parser_ParseSimpleTableReference
                       quickstep_parser_ParseSample
                       quickstep_parser_ParseString
@@ -274,6 +279,7 @@ target_link_libraries(quickstep_parser_ParseStatement
                       quickstep_parser_ParsePredicate
                       quickstep_parser_ParsePriority
                       quickstep_parser_ParseSelect
+                      quickstep_parser_ParseSetOperation
                       quickstep_parser_ParseString
                       quickstep_parser_ParseSubqueryTableReference
                       quickstep_parser_ParseTreeNode
@@ -293,7 +299,7 @@ target_link_libraries(quickstep_parser_ParseKeyValue
 target_link_libraries(quickstep_parser_ParseSubqueryExpression
                       glog
                       quickstep_parser_ParseExpression
-                      quickstep_parser_ParseSelect
+                      quickstep_parser_ParseSetOperation
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_parser_ParseSubqueryTableReference
                       quickstep_parser_ParseSubqueryExpression
@@ -347,6 +353,7 @@ target_link_libraries(quickstep_parser_SqlParser
                       quickstep_parser_ParseSample
                       quickstep_parser_ParseSelect
                       quickstep_parser_ParseSelectionClause
+                      quickstep_parser_ParseSetOperation
                       quickstep_parser_ParseSimpleTableReference
                       quickstep_parser_ParseStatement
                       quickstep_parser_ParseString
@@ -426,6 +433,7 @@ target_link_libraries(quickstep_parser
                       quickstep_parser_ParseSample
                       quickstep_parser_ParseSelect
                       quickstep_parser_ParseSelectionClause
+                      quickstep_parser_ParseSetOperation
                       quickstep_parser_ParseSimpleTableReference
                       quickstep_parser_ParseStatement
                       quickstep_parser_ParseString

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/ParseSetOperation.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseSetOperation.hpp b/parser/ParseSetOperation.hpp
new file mode 100644
index 0000000..93e4c1b
--- /dev/null
+++ b/parser/ParseSetOperation.hpp
@@ -0,0 +1,142 @@
+/**
+ * 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_SET_OPERATION_HPP_
+#define QUICKSTEP_PARSER_PARSE_SET_OPERATION_HPP_
+
+#include <string>
+#include <vector>
+
+#include "parser/ParseTreeNode.hpp"
+#include "utility/Macros.hpp"
+#include "utility/PtrList.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+
+/** \addtogroup Parser
+ *  @{
+ */
+
+/**
+ * @brief A parsed representation of set operations.
+ */
+class ParseSetOperation : public ParseTreeNode {
+ public:
+  /**
+   * @brief The possible types of set operations.
+   */
+  enum SetOperationType {
+    kIntersect = 0,
+    kSelect,
+    kUnion,
+    kUnionAll
+  };
+
+  /**
+   * @brief Constructor.
+   *
+   * @param line_number Line number of the set operation token in the SQL statement.
+   * @param column_number Column number of the set operation toke in the SQL statement.
+   * @param set_operation The set operation type.
+   */
+  ParseSetOperation(const int line_number,
+                    const int column_number,
+                    const SetOperationType set_operation_type)
+      : ParseTreeNode(line_number, column_number),
+        set_operation_type_(set_operation_type) {
+  }
+
+  /**
+   * @brief Destructor.
+   */
+  ~ParseSetOperation() override {}
+
+  std::string getName() const override {
+    return "SetOperation";
+  }
+
+  /**
+   * @return The set operation type.
+   */
+  SetOperationType getOperationType() const {
+    return set_operation_type_;
+  }
+
+  /**
+   * @return The operands of the set operation.
+   */
+  const PtrList<ParseTreeNode>& operands() const {
+    return operands_;
+  }
+
+  /**
+   * @brief Add an operand for the set operation.
+   *
+   * @param operand The operand.
+   */
+  void addOperand(ParseTreeNode *operand) {
+    operands_.push_back(operand);
+  }
+
+ 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("set_operation_type");
+    switch (set_operation_type_) {
+      case kIntersect:
+        inline_field_values->push_back("Intersect");
+        break;
+      case kSelect:
+        inline_field_values->push_back("Select");
+        break;
+      case kUnion:
+        inline_field_values->push_back("Union");
+        break;
+      case kUnionAll:
+        inline_field_values->push_back("UnionAll");
+        break;
+      default:
+        LOG(FATAL) << "Unknown set operation type.";
+    }
+
+    container_child_field_names->push_back("children");
+    container_child_fields->emplace_back();
+    for (const ParseTreeNode &child : operands_) {
+      container_child_fields->back().push_back(&child);
+    }
+  }
+
+ private:
+  PtrList<ParseTreeNode> operands_;
+  const SetOperationType set_operation_type_;
+
+  DISALLOW_COPY_AND_ASSIGN(ParseSetOperation);
+};
+
+/** @} */
+
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_PARSER_PARSE_SET_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/ParseStatement.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseStatement.hpp b/parser/ParseStatement.hpp
index cb5a1b5..d876163 100644
--- a/parser/ParseStatement.hpp
+++ b/parser/ParseStatement.hpp
@@ -34,6 +34,7 @@
 #include "parser/ParsePredicate.hpp"
 #include "parser/ParsePriority.hpp"
 #include "parser/ParseSelect.hpp"
+#include "parser/ParseSetOperation.hpp"
 #include "parser/ParseString.hpp"
 #include "parser/ParseSubqueryTableReference.hpp"
 #include "parser/ParseTreeNode.hpp"
@@ -62,7 +63,7 @@ class ParseStatement : public ParseTreeNode {
     kCreateTable,
     kCreateIndex,
     kDropTable,
-    kSelect,
+    kSetOperation,
     kInsert,
     kCopyFrom,
     kUpdate,
@@ -479,9 +480,9 @@ class ParseStatementDropTable : public ParseStatement {
 };
 
 /**
- * @brief The parsed representation of a SELECT statement.
+ * @brief The parsed representation of an UNION/INTERSECT/SELECT statement.
  **/
-class ParseStatementSelect : public ParseStatement {
+class ParseStatementSetOperation : public ParseStatement {
  public:
   /**
    * @brief Constructor.
@@ -489,18 +490,18 @@ class ParseStatementSelect : public ParseStatement {
    *
    * @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 select_query The top-level SELECT query.
+   * @param set_operation_query The top level set operation query
    * @param with_clause The WITH clause of common table query expressions.
    * @param priority_clause The PRIORITY clause of this query. If not valid or
    *        not present, this is NULL.
    **/
-  ParseStatementSelect(const int line_number,
-                       const int column_number,
-                       ParseSelect *select_query,
-                       PtrVector<ParseSubqueryTableReference> *with_clause,
-                       ParsePriority *priority_clause)
+  ParseStatementSetOperation(const int line_number,
+                             const int column_number,
+                             ParseSetOperation *set_operation_query,
+                             PtrVector<ParseSubqueryTableReference> *with_clause,
+                             ParsePriority *priority_clause)
       : ParseStatement(line_number, column_number),
-        select_query_(select_query),
+        set_operation_query_(set_operation_query),
         with_clause_(with_clause),
         priority_clause_(priority_clause) {
   }
@@ -508,20 +509,20 @@ class ParseStatementSelect : public ParseStatement {
   /**
    * @brief Destructor.
    */
-  ~ParseStatementSelect() override {
+  ~ParseStatementSetOperation() override {
   }
 
   StatementType getStatementType() const override {
-    return kSelect;
+    return kSetOperation;
   }
 
-  std::string getName() const override { return "SelectStatement"; }
+  std::string getName() const override { return "SetOperationStatement"; }
 
   /**
-   * @return Gets the top-level SELECT query.
+   * @return Gets the top-level set operation query.
    */
-  const ParseSelect* select_query() const {
-    return select_query_.get();
+  const ParseSetOperation* set_operation_query() const {
+    return set_operation_query_.get();
   }
 
   /**
@@ -549,8 +550,8 @@ class ParseStatementSelect : public ParseStatement {
       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 {
-    non_container_child_field_names->push_back("select_query");
-    non_container_child_fields->push_back(select_query_.get());
+    non_container_child_field_names->push_back("set_operation_query");
+    non_container_child_fields->push_back(set_operation_query_.get());
 
     if (with_clause_ != nullptr && !with_clause_->empty()) {
       container_child_field_names->push_back("with_clause");
@@ -567,11 +568,11 @@ class ParseStatementSelect : public ParseStatement {
   }
 
  private:
-  std::unique_ptr<ParseSelect> select_query_;
+  std::unique_ptr<ParseSetOperation> set_operation_query_;
   std::unique_ptr<PtrVector<ParseSubqueryTableReference>> with_clause_;
   std::unique_ptr<ParsePriority> priority_clause_;
 
-  DISALLOW_COPY_AND_ASSIGN(ParseStatementSelect);
+  DISALLOW_COPY_AND_ASSIGN(ParseStatementSetOperation);
 };
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/ParseSubqueryExpression.cpp
----------------------------------------------------------------------
diff --git a/parser/ParseSubqueryExpression.cpp b/parser/ParseSubqueryExpression.cpp
index 991e00c..02fe342 100644
--- a/parser/ParseSubqueryExpression.cpp
+++ b/parser/ParseSubqueryExpression.cpp
@@ -41,7 +41,7 @@ void ParseSubqueryExpression::getFieldStringItems(
     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(query_.get());
+  non_container_child_fields->push_back(set_operation_.get());
 }
 
 }  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/ParseSubqueryExpression.hpp
----------------------------------------------------------------------
diff --git a/parser/ParseSubqueryExpression.hpp b/parser/ParseSubqueryExpression.hpp
index c7f9ccf..e0c0ed6 100644
--- a/parser/ParseSubqueryExpression.hpp
+++ b/parser/ParseSubqueryExpression.hpp
@@ -25,7 +25,7 @@
 #include <vector>
 
 #include "parser/ParseExpression.hpp"
-#include "parser/ParseSelect.hpp"
+#include "parser/ParseSetOperation.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -46,13 +46,13 @@ class ParseSubqueryExpression : public ParseExpression {
    *
    * @param line_number The line number of the beginning of the subquery expression.
    * @param column_number The column number of the beginning of the subquery expression.
-   * @param query The SELECT subquery.
+   * @param set_operation The set operation subquery.
    */
   ParseSubqueryExpression(const int line_number,
                           const int column_number,
-                          ParseSelect *query)
+                          ParseSetOperation *set_operation)
       : ParseExpression(line_number, column_number),
-        query_(query) {
+        set_operation_(set_operation) {
   }
 
   ~ParseSubqueryExpression() override {
@@ -63,9 +63,9 @@ class ParseSubqueryExpression : public ParseExpression {
   }
 
   /**
-   * @return The SELECT subquery.
+   * @return The set operation subquery.
    */
-  const ParseSelect* query() const { return query_.get(); }
+  const ParseSetOperation* set_operation() const { return set_operation_.get(); }
 
   std::string getName() const override { return "SubqueryExpression"; }
 
@@ -81,7 +81,7 @@ class ParseSubqueryExpression : public ParseExpression {
       std::vector<std::vector<const ParseTreeNode*>> *container_child_fields) const override;
 
  private:
-  std::unique_ptr<ParseSelect> query_;
+  std::unique_ptr<ParseSetOperation> set_operation_;
   DISALLOW_COPY_AND_ASSIGN(ParseSubqueryExpression);
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/SqlLexer.lpp
----------------------------------------------------------------------
diff --git a/parser/SqlLexer.lpp b/parser/SqlLexer.lpp
index 0953dc6..b5c2b3e 100644
--- a/parser/SqlLexer.lpp
+++ b/parser/SqlLexer.lpp
@@ -70,6 +70,7 @@ class ParseSelectionClause;
 class ParseSelectionItem;
 class ParseSelectionItemScalar;
 class ParseSelectionList;
+class ParseSetOperation;
 class ParseSimpleTableReference;
 class ParseSimpleWhenClause;
 class ParseStringKeyLiteralValues;
@@ -81,7 +82,7 @@ class ParseStatementDropTable;
 class ParseStatementInsert;
 class ParseStatementInsertTuple;
 class ParseStatementInsertSelection;
-class ParseStatementSelect;
+class ParseStatementSetOperation;
 class ParseStatementQuit;
 class ParseStatementUpdate;
 class ParseSubqueryExpression;
@@ -228,6 +229,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "insert"           return TOKEN_INSERT;
   "int"              return TOKEN_INTEGER;
   "integer"          return TOKEN_INTEGER;
+  "intersect"        return TOKEN_INTERSECT;
   "interval"         return TOKEN_INTERVAL;
   "into"             return TOKEN_INTO;
   "is"               return TOKEN_IS;
@@ -277,6 +279,7 @@ unsigned_numeric_literal {exact_numeric_literal}|{approximate_numeric_literal}
   "true"             return TOKEN_TRUE;
   "tuplesample"      return TOKEN_TUPLESAMPLE;
   "unbounded"        return TOKEN_UNBOUNDED;
+  "union"            return TOKEN_UNION;
   "unique"           return TOKEN_UNIQUE;
   "update"           return TOKEN_UPDATE;
   "using"            return TOKEN_USING;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/SqlParser.ypp
----------------------------------------------------------------------
diff --git a/parser/SqlParser.ypp b/parser/SqlParser.ypp
index 29b69d7..5db2171 100644
--- a/parser/SqlParser.ypp
+++ b/parser/SqlParser.ypp
@@ -90,6 +90,7 @@ typedef struct YYLTYPE {
 #include "parser/ParseSample.hpp"
 #include "parser/ParseSelect.hpp"
 #include "parser/ParseSelectionClause.hpp"
+#include "parser/ParseSetOperation.hpp"
 #include "parser/ParseSimpleTableReference.hpp"
 #include "parser/ParseStatement.hpp"
 #include "parser/ParseString.hpp"
@@ -148,6 +149,8 @@ typedef void* yyscan_t;
   quickstep::ParseSelectionItem *selection_item_;
   quickstep::ParseSelectionList *selection_list_;
 
+  quickstep::ParseSetOperation *set_operation_;
+
   quickstep::ParseTableReference *table_reference_;
   quickstep::PtrList<quickstep::ParseTableReference> *table_reference_list_;
   quickstep::ParseTableReferenceSignature *table_reference_signature_;
@@ -175,7 +178,7 @@ typedef void* yyscan_t;
   quickstep::PtrVector<quickstep::ParseString> *command_argument_list_;
 
   quickstep::ParseStatement *statement_;
-  quickstep::ParseStatementSelect *select_statement_;
+  quickstep::ParseStatementSetOperation *set_operation_statement_;
   quickstep::ParseStatementUpdate *update_statement_;
   quickstep::ParseStatementInsert *insert_statement_;
   quickstep::ParseStatementDelete *delete_statement_;
@@ -242,6 +245,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %left '*' '/'
 %right UNARY_PLUS UNARY_MINUS
 %left '.'
+%left TOKEN_ALL TOKEN_UNION TOKEN_INTERSECT
 
 %token TOKEN_ADD;
 %token TOKEN_ALL;
@@ -298,6 +302,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_INNER;
 %token TOKEN_INSERT;
 %token TOKEN_INTEGER;
+%token TOKEN_INTERSECT;
 %token TOKEN_INTERVAL;
 %token TOKEN_INTO;
 %token TOKEN_JOIN;
@@ -345,6 +350,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %token TOKEN_TRUE;
 %token TOKEN_TUPLESAMPLE;
 %token TOKEN_UNBOUNDED;
+%token TOKEN_UNION;
 %token TOKEN_UNIQUE;
 %token TOKEN_UPDATE;
 %token TOKEN_USING;
@@ -368,6 +374,7 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 %type <boolean_value_>
   boolean_value
   frame_mode
+  opt_all_distinct
 
 %type <literal_value_>
   literal_value
@@ -485,8 +492,12 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
   sql_statement
   alter_table_statement
 
-%type <select_statement_>
-  select_statement
+%type <set_operation_statement_>
+  set_operation_statement
+
+%type <set_operation_>
+  set_operation_union
+  set_operation_intersect
 
 %type <select_query_>
   select_query
@@ -605,7 +616,6 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
 
 /*
 %type <int_val>
-  opt_all_distinct      // unimplemented
   table_constraint_def  // unimplemented
   table_constraint_def_commalist        // unimplemented
   opt_table_constraint_def_commalist    // unimplemented
@@ -676,7 +686,7 @@ sql_statement:
   | quit_statement {
     $$ = $1;
   }
-  | select_statement {
+  | set_operation_statement {
     $$ = $1;
   }
   | update_statement {
@@ -1146,13 +1156,16 @@ assignment_item:
     $$ = new quickstep::ParseAssignment(@1.first_line, @1.first_column, $1, $3);
   };
 
-/* Select Queries */
-select_statement:
-  select_query opt_priority_clause {
-    $$ = new quickstep::ParseStatementSelect(@1.first_line, @1.first_column, $1, nullptr, $2);
+/**
+ * Set Operation Queries.
+ * Select Queries are now included in set operations.
+ **/
+set_operation_statement:
+  set_operation_union opt_priority_clause {
+    $$ = new quickstep::ParseStatementSetOperation(@1.first_line, @1.first_column, $1, nullptr, $2);
   }
-  | with_clause select_query opt_priority_clause {
-    $$ = new quickstep::ParseStatementSelect(@1.first_line, @1.first_column, $2, $1, $3);
+  | with_clause set_operation_union opt_priority_clause {
+    $$ = new quickstep::ParseStatementSetOperation(@1.first_line, @1.first_column, $2, $1, $3);
   };
 
 opt_priority_clause:
@@ -1198,23 +1211,49 @@ with_list_element:
     $$->set_table_reference_signature($1);
   };
 
+set_operation_union:
+  set_operation_union TOKEN_UNION opt_all_distinct set_operation_intersect {
+    if ($3) {
+      $$ = new quickstep::ParseSetOperation(@1.first_line, @1.first_column, quickstep::ParseSetOperation::kUnion);
+    } else {
+      $$ = new quickstep::ParseSetOperation(@1.first_line, @1.first_column, quickstep::ParseSetOperation::kUnionAll);
+    }
+    $$->addOperand($1);
+    $$->addOperand($4);
+  }
+  | set_operation_intersect {
+    $$ = $1;
+  }
+
+set_operation_intersect:
+  set_operation_intersect TOKEN_INTERSECT select_query {
+    $$ = new quickstep::ParseSetOperation(@1.first_line, @1.first_column, quickstep::ParseSetOperation::kIntersect);
+    quickstep::ParseSetOperation *op = new quickstep::ParseSetOperation(
+        @3.first_line, @3.first_column, quickstep::ParseSetOperation::kSelect);
+    op->addOperand($3);
+    $$->addOperand($1);
+    $$->addOperand(op);
+  }
+  | select_query {
+    $$ = new quickstep::ParseSetOperation(@1.first_line, @1.first_column, quickstep::ParseSetOperation::kSelect);
+    $$->addOperand($1);
+  }
+
 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 opt_window_clause {
     $$ = new quickstep::ParseSelect(@1.first_line, @1.first_column, $3, $4, $5, $6, $7, $8, $9, $10);
-  };
+  }
 
 opt_all_distinct:
   {
-    /* $$ = nullptr; */
+    $$ = true;  // Distinct
   }
   | TOKEN_ALL {
-    NotSupported(&@1, yyscanner, "ALL in selection");
-    YYERROR;
+    $$ = false;  // All
   }
   | TOKEN_DISTINCT {
-    NotSupported(&@1, yyscanner, "DISTINCT in selection");
-    YYERROR;
+    $$ = true;  // Distinct
   };
 
 selection:
@@ -1252,7 +1291,7 @@ from_clause:
   };
 
 subquery_expression:
-  '(' select_query ')' {
+  '(' set_operation_union ')' {
     $$ = new quickstep::ParseSubqueryExpression(@1.first_line, @1.first_column, $2);
   };