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/09 00:31:38 UTC

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

Repository: incubator-quickstep
Updated Branches:
  refs/heads/master 4432828fd -> 5b7b5cb84


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/query_optimizer/logical/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/CMakeLists.txt b/query_optimizer/logical/CMakeLists.txt
index 8aca550..4480e0b 100644
--- a/query_optimizer/logical/CMakeLists.txt
+++ b/query_optimizer/logical/CMakeLists.txt
@@ -35,6 +35,7 @@ add_library(quickstep_queryoptimizer_logical_NestedLoopsJoin ../../empty_src.cpp
 add_library(quickstep_queryoptimizer_logical_PatternMatcher ../../empty_src.cpp PatternMatcher.hpp)
 add_library(quickstep_queryoptimizer_logical_Project Project.cpp Project.hpp)
 add_library(quickstep_queryoptimizer_logical_Sample Sample.cpp Sample.hpp)
+add_library(quickstep_queryoptimizer_logical_SetOperation ../../empty_src.cpp SetOperation.hpp)
 add_library(quickstep_queryoptimizer_logical_SharedSubplanReference
             SharedSubplanReference.cpp
             SharedSubplanReference.hpp)
@@ -204,6 +205,13 @@ target_link_libraries(quickstep_queryoptimizer_logical_Sample
                       quickstep_queryoptimizer_logical_LogicalType
                       quickstep_utility_Cast
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_logical_SetOperation
+                      quickstep_queryoptimizer_OptimizerTree
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_logical_Logical
+                      quickstep_queryoptimizer_logical_LogicalType
+                      quickstep_utility_Cast
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_logical_SharedSubplanReference
                       glog
                       quickstep_queryoptimizer_OptimizerTree
@@ -298,6 +306,7 @@ target_link_libraries(quickstep_queryoptimizer_logical
                       quickstep_queryoptimizer_logical_PatternMatcher
                       quickstep_queryoptimizer_logical_Project
                       quickstep_queryoptimizer_logical_Sample
+                      quickstep_queryoptimizer_logical_SetOperation
                       quickstep_queryoptimizer_logical_SharedSubplanReference
                       quickstep_queryoptimizer_logical_Sort
                       quickstep_queryoptimizer_logical_TableGenerator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/query_optimizer/logical/LogicalType.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/LogicalType.hpp b/query_optimizer/logical/LogicalType.hpp
index d1011b0..21ffdca 100644
--- a/query_optimizer/logical/LogicalType.hpp
+++ b/query_optimizer/logical/LogicalType.hpp
@@ -46,6 +46,7 @@ enum class LogicalType {
   kNestedLoopsJoin,
   kProject,
   kSample,
+  kSetOperation,
   kSharedSubplanReference,
   kSort,
   kTableGenerator,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/query_optimizer/logical/PatternMatcher.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/PatternMatcher.hpp b/query_optimizer/logical/PatternMatcher.hpp
index b931ace..23c2872 100644
--- a/query_optimizer/logical/PatternMatcher.hpp
+++ b/query_optimizer/logical/PatternMatcher.hpp
@@ -42,6 +42,7 @@ class Join;
 class MultiwayCartesianJoin;
 class NestedLoopsJoin;
 class Project;
+class SetOperation;
 class SharedSubplanReference;
 class Sort;
 class TableReference;
@@ -128,6 +129,7 @@ using SomeJoin = SomeLogicalNode<Join,
 using SomeMultiwayCartesianJoin = SomeLogicalNode<MultiwayCartesianJoin, LogicalType::kMultiwayCartesianJoin>;
 using SomeNestedLoopsJoin = SomeLogicalNode<NestedLoopsJoin, LogicalType::kNestedLoopsJoin>;
 using SomeProject = SomeLogicalNode<Project, LogicalType::kProject>;
+using SomeSetOperation = SomeLogicalNode<SetOperation, LogicalType::kSetOperation>;
 using SomeSharedSubplanReference = SomeLogicalNode<SharedSubplanReference, LogicalType::kSharedSubplanReference>;
 using SomeSort = SomeLogicalNode<Sort, LogicalType::kSort>;
 using SomeTableReference = SomeLogicalNode<TableReference, LogicalType::kTableReference>;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/query_optimizer/logical/SetOperation.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/logical/SetOperation.hpp b/query_optimizer/logical/SetOperation.hpp
new file mode 100644
index 0000000..7e45231
--- /dev/null
+++ b/query_optimizer/logical/SetOperation.hpp
@@ -0,0 +1,182 @@
+/**
+ * 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_QUERY_OPTIMIZER_LOGICAL_SET_OPERATION_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_SET_OPERATION_HPP_
+
+#include <string>
+#include <type_traits>
+#include <vector>
+
+#include "query_optimizer/OptimizerTree.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/logical/Logical.hpp"
+#include "query_optimizer/logical/LogicalType.hpp"
+#include "utility/Cast.hpp"
+#include "utility/Macros.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+namespace logical {
+
+/** \addtogroup OptimizerLogical
+ *  @{
+ */
+
+class SetOperation;
+typedef std::shared_ptr<const SetOperation> SetOperationPtr;
+
+/**
+ * @brief Base class for logical set operation operator that combines tuples
+ *        from two or more relations.
+ */
+class SetOperation : public Logical {
+ public:
+  enum SetOperationType {
+    kIntersect = 0,
+    kUnion,
+    kUnionAll
+  };
+
+  /**
+   * @brief Destructor.
+   */
+  ~SetOperation() override {}
+
+  LogicalType getLogicalType() const override {
+    return LogicalType::kSetOperation;
+  }
+
+  std::string getName() const override {
+    switch (set_operation_type_) {
+      case SetOperationType::kIntersect:
+        return "Intersect";
+      case SetOperationType::kUnion:
+        return "Union";
+      case SetOperationType::kUnionAll:
+        return "UnionAll";
+      default:
+        LOG(FATAL) << "Invalid set operation type: "
+                   << static_cast<typename std::underlying_type<SetOperationType>::type>(set_operation_type_);
+    }
+  }
+
+  /**
+   * @return The set operation type
+   */
+  SetOperationType getSetOperationType() const {
+    return set_operation_type_;
+  }
+
+  /**
+   * @return The operands for set operation.
+   */
+  const std::vector<LogicalPtr>& getOperands() const {
+    return operands_;
+  }
+
+  std::vector<expressions::AttributeReferencePtr> getOutputAttributes() const override {
+    return project_attributes_;
+  }
+
+  std::vector<expressions::AttributeReferencePtr> getReferencedAttributes() const override {
+    std::vector<expressions::AttributeReferencePtr> referenced_attributes;
+    for (const auto &operand : operands_) {
+      const std::vector<expressions::AttributeReferencePtr> reference =
+          operand->getOutputAttributes();
+      referenced_attributes.insert(referenced_attributes.end(),
+                                   reference.begin(),
+                                   reference.end());
+    }
+    return referenced_attributes;
+  }
+
+  LogicalPtr copyWithNewChildren(
+      const std::vector<LogicalPtr> &new_children) const override {
+    DCHECK_EQ(new_children.size(), children().size());
+    return SetOperation::Create(set_operation_type_, new_children, project_attributes_);
+  }
+
+  /**
+   * @brief Create a logical set operation operator that combines two or more
+   *        relations.
+   *
+   * @param type The type of set operation.
+   * @param operands The operands of the set operation.
+   * @param project_attributes The project attributes of this set operation.
+   */
+  static SetOperationPtr Create(
+      const SetOperationType type,
+      const std::vector<LogicalPtr> &operands,
+      const std::vector<expressions::AttributeReferencePtr> &project_attributes) {
+    DCHECK_GT(operands.size(), 1u);
+    return SetOperationPtr(new SetOperation(type, operands, project_attributes));
+  }
+
+ 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<OptimizerTreeBaseNodePtr> *non_container_child_fields,
+      std::vector<std::string> *container_child_field_names,
+      std::vector<std::vector<OptimizerTreeBaseNodePtr>> *container_child_fields) const override {
+    inline_field_names->push_back("set_operation_type");
+    inline_field_values->push_back(getName());
+
+    container_child_field_names->push_back("operands");
+    container_child_fields->push_back(
+        CastSharedPtrVector<OptimizerTreeBase>(operands_));
+
+    container_child_field_names->push_back("project_attributes");
+    container_child_fields->push_back(
+        CastSharedPtrVector<OptimizerTreeBase>(project_attributes_));
+  }
+
+ private:
+  /**
+   * @brief Constructor.
+   */
+  SetOperation(const SetOperationType set_operation_type,
+               const std::vector<LogicalPtr> &operands,
+               const std::vector<expressions::AttributeReferencePtr> &project_attributes)
+    : set_operation_type_(set_operation_type),
+      operands_(operands),
+      project_attributes_(project_attributes) {
+    for (const LogicalPtr &operand : operands) {
+      addChild(operand);
+    }
+  }
+
+  const SetOperationType set_operation_type_;
+  const std::vector<LogicalPtr> operands_;
+  const std::vector<expressions::AttributeReferencePtr> project_attributes_;
+
+  DISALLOW_COPY_AND_ASSIGN(SetOperation);
+};
+
+/** @} */
+
+}  // namespace logical
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_LOGICAL_SET_OPERATION_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/query_optimizer/resolver/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/CMakeLists.txt b/query_optimizer/resolver/CMakeLists.txt
index a34273e..4e364a6 100644
--- a/query_optimizer/resolver/CMakeLists.txt
+++ b/query_optimizer/resolver/CMakeLists.txt
@@ -109,6 +109,7 @@ target_link_libraries(quickstep_queryoptimizer_resolver_Resolver
                       quickstep_queryoptimizer_logical_MultiwayCartesianJoin
                       quickstep_queryoptimizer_logical_Project
                       quickstep_queryoptimizer_logical_Sample
+                      quickstep_queryoptimizer_logical_SetOperation
                       quickstep_queryoptimizer_logical_SharedSubplanReference
                       quickstep_queryoptimizer_logical_Sort
                       quickstep_queryoptimizer_logical_TableGenerator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/query_optimizer/resolver/Resolver.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.cpp b/query_optimizer/resolver/Resolver.cpp
index ed465e5..2d3a06b 100644
--- a/query_optimizer/resolver/Resolver.cpp
+++ b/query_optimizer/resolver/Resolver.cpp
@@ -104,6 +104,7 @@
 #include "query_optimizer/logical/MultiwayCartesianJoin.hpp"
 #include "query_optimizer/logical/Project.hpp"
 #include "query_optimizer/logical/Sample.hpp"
+#include "query_optimizer/logical/SetOperation.hpp"
 #include "query_optimizer/logical/SharedSubplanReference.hpp"
 #include "query_optimizer/logical/Sort.hpp"
 #include "query_optimizer/logical/TableGenerator.hpp"
@@ -372,23 +373,23 @@ L::LogicalPtr Resolver::resolve(const ParseStatement &parse_query) {
       }
       break;
     }
-    case ParseStatement::kSelect: {
-      const ParseStatementSelect &select_statement =
-          static_cast<const ParseStatementSelect&>(parse_query);
-      if (select_statement.with_clause() != nullptr) {
-        resolveWithClause(*select_statement.with_clause());
+    case ParseStatement::kSetOperation: {
+      const ParseStatementSetOperation &set_operation_statement =
+          static_cast<const ParseStatementSetOperation&>(parse_query);
+      if (set_operation_statement.with_clause() != nullptr) {
+        resolveWithClause(*set_operation_statement.with_clause());
       }
       logical_plan_ =
-          resolveSelect(*select_statement.select_query(),
-                        "" /* select_name */,
-                        nullptr /* No Type hints */,
-                        nullptr /* parent_resolver */);
-      if (select_statement.with_clause() != nullptr) {
+          resolveSetOperation(*set_operation_statement.set_operation_query(),
+                              "" /* set_operation_name */,
+                              nullptr /* type_hints */,
+                              nullptr /* parent_resolver */);
+      if (set_operation_statement.with_clause() != nullptr) {
         // Report an error if there is a WITH query that is not actually used.
         if (!with_queries_info_.unreferenced_query_indexes.empty()) {
           int unreferenced_with_query_index = *with_queries_info_.unreferenced_query_indexes.begin();
           const ParseSubqueryTableReference &unreferenced_with_query =
-              (*select_statement.with_clause())[unreferenced_with_query_index];
+              (*set_operation_statement.with_clause())[unreferenced_with_query_index];
           THROW_SQL_ERROR_AT(&unreferenced_with_query)
               << "WITH query "
               << unreferenced_with_query.table_reference_signature()->table_alias()->value()
@@ -1350,16 +1351,173 @@ L::LogicalPtr Resolver::resolveSelect(
   return logical_plan;
 }
 
+L::LogicalPtr Resolver::resolveSetOperations(
+    const ParseSetOperation &parse_set_operations,
+    const std::string &set_operation_name,
+    const std::vector<const Type*> *type_hints,
+    const NameResolver *parent_resolver) {
+  std::vector<const ParseSetOperation*> operands;
+  CollapseSetOperation(parse_set_operations, parse_set_operations, &operands);
+
+  DCHECK_LT(1u, operands.size());
+  std::vector<L::LogicalPtr> resolved_operations;
+  std::vector<std::vector<E::AttributeReferencePtr>> attribute_matrix;
+
+  // Resolve the first operation, and get the output attributes.
+  auto iter = operands.begin();
+  const ParseSetOperation &operation = static_cast<const ParseSetOperation&>(**iter);
+  L::LogicalPtr operation_logical =
+      resolveSetOperation(operation, set_operation_name, type_hints, parent_resolver);
+  const std::vector<E::AttributeReferencePtr> operation_attributes =
+      operation_logical->getOutputAttributes();
+  attribute_matrix.push_back(operation_attributes);
+  resolved_operations.push_back(operation_logical);
+
+  // Resolve the rest operations, and check the size of output attributes.
+  for (++iter; iter != operands.end(); ++iter) {
+    const ParseSetOperation &current_operation =
+        static_cast<const ParseSetOperation&>(**iter);
+    L::LogicalPtr current_logical =
+        resolveSetOperation(current_operation, set_operation_name, type_hints, parent_resolver);
+    attribute_matrix.emplace_back(current_logical->getOutputAttributes());
+
+    // Check output attributes size.
+    // Detailed type check and type cast will perform later.
+    if (attribute_matrix.back().size() != operation_attributes.size()) {
+      THROW_SQL_ERROR_AT(&current_operation)
+          << "Can not perform " << parse_set_operations.getName()
+          << "opeartion between " << std::to_string(attribute_matrix.back().size())
+          << "and " << std::to_string(operation_attributes.size())
+          << "columns";
+    }
+
+    resolved_operations.push_back(current_logical);
+  }
+
+  // Get the possible output attributes that the attributes of all operands can cast to.
+  std::vector<E::AttributeReferencePtr> possible_attributes;
+  for (std::size_t aid = 0; aid < operation_attributes.size(); ++aid) {
+    E::AttributeReferencePtr possible_attribute = attribute_matrix[0][aid];
+    for (std::size_t opid = 1; opid < resolved_operations.size(); ++opid) {
+      const Type &current_type = attribute_matrix[opid][aid]->getValueType();
+      const Type &possible_type = possible_attribute->getValueType();
+      if (!possible_type.equals(current_type)) {
+        if (possible_type.getSuperTypeID() == Type::SuperTypeID::kNumeric &&
+            current_type.getSuperTypeID() == Type::SuperTypeID::kNumeric) {
+          if (possible_type.isSafelyCoercibleFrom(current_type)) {
+            // Cast current_type to possible_type.
+            // Possible_attribute remain the same, nothing needs to change.
+          } else if (current_type.isSafelyCoercibleFrom(possible_type)) {
+            // Cast possible_type to current_type.
+            possible_attribute = attribute_matrix[opid][aid];
+          } else {
+            // Can not cast between possible_type and current_type.
+            // Throw an SQL error.
+            THROW_SQL_ERROR_AT(&parse_set_operations)
+                << "There is not a safely coerce between "
+                << current_type.getName()
+                << "and " << possible_type.getName();
+          }
+        } else {
+          THROW_SQL_ERROR_AT(&parse_set_operations)
+              << "Does not support cast operation between non-numeric types"
+              << current_type.getName()
+              << "and " << possible_type.getName();
+        }
+      }
+    }
+    possible_attributes.push_back(possible_attribute);
+  }
+
+  for (std::size_t opid = 0; opid < operation_attributes.size(); ++opid) {
+    // Generate a cast operation if needed.
+    std::vector<E::NamedExpressionPtr> cast_expressions;
+    for (std::size_t aid = 0; aid < operation_attributes.size(); ++aid) {
+      const E::AttributeReferencePtr current_attr = attribute_matrix[opid][aid];
+      const Type &current_type = current_attr->getValueType();
+      const Type &possible_type = possible_attributes[aid]->getValueType();
+      if (possible_type.equals(current_type)) {
+        cast_expressions.emplace_back(current_attr);
+      } else {
+        cast_expressions.emplace_back(
+            E::Alias::Create(context_->nextExprId(),
+                             E::Cast::Create(current_attr, possible_type),
+                             current_attr->attribute_name(),
+                             current_attr->attribute_alias()));
+      }
+    }
+    resolved_operations[opid] = L::Project::Create(resolved_operations[opid], cast_expressions);
+  }
+
+  std::vector<E::AttributeReferencePtr> output_attributes;
+  for (const auto &attr : possible_attributes) {
+    output_attributes.emplace_back(
+        E::AttributeReference::Create(context_->nextExprId(),
+                                      attr->attribute_name(),
+                                      attr->attribute_alias(),
+                                      "" /* relation_name */,
+                                      attr->getValueType(),
+                                      attr->scope()));
+  }
+
+  // Generate the set operation logical node.
+  switch (parse_set_operations.getOperationType()) {
+    case ParseSetOperation::kIntersect:
+      return L::SetOperation::Create(
+          L::SetOperation::kIntersect, resolved_operations, output_attributes);
+    case ParseSetOperation::kUnion:
+      return L::SetOperation::Create(
+          L::SetOperation::kUnion, resolved_operations, output_attributes);
+    case ParseSetOperation::kUnionAll:
+      return L::SetOperation::Create(
+          L::SetOperation::kUnionAll, resolved_operations, output_attributes);
+    default:
+      LOG(FATAL) << "Unknown operation: " << parse_set_operations.toString();
+      return nullptr;
+  }
+}
+
+L::LogicalPtr Resolver::resolveSetOperation(
+    const ParseSetOperation &set_operation_query,
+    const std::string &set_operation_name,
+    const std::vector<const Type*> *type_hints,
+    const NameResolver *parent_resolver) {
+  switch (set_operation_query.getOperationType()) {
+    case ParseSetOperation::kIntersect:
+    case ParseSetOperation::kUnion:
+    case ParseSetOperation::kUnionAll: {
+      return resolveSetOperations(set_operation_query,
+                                  set_operation_name,
+                                  type_hints,
+                                  parent_resolver);
+    }
+    case ParseSetOperation::kSelect: {
+      DCHECK_EQ(1u, set_operation_query.operands().size());
+      const ParseSelect &select_query =
+          static_cast<const ParseSelect&>(set_operation_query.operands().front());
+      return resolveSelect(select_query,
+                           set_operation_name,
+                           type_hints,
+                           parent_resolver);
+    }
+    default:
+      LOG(FATAL) << "Unknown set operation: " << set_operation_query.toString();
+      return nullptr;
+  }
+}
+
 E::SubqueryExpressionPtr Resolver::resolveSubqueryExpression(
     const ParseSubqueryExpression &parse_subquery_expression,
     const std::vector<const Type*> *type_hints,
     ExpressionResolutionInfo *expression_resolution_info,
     const bool has_single_column) {
+
+  // Subquery is now a set operation, not only a select operation
   L::LogicalPtr logical_subquery =
-      resolveSelect(*parse_subquery_expression.query(),
-                    "" /* select_name */,
-                    type_hints,
-                    &expression_resolution_info->name_resolver);
+      resolveSetOperation(*parse_subquery_expression.set_operation(),
+                          "" /* set_operation_name */,
+                          type_hints,
+                          &expression_resolution_info->name_resolver);
 
   // Raise SQL error if the subquery is expected to return only one column but
   // it returns multiple columns.
@@ -1614,10 +1772,10 @@ L::LogicalPtr Resolver::resolveTableReference(const ParseTableReference &table_r
       DCHECK(reference_signature->table_alias() != nullptr);
 
       reference_alias = reference_signature->table_alias();
-      logical_plan = resolveSelect(
-          *static_cast<const ParseSubqueryTableReference&>(table_reference).subquery_expr()->query(),
+      logical_plan = resolveSetOperation(
+          *static_cast<const ParseSubqueryTableReference&>(table_reference).subquery_expr()->set_operation(),
           reference_alias->value(),
-          nullptr /* No Type hints */,
+          nullptr /* type_hints */,
           nullptr /* parent_resolver */);
 
       if (reference_signature->column_aliases() != nullptr) {
@@ -3164,6 +3322,20 @@ void Resolver::rewriteIfOrdinalReference(
   }
 }
 
+void Resolver::CollapseSetOperation(const ParseSetOperation &toplevel,
+                                    const ParseSetOperation &current,
+                                    std::vector<const ParseSetOperation*> *output) {
+  if (current.getOperationType() == ParseSetOperation::kSelect ||
+      current.getOperationType() != toplevel.getOperationType()) {
+    output->emplace_back(&current);
+  } else {
+    for (const auto &child : current.operands()) {
+      CollapseSetOperation(
+          toplevel, static_cast<const ParseSetOperation&>(child), output);
+    }
+  }
+}
+
 std::string Resolver::GenerateWindowAggregateAttributeAlias(int index) {
   return "$window_aggregate" + std::to_string(index);
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/query_optimizer/resolver/Resolver.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/resolver/Resolver.hpp b/query_optimizer/resolver/Resolver.hpp
index a3d0833..1ae565a 100644
--- a/query_optimizer/resolver/Resolver.hpp
+++ b/query_optimizer/resolver/Resolver.hpp
@@ -50,6 +50,7 @@ class ParsePredicate;
 class ParseSearchedCaseExpression;
 class ParseSelect;
 class ParseSelectionClause;
+class ParseSetOperation;
 class ParseSimpleCaseExpression;
 class ParseSimpleTableReference;
 class ParseSubqueryTableReference;
@@ -61,7 +62,7 @@ class ParseStatementDelete;
 class ParseStatementDropTable;
 class ParseStatementInsertSelection;
 class ParseStatementInsertTuple;
-class ParseStatementSelect;
+class ParseStatementSetOperation;
 class ParseStatementUpdate;
 class ParseString;
 class ParseSubqueryExpression;
@@ -183,6 +184,24 @@ class Resolver {
       const NameResolver *parent_resolver);
 
   /**
+   * @brief Resolves multiple set operations at the same level.
+   */
+  logical::LogicalPtr resolveSetOperations(
+      const ParseSetOperation &parse_set_operations,
+      const std::string &set_operation_name,
+      const std::vector<const Type*> *type_hints,
+      const NameResolver *parent_resolver);
+
+  /**
+   * @brief Resolves a set operation and returns a logical plan.
+   */
+  logical::LogicalPtr resolveSetOperation(
+      const ParseSetOperation &set_operation_query,
+      const std::string &set_operation_name,
+      const std::vector<const Type*> *type_hints,
+      const NameResolver *parent_resolver);
+
+  /**
    * @brief Resolves a CREATE TABLE query and returns a logical plan.
    *
    * @param create_table_statement The CREATE TABLE parse tree.
@@ -559,6 +578,13 @@ class Resolver {
                                  const expressions::ScalarPtr &right_operand) const;
 
   /**
+   * @brief Collapse tree of homogenous set operations into a flat vector.
+   */
+  static void CollapseSetOperation(const ParseSetOperation &toplevel,
+                                   const ParseSetOperation &current,
+                                   std::vector<const ParseSetOperation*> *output);
+
+  /**
    * @brief Generates an internal alias for an aggregate attribute.
    *
    * @param index The index of the aggregate attribute used for generating the


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

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/tests/Select.test
----------------------------------------------------------------------
diff --git a/parser/tests/Select.test b/parser/tests/Select.test
index 8e47519..1618807 100644
--- a/parser/tests/Select.test
+++ b/parser/tests/Select.test
@@ -17,11 +17,13 @@
 
 SELECT * FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 # If the error location is beyond the end of the SQL,
@@ -91,70 +93,72 @@ SELECT 1, 2, 3, 4
 
 SELECT 1, 2, 1+1, 1-1, 1-1.2+1-2.3, attr1, attr2, attr1+1, attr1+1*2+attr2, attr1+1*(2+attr2) FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=2,float_like=false]
-  | +-SelectListItem
-  | | +-Add
-  | |   +-left_operand=Literal
-  | |   | +-NumericLiteral[numeric_string=1,float_like=false]
-  | |   +-right_operand=Literal
-  | |     +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-SelectListItem
-  | | +-Subtract
-  | |   +-left_operand=Literal
-  | |   | +-NumericLiteral[numeric_string=1,float_like=false]
-  | |   +-right_operand=Literal
-  | |     +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-SelectListItem
-  | | +-Subtract
-  | |   +-left_operand=Add
-  | |   | +-left_operand=Subtract
-  | |   | | +-left_operand=Literal
-  | |   | | | +-NumericLiteral[numeric_string=1,float_like=false]
-  | |   | | +-right_operand=Literal
-  | |   | |   +-NumericLiteral[numeric_string=1.2,float_like=true]
-  | |   | +-right_operand=Literal
-  | |   |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | |   +-right_operand=Literal
-  | |     +-NumericLiteral[numeric_string=2.3,float_like=true]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=attr1]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=attr2]
-  | +-SelectListItem
-  | | +-Add
-  | |   +-left_operand=AttributeReference[attribute_name=attr1]
-  | |   +-right_operand=Literal
-  | |     +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-SelectListItem
-  | | +-Add
-  | |   +-left_operand=Add
-  | |   | +-left_operand=AttributeReference[attribute_name=attr1]
-  | |   | +-right_operand=Multiply
-  | |   |   +-left_operand=Literal
-  | |   |   | +-NumericLiteral[numeric_string=1,float_like=false]
-  | |   |   +-right_operand=Literal
-  | |   |     +-NumericLiteral[numeric_string=2,float_like=false]
-  | |   +-right_operand=AttributeReference[attribute_name=attr2]
-  | +-SelectListItem
-  |   +-Add
-  |     +-left_operand=AttributeReference[attribute_name=attr1]
-  |     +-right_operand=Multiply
-  |       +-left_operand=Literal
-  |       | +-NumericLiteral[numeric_string=1,float_like=false]
-  |       +-right_operand=Add
-  |         +-left_operand=Literal
-  |         | +-NumericLiteral[numeric_string=2,float_like=false]
-  |         +-right_operand=AttributeReference[attribute_name=attr2]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=2,float_like=false]
+      | +-SelectListItem
+      | | +-Add
+      | |   +-left_operand=Literal
+      | |   | +-NumericLiteral[numeric_string=1,float_like=false]
+      | |   +-right_operand=Literal
+      | |     +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-SelectListItem
+      | | +-Subtract
+      | |   +-left_operand=Literal
+      | |   | +-NumericLiteral[numeric_string=1,float_like=false]
+      | |   +-right_operand=Literal
+      | |     +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-SelectListItem
+      | | +-Subtract
+      | |   +-left_operand=Add
+      | |   | +-left_operand=Subtract
+      | |   | | +-left_operand=Literal
+      | |   | | | +-NumericLiteral[numeric_string=1,float_like=false]
+      | |   | | +-right_operand=Literal
+      | |   | |   +-NumericLiteral[numeric_string=1.2,float_like=true]
+      | |   | +-right_operand=Literal
+      | |   |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | |   +-right_operand=Literal
+      | |     +-NumericLiteral[numeric_string=2.3,float_like=true]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=attr1]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=attr2]
+      | +-SelectListItem
+      | | +-Add
+      | |   +-left_operand=AttributeReference[attribute_name=attr1]
+      | |   +-right_operand=Literal
+      | |     +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-SelectListItem
+      | | +-Add
+      | |   +-left_operand=Add
+      | |   | +-left_operand=AttributeReference[attribute_name=attr1]
+      | |   | +-right_operand=Multiply
+      | |   |   +-left_operand=Literal
+      | |   |   | +-NumericLiteral[numeric_string=1,float_like=false]
+      | |   |   +-right_operand=Literal
+      | |   |     +-NumericLiteral[numeric_string=2,float_like=false]
+      | |   +-right_operand=AttributeReference[attribute_name=attr2]
+      | +-SelectListItem
+      |   +-Add
+      |     +-left_operand=AttributeReference[attribute_name=attr1]
+      |     +-right_operand=Multiply
+      |       +-left_operand=Literal
+      |       | +-NumericLiteral[numeric_string=1,float_like=false]
+      |       +-right_operand=Add
+      |         +-left_operand=Literal
+      |         | +-NumericLiteral[numeric_string=2,float_like=false]
+      |         +-right_operand=AttributeReference[attribute_name=attr2]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT * FROM 123
@@ -167,52 +171,58 @@ SELECT * FROM 123
 # Alias
 SELECT 1 AS a, 2+1 AS b, 3 a, 4 b FROM test AS a, test a
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem[alias=a]
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-SelectListItem[alias=b]
-  | | +-Add
-  | |   +-left_operand=Literal
-  | |   | +-NumericLiteral[numeric_string=2,float_like=false]
-  | |   +-right_operand=Literal
-  | |     +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-SelectListItem[alias=a]
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=3,float_like=false]
-  | +-SelectListItem[alias=b]
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=4,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
-    | +-table_signature=TableSignature[table_alias=a]
-    +-TableReference[table=test]
-      +-table_signature=TableSignature[table_alias=a]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem[alias=a]
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-SelectListItem[alias=b]
+      | | +-Add
+      | |   +-left_operand=Literal
+      | |   | +-NumericLiteral[numeric_string=2,float_like=false]
+      | |   +-right_operand=Literal
+      | |     +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-SelectListItem[alias=a]
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=3,float_like=false]
+      | +-SelectListItem[alias=b]
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=4,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
+        | +-table_signature=TableSignature[table_alias=a]
+        +-TableReference[table=test]
+          +-table_signature=TableSignature[table_alias=a]
 ==
 
 # Column list aliases in FROM.
 SELECT 1 FROM test test_alias(attr, attr1, attr2, attr3), (SELECT * FROM test) AS subquery(attr1, attr1, attr4)
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
-    | +-table_signature=TableSignature[table_alias=test_alias,
-    |   columns=(attr, attr1, attr2, attr3)]
-    +-SubqueryTable
-      +-table_signature=TableSignature[table_alias=subquery,
-      | columns=(attr1, attr1, attr4)]
-      +-SubqueryExpression
-        +-Select
-          +-select_clause=SelectStar
-          +-from_clause=
-            +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
+        | +-table_signature=TableSignature[table_alias=test_alias,
+        |   columns=(attr, attr1, attr2, attr3)]
+        +-SubqueryTable
+          +-table_signature=TableSignature[table_alias=subquery,
+          | columns=(attr1, attr1, attr4)]
+          +-SubqueryExpression
+            +-SetOperation[set_operation_type=Select]
+              +-children=
+                +-Select
+                  +-select_clause=SelectStar
+                  +-from_clause=
+                    +-TableReference[table=test]
 ==
 
 # Table subquery must be named.
@@ -253,147 +263,157 @@ SELECT 1 FROM test AS a+1
 
 SELECT 1 FROM test WHERE 1=1 AND attr1=1 AND attr+1>=1 AND attr-1<=1 AND attr+1>1 AND attr-1<1
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-where_clause=And
-  | +-Equal
-  | | +-left_operand=Literal
-  | | | +-NumericLiteral[numeric_string=1,float_like=false]
-  | | +-right_operand=Literal
-  | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=attr1]
-  | | +-right_operand=Literal
-  | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-GreaterOrEqual
-  | | +-left_operand=Add
-  | | | +-left_operand=AttributeReference[attribute_name=attr]
-  | | | +-right_operand=Literal
-  | | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | | +-right_operand=Literal
-  | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-LessOrEqual
-  | | +-left_operand=Subtract
-  | | | +-left_operand=AttributeReference[attribute_name=attr]
-  | | | +-right_operand=Literal
-  | | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | | +-right_operand=Literal
-  | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-Greater
-  | | +-left_operand=Add
-  | | | +-left_operand=AttributeReference[attribute_name=attr]
-  | | | +-right_operand=Literal
-  | | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | | +-right_operand=Literal
-  | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-Less
-  |   +-left_operand=Subtract
-  |   | +-left_operand=AttributeReference[attribute_name=attr]
-  |   | +-right_operand=Literal
-  |   |   +-NumericLiteral[numeric_string=1,float_like=false]
-  |   +-right_operand=Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-where_clause=And
+      | +-Equal
+      | | +-left_operand=Literal
+      | | | +-NumericLiteral[numeric_string=1,float_like=false]
+      | | +-right_operand=Literal
+      | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=attr1]
+      | | +-right_operand=Literal
+      | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-GreaterOrEqual
+      | | +-left_operand=Add
+      | | | +-left_operand=AttributeReference[attribute_name=attr]
+      | | | +-right_operand=Literal
+      | | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | | +-right_operand=Literal
+      | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-LessOrEqual
+      | | +-left_operand=Subtract
+      | | | +-left_operand=AttributeReference[attribute_name=attr]
+      | | | +-right_operand=Literal
+      | | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | | +-right_operand=Literal
+      | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-Greater
+      | | +-left_operand=Add
+      | | | +-left_operand=AttributeReference[attribute_name=attr]
+      | | | +-right_operand=Literal
+      | | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | | +-right_operand=Literal
+      | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-Less
+      |   +-left_operand=Subtract
+      |   | +-left_operand=AttributeReference[attribute_name=attr]
+      |   | +-right_operand=Literal
+      |   |   +-NumericLiteral[numeric_string=1,float_like=false]
+      |   +-right_operand=Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1 FROM test WHERE attr BETWEEN 1 AND 2 AND attr BETWEEN 2 AND 1
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-where_clause=And
-  | +-Between
-  | | +-check_operand=AttributeReference[attribute_name=attr]
-  | | +-lower_bound_operand=Literal
-  | | | +-NumericLiteral[numeric_string=1,float_like=false]
-  | | +-upper_bound_operand=Literal
-  | |   +-NumericLiteral[numeric_string=2,float_like=false]
-  | +-Between
-  |   +-check_operand=AttributeReference[attribute_name=attr]
-  |   +-lower_bound_operand=Literal
-  |   | +-NumericLiteral[numeric_string=2,float_like=false]
-  |   +-upper_bound_operand=Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-where_clause=And
+      | +-Between
+      | | +-check_operand=AttributeReference[attribute_name=attr]
+      | | +-lower_bound_operand=Literal
+      | | | +-NumericLiteral[numeric_string=1,float_like=false]
+      | | +-upper_bound_operand=Literal
+      | |   +-NumericLiteral[numeric_string=2,float_like=false]
+      | +-Between
+      |   +-check_operand=AttributeReference[attribute_name=attr]
+      |   +-lower_bound_operand=Literal
+      |   | +-NumericLiteral[numeric_string=2,float_like=false]
+      |   +-upper_bound_operand=Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1 FROM test WHERE attr NOT BETWEEN 1 AND 2
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-where_clause=Not
-  | +-Between
-  |   +-check_operand=AttributeReference[attribute_name=attr]
-  |   +-lower_bound_operand=Literal
-  |   | +-NumericLiteral[numeric_string=1,float_like=false]
-  |   +-upper_bound_operand=Literal
-  |     +-NumericLiteral[numeric_string=2,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-where_clause=Not
+      | +-Between
+      |   +-check_operand=AttributeReference[attribute_name=attr]
+      |   +-lower_bound_operand=Literal
+      |   | +-NumericLiteral[numeric_string=1,float_like=false]
+      |   +-upper_bound_operand=Literal
+      |     +-NumericLiteral[numeric_string=2,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1 FROM test ORDER BY attr, 1
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  | | +-AttributeReference[attribute_name=attr]
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      | | +-AttributeReference[attribute_name=attr]
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT fun(attr)+1 FROM test GROUP BY fun()+fun(attr), attr1+attr2 HAVING attr1>1 AND fun(*)>1
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Add
-  |     +-left_operand=FunctionCall[name=fun]
-  |     | +-AttributeReference[attribute_name=attr]
-  |     +-right_operand=Literal
-  |       +-NumericLiteral[numeric_string=1,float_like=false]
-  +-group_by=GroupBy
-  | +-Add
-  | | +-left_operand=FunctionCall[name=fun]
-  | | +-right_operand=FunctionCall[name=fun]
-  | |   +-AttributeReference[attribute_name=attr]
-  | +-Add
-  |   +-left_operand=AttributeReference[attribute_name=attr1]
-  |   +-right_operand=AttributeReference[attribute_name=attr2]
-  +-having=HAVING
-  | +-And
-  |   +-Greater
-  |   | +-left_operand=AttributeReference[attribute_name=attr1]
-  |   | +-right_operand=Literal
-  |   |   +-NumericLiteral[numeric_string=1,float_like=false]
-  |   +-Greater
-  |     +-left_operand=FunctionCall[name=fun,is_star=true]
-  |     +-right_operand=Literal
-  |       +-NumericLiteral[numeric_string=1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Add
+      |     +-left_operand=FunctionCall[name=fun]
+      |     | +-AttributeReference[attribute_name=attr]
+      |     +-right_operand=Literal
+      |       +-NumericLiteral[numeric_string=1,float_like=false]
+      +-group_by=GroupBy
+      | +-Add
+      | | +-left_operand=FunctionCall[name=fun]
+      | | +-right_operand=FunctionCall[name=fun]
+      | |   +-AttributeReference[attribute_name=attr]
+      | +-Add
+      |   +-left_operand=AttributeReference[attribute_name=attr1]
+      |   +-right_operand=AttributeReference[attribute_name=attr2]
+      +-having=HAVING
+      | +-And
+      |   +-Greater
+      |   | +-left_operand=AttributeReference[attribute_name=attr1]
+      |   | +-right_operand=Literal
+      |   |   +-NumericLiteral[numeric_string=1,float_like=false]
+      |   +-Greater
+      |     +-left_operand=FunctionCall[name=fun,is_star=true]
+      |     +-right_operand=Literal
+      |       +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 # ORDER BY
@@ -404,52 +424,56 @@ SELECT 1 FROM test ORDER BY 1 ASC,
                             attr1+1 NULLS FIRST,
                             fun(attr)/2
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-OrderByItem[is_asc=false,nulls_first=true]
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=2,float_like=false]
-  | +-OrderByItem[is_asc=true,nulls_first=true]
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=3,float_like=false]
-  | +-OrderByItem[is_asc=false,nulls_first=false]
-  | | +-AttributeReference[attribute_name=attr1]
-  | +-OrderByItem[is_asc=true,nulls_first=true]
-  | | +-Add
-  | |   +-left_operand=AttributeReference[attribute_name=attr1]
-  | |   +-right_operand=Literal
-  | |     +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  |   +-Divide
-  |     +-left_operand=FunctionCall[name=fun]
-  |     | +-AttributeReference[attribute_name=attr]
-  |     +-right_operand=Literal
-  |       +-NumericLiteral[numeric_string=2,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-OrderByItem[is_asc=false,nulls_first=true]
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=2,float_like=false]
+      | +-OrderByItem[is_asc=true,nulls_first=true]
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=3,float_like=false]
+      | +-OrderByItem[is_asc=false,nulls_first=false]
+      | | +-AttributeReference[attribute_name=attr1]
+      | +-OrderByItem[is_asc=true,nulls_first=true]
+      | | +-Add
+      | |   +-left_operand=AttributeReference[attribute_name=attr1]
+      | |   +-right_operand=Literal
+      | |     +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      |   +-Divide
+      |     +-left_operand=FunctionCall[name=fun]
+      |     | +-AttributeReference[attribute_name=attr]
+      |     +-right_operand=Literal
+      |       +-NumericLiteral[numeric_string=2,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 # Limit
 SELECT 1 FROM test LIMIT 1
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-limit=LIMIT
-  | +-NumericLiteral[numeric_string=1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-limit=LIMIT
+      | +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1 FROM test LIMIT 1.1
@@ -483,14 +507,16 @@ SELECT 1 FROM test LIMIT abc
 # Priority
 SELECT 1 FROM test WITH PRIORITY 1
 --
-SelectStatement
-+-select_query=Select
-| +-select_clause=SelectList
-| | +-SelectListItem
-| |   +-Literal
-| |     +-NumericLiteral[numeric_string=1,float_like=false]
-| +-from_clause=
-|   +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+| +-children=
+|   +-Select
+|     +-select_clause=SelectList
+|     | +-SelectListItem
+|     |   +-Literal
+|     |     +-NumericLiteral[numeric_string=1,float_like=false]
+|     +-from_clause=
+|       +-TableReference[table=test]
 +-priority=PRIORITY
   +-NumericLiteral[numeric_string=1,float_like=false]
 ==
@@ -530,25 +556,31 @@ SELECT 1 FROM test WITH PRIORITY abc
 # Subqueries in the FROM clause.
 SELECT * FROM test, (select * FROM test) AS a, (select * FROM test) a
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-from_clause=
-    +-TableReference[table=test]
-    +-SubqueryTable
-    | +-table_signature=TableSignature[table_alias=a]
-    | +-SubqueryExpression
-    |   +-Select
-    |     +-select_clause=SelectStar
-    |     +-from_clause=
-    |       +-TableReference[table=test]
-    +-SubqueryTable
-      +-table_signature=TableSignature[table_alias=a]
-      +-SubqueryExpression
-        +-Select
-          +-select_clause=SelectStar
-          +-from_clause=
-            +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-from_clause=
+        +-TableReference[table=test]
+        +-SubqueryTable
+        | +-table_signature=TableSignature[table_alias=a]
+        | +-SubqueryExpression
+        |   +-SetOperation[set_operation_type=Select]
+        |     +-children=
+        |       +-Select
+        |         +-select_clause=SelectStar
+        |         +-from_clause=
+        |           +-TableReference[table=test]
+        +-SubqueryTable
+          +-table_signature=TableSignature[table_alias=a]
+          +-SubqueryExpression
+            +-SetOperation[set_operation_type=Select]
+              +-children=
+                +-Select
+                  +-select_clause=SelectStar
+                  +-from_clause=
+                    +-TableReference[table=test]
 ==
 
 SELECT * FROM test, SELECT * FROM test
@@ -560,68 +592,84 @@ SELECT * FROM test, SELECT * FROM test
 
 SELECT * FROM (select * FROM (select * FROM (select * FROM test) a ) a ) a
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-from_clause=
-    +-SubqueryTable
-      +-table_signature=TableSignature[table_alias=a]
-      +-SubqueryExpression
-        +-Select
-          +-select_clause=SelectStar
-          +-from_clause=
-            +-SubqueryTable
-              +-table_signature=TableSignature[table_alias=a]
-              +-SubqueryExpression
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-from_clause=
+        +-SubqueryTable
+          +-table_signature=TableSignature[table_alias=a]
+          +-SubqueryExpression
+            +-SetOperation[set_operation_type=Select]
+              +-children=
                 +-Select
                   +-select_clause=SelectStar
                   +-from_clause=
                     +-SubqueryTable
                       +-table_signature=TableSignature[table_alias=a]
                       +-SubqueryExpression
-                        +-Select
-                          +-select_clause=SelectStar
-                          +-from_clause=
-                            +-TableReference[table=test]
+                        +-SetOperation[set_operation_type=Select]
+                          +-children=
+                            +-Select
+                              +-select_clause=SelectStar
+                              +-from_clause=
+                                +-SubqueryTable
+                                  +-table_signature=TableSignature[table_alias=a]
+                                  +-SubqueryExpression
+                                    +-SetOperation[set_operation_type=Select]
+                                      +-children=
+                                        +-Select
+                                          +-select_clause=SelectStar
+                                          +-from_clause=
+                                            +-TableReference[table=test]
 ==
 
 # Subqueries are not supported yet in clauses other than the FROM clause.
 SELECT (select * FROM test) FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-SubqueryExpression
-  |     +-Select
-  |       +-select_clause=SelectStar
-  |       +-from_clause=
-  |         +-TableReference[table=test]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-SubqueryExpression
+      |     +-SetOperation[set_operation_type=Select]
+      |       +-children=
+      |         +-Select
+      |           +-select_clause=SelectStar
+      |           +-from_clause=
+      |             +-TableReference[table=test]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1 FROM test WHERE 1 > (select 1 FROM test)
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-where_clause=Greater
-  | +-left_operand=Literal
-  | | +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-right_operand=SubqueryExpression
-  |   +-Select
-  |     +-select_clause=SelectList
-  |     | +-SelectListItem
-  |     |   +-Literal
-  |     |     +-NumericLiteral[numeric_string=1,float_like=false]
-  |     +-from_clause=
-  |       +-TableReference[table=test]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-where_clause=Greater
+      | +-left_operand=Literal
+      | | +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-right_operand=SubqueryExpression
+      |   +-SetOperation[set_operation_type=Select]
+      |     +-children=
+      |       +-Select
+      |         +-select_clause=SelectList
+      |         | +-SelectListItem
+      |         |   +-Literal
+      |         |     +-NumericLiteral[numeric_string=1,float_like=false]
+      |         +-from_clause=
+      |           +-TableReference[table=test]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 #
@@ -632,82 +680,96 @@ WITH a (col) AS (SELECT 1 FROM test GROUP BY a ORDER BY b LIMIT 1),
      c AS (SELECT 1 FROM test)
 SELECT 1 FROM a, b, c, d
 --
-SelectStatement
-+-select_query=Select
-| +-select_clause=SelectList
-| | +-SelectListItem
-| |   +-Literal
-| |     +-NumericLiteral[numeric_string=1,float_like=false]
-| +-from_clause=
-|   +-TableReference[table=a]
-|   +-TableReference[table=b]
-|   +-TableReference[table=c]
-|   +-TableReference[table=d]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+| +-children=
+|   +-Select
+|     +-select_clause=SelectList
+|     | +-SelectListItem
+|     |   +-Literal
+|     |     +-NumericLiteral[numeric_string=1,float_like=false]
+|     +-from_clause=
+|       +-TableReference[table=a]
+|       +-TableReference[table=b]
+|       +-TableReference[table=c]
+|       +-TableReference[table=d]
 +-with_clause=
   +-SubqueryTable
   | +-table_signature=TableSignature[table_alias=a,columns=(col)]
   | +-SubqueryExpression
-  |   +-Select
-  |     +-select_clause=SelectList
-  |     | +-SelectListItem
-  |     |   +-Literal
-  |     |     +-NumericLiteral[numeric_string=1,float_like=false]
-  |     +-group_by=GroupBy
-  |     | +-AttributeReference[attribute_name=a]
-  |     +-order_by=OrderBy
-  |     | +-OrderByItem[is_asc=true,nulls_first=false]
-  |     |   +-AttributeReference[attribute_name=b]
-  |     +-limit=LIMIT
-  |     | +-NumericLiteral[numeric_string=1,float_like=false]
-  |     +-from_clause=
-  |       +-TableReference[table=test]
+  |   +-SetOperation[set_operation_type=Select]
+  |     +-children=
+  |       +-Select
+  |         +-select_clause=SelectList
+  |         | +-SelectListItem
+  |         |   +-Literal
+  |         |     +-NumericLiteral[numeric_string=1,float_like=false]
+  |         +-group_by=GroupBy
+  |         | +-AttributeReference[attribute_name=a]
+  |         +-order_by=OrderBy
+  |         | +-OrderByItem[is_asc=true,nulls_first=false]
+  |         |   +-AttributeReference[attribute_name=b]
+  |         +-limit=LIMIT
+  |         | +-NumericLiteral[numeric_string=1,float_like=false]
+  |         +-from_clause=
+  |           +-TableReference[table=test]
   +-SubqueryTable
   | +-table_signature=TableSignature[table_alias=b,columns=(col1, col2, col3)]
   | +-SubqueryExpression
-  |   +-Select
-  |     +-select_clause=SelectStar
-  |     +-from_clause=
-  |       +-SubqueryTable
-  |         +-table_signature=TableSignature[table_alias=a]
-  |         +-SubqueryExpression
-  |           +-Select
-  |             +-select_clause=SelectStar
-  |             +-from_clause=
-  |               +-TableReference[table=test]
+  |   +-SetOperation[set_operation_type=Select]
+  |     +-children=
+  |       +-Select
+  |         +-select_clause=SelectStar
+  |         +-from_clause=
+  |           +-SubqueryTable
+  |             +-table_signature=TableSignature[table_alias=a]
+  |             +-SubqueryExpression
+  |               +-SetOperation[set_operation_type=Select]
+  |                 +-children=
+  |                   +-Select
+  |                     +-select_clause=SelectStar
+  |                     +-from_clause=
+  |                       +-TableReference[table=test]
   +-SubqueryTable
     +-table_signature=TableSignature[table_alias=c]
     +-SubqueryExpression
-      +-Select
-        +-select_clause=SelectList
-        | +-SelectListItem
-        |   +-Literal
-        |     +-NumericLiteral[numeric_string=1,float_like=false]
-        +-from_clause=
-          +-TableReference[table=test]
+      +-SetOperation[set_operation_type=Select]
+        +-children=
+          +-Select
+            +-select_clause=SelectList
+            | +-SelectListItem
+            |   +-Literal
+            |     +-NumericLiteral[numeric_string=1,float_like=false]
+            +-from_clause=
+              +-TableReference[table=test]
 ==
 
 WITH a AS (SELECT 1 FROM test)
 SELECT 1 FROM a
 --
-SelectStatement
-+-select_query=Select
-| +-select_clause=SelectList
-| | +-SelectListItem
-| |   +-Literal
-| |     +-NumericLiteral[numeric_string=1,float_like=false]
-| +-from_clause=
-|   +-TableReference[table=a]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+| +-children=
+|   +-Select
+|     +-select_clause=SelectList
+|     | +-SelectListItem
+|     |   +-Literal
+|     |     +-NumericLiteral[numeric_string=1,float_like=false]
+|     +-from_clause=
+|       +-TableReference[table=a]
 +-with_clause=
   +-SubqueryTable
     +-table_signature=TableSignature[table_alias=a]
     +-SubqueryExpression
-      +-Select
-        +-select_clause=SelectList
-        | +-SelectListItem
-        |   +-Literal
-        |     +-NumericLiteral[numeric_string=1,float_like=false]
-        +-from_clause=
-          +-TableReference[table=test]
+      +-SetOperation[set_operation_type=Select]
+        +-children=
+          +-Select
+            +-select_clause=SelectList
+            | +-SelectListItem
+            |   +-Literal
+            |     +-NumericLiteral[numeric_string=1,float_like=false]
+            +-from_clause=
+              +-TableReference[table=test]
 ==
 
 # AS cannot be omitted.
@@ -774,14 +836,16 @@ SELECT 1 FROM test WHERE tb1 = TRUE
 # FIXME(chasseur, qzeng): The result is wrong. Add support for hexadecimal literals.
 SELECT 0xfff FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem[alias=xfff]
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=0,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem[alias=xfff]
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=0,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT x'fff' FROM test
@@ -795,35 +859,37 @@ SELECT 123e23, 123e-123, 123e-2+1, 123e-2*2, 123e-123,
        1e100000000
 FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=123e23,float_like=true]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=123e-123,float_like=true]
-  | +-SelectListItem
-  | | +-Add
-  | |   +-left_operand=Literal
-  | |   | +-NumericLiteral[numeric_string=123e-2,float_like=true]
-  | |   +-right_operand=Literal
-  | |     +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-SelectListItem
-  | | +-Multiply
-  | |   +-left_operand=Literal
-  | |   | +-NumericLiteral[numeric_string=123e-2,float_like=true]
-  | |   +-right_operand=Literal
-  | |     +-NumericLiteral[numeric_string=2,float_like=false]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=123e-123,float_like=true]
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1e100000000,float_like=true]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=123e23,float_like=true]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=123e-123,float_like=true]
+      | +-SelectListItem
+      | | +-Add
+      | |   +-left_operand=Literal
+      | |   | +-NumericLiteral[numeric_string=123e-2,float_like=true]
+      | |   +-right_operand=Literal
+      | |     +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-SelectListItem
+      | | +-Multiply
+      | |   +-left_operand=Literal
+      | |   | +-NumericLiteral[numeric_string=123e-2,float_like=true]
+      | |   +-right_operand=Literal
+      | |     +-NumericLiteral[numeric_string=2,float_like=false]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=123e-123,float_like=true]
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1e100000000,float_like=true]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 9223372036854775805, 9223372036854775807, 9223372036854775809, -9223372036854775805,
@@ -831,77 +897,88 @@ SELECT 9223372036854775805, 9223372036854775807, 9223372036854775809, -922337203
        -9223372036854775800.8, -1.123456789012345678901234567890
 FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=9223372036854775805,float_like=false]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=9223372036854775807,float_like=false]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=9223372036854775809,float_like=false]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=-9223372036854775805,float_like=false]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=-9223372036854775807,float_like=false]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=-9223372036854775809,float_like=false]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=9223372036854775800.8,float_like=true]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=1.123456789012345678901234567890,
-  | |     float_like=true]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=-9223372036854775800.8,float_like=true]
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=-1.123456789012345678901234567890,
-  |       float_like=true]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=9223372036854775805,float_like=false]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=9223372036854775807,float_like=false]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=9223372036854775809,float_like=false]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=-9223372036854775805,
+      | |     float_like=false]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=-9223372036854775807,
+      | |     float_like=false]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=-9223372036854775809,
+      | |     float_like=false]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=9223372036854775800.8,
+      | |     float_like=true]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=1.123456789012345678901234567890,
+      | |     float_like=true]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=-9223372036854775800.8,
+      | |     float_like=true]
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=-1.123456789012345678901234567890,
+      |       float_like=true]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 # Escape characters.
 SELECT 1, e'$asdfg\'\'\"\"\t\r\n' FROM test WHERE char_col = e'\'asdfg\''
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-SelectListItem
-  |   +-Literal
-  |     +-StringLiteral[value=$asdfg''""	
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-SelectListItem
+      |   +-Literal
+      |     +-StringLiteral[value=$asdfg''""	
 ]
-  +-where_clause=Equal
-  | +-left_operand=AttributeReference[attribute_name=char_col]
-  | +-right_operand=Literal
-  |   +-StringLiteral[value='asdfg']
-  +-from_clause=
-    +-TableReference[table=test]
+      +-where_clause=Equal
+      | +-left_operand=AttributeReference[attribute_name=char_col]
+      | +-right_operand=Literal
+      |   +-StringLiteral[value='asdfg']
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 # Double-quoted strings are identifiers, which do not need to be escaped.
 SELECT 1 "abc\n" FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem[alias=abc\n]
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem[alias=abc\n]
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1 "abc
@@ -909,17 +986,19 @@ SELECT 1 "abc
 def
 " FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem[alias=abc
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem[alias=abc
 
 def
 ]
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 # Currently predicate is not treated as a regular scalar expression.
@@ -932,17 +1011,19 @@ SELECT 1 FROM test WHERE (1>1)=(1<1)
 
 SELECT 1 """this IS a double-quoted string""", 2 "this IS NOT a double-quoted string" FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem[alias="this IS a double-quoted string"]
-  | | +-Literal
-  | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-SelectListItem[alias=this IS NOT a double-quoted string]
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=2,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem[alias="this IS a double-quoted string"]
+      | | +-Literal
+      | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-SelectListItem[alias=this IS NOT a double-quoted string]
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=2,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 #
@@ -953,66 +1034,74 @@ SELECT '1998-12-01',
        DATE '1998-12-01'
 FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1998-12-01]
-  | +-SelectListItem
-  |   +-Literal
-  |     +-StringLiteral[value=1998-12-01,explicit_type=Date]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1998-12-01]
+      | +-SelectListItem
+      |   +-Literal
+      |     +-StringLiteral[value=1998-12-01,explicit_type=Date]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 # Year before 1970.
 SELECT DATE '1960-12-12',
        DATE '1901-12-14' FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1960-12-12,explicit_type=Date]
-  | +-SelectListItem
-  |   +-Literal
-  |     +-StringLiteral[value=1901-12-14,explicit_type=Date]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1960-12-12,explicit_type=Date]
+      | +-SelectListItem
+      |   +-Literal
+      |     +-StringLiteral[value=1901-12-14,explicit_type=Date]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT DATE '1998-2-12', DATE '1998-12-2' FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1998-2-12,explicit_type=Date]
-  | +-SelectListItem
-  |   +-Literal
-  |     +-StringLiteral[value=1998-12-2,explicit_type=Date]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1998-2-12,explicit_type=Date]
+      | +-SelectListItem
+      |   +-Literal
+      |     +-StringLiteral[value=1998-12-2,explicit_type=Date]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT DATE '+1921-12-12',
        DATE '+10001-12-12'
 FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=+1921-12-12,explicit_type=Date]
-  | +-SelectListItem
-  |   +-Literal
-  |     +-StringLiteral[value=+10001-12-12,explicit_type=Date]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=+1921-12-12,explicit_type=Date]
+      | +-SelectListItem
+      |   +-Literal
+      |     +-StringLiteral[value=+10001-12-12,explicit_type=Date]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT DATE 'a1998-12-12' FROM test
@@ -1054,20 +1143,22 @@ SELECT DATE '1996-02-29',
        DATE '1997-03-31',
        DATE '1998-04-30' FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1996-02-29,explicit_type=Date]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1997-03-31,explicit_type=Date]
-  | +-SelectListItem
-  |   +-Literal
-  |     +-StringLiteral[value=1998-04-30,explicit_type=Date]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1996-02-29,explicit_type=Date]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1997-03-31,explicit_type=Date]
+      | +-SelectListItem
+      |   +-Literal
+      |     +-StringLiteral[value=1998-04-30,explicit_type=Date]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT DATE '1999-02-29' FROM test
@@ -1082,17 +1173,19 @@ SELECT DATETIME '2007-05-08 12:35:29',
        DATETIME '2007-05-08 12:35:29.010'
 FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=2007-05-08 12:35:29,explicit_type=Datetime]
-  | +-SelectListItem
-  |   +-Literal
-  |     +-StringLiteral[value=2007-05-08 12:35:29.010,explicit_type=Datetime]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=2007-05-08 12:35:29,explicit_type=Datetime]
+      | +-SelectListItem
+      |   +-Literal
+      |     +-StringLiteral[value=2007-05-08 12:35:29.010,explicit_type=Datetime]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT DATE '1999-04-31' FROM test
@@ -1104,21 +1197,23 @@ SELECT DATE '1999-04-31' FROM test
 
 SELECT 1 FROM test WHERE attr_date <= DATE '1998-12-01' - INTERVAL '96 day'
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-where_clause=LessOrEqual
-  | +-left_operand=AttributeReference[attribute_name=attr_date]
-  | +-right_operand=Subtract
-  |   +-left_operand=Literal
-  |   | +-StringLiteral[value=1998-12-01,explicit_type=Date]
-  |   +-right_operand=Literal
-  |     +-StringLiteral[value=96 day,explicit_type=DatetimeInterval]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-where_clause=LessOrEqual
+      | +-left_operand=AttributeReference[attribute_name=attr_date]
+      | +-right_operand=Subtract
+      |   +-left_operand=Literal
+      |   | +-StringLiteral[value=1998-12-01,explicit_type=Date]
+      |   +-right_operand=Literal
+      |     +-StringLiteral[value=96 day,explicit_type=DatetimeInterval]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT INTERVAL '1 us', INTERVAL '1 ms', INTERVAL '1 s', INTERVAL '1 minute',
@@ -1133,83 +1228,85 @@ SELECT INTERVAL '1 us', INTERVAL '1 ms', INTERVAL '1 s', INTERVAL '1 minute',
            YEARMONTH INTERVAL '1 century', YEARMONTH INTERVAL '1 millennium'
 FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 us,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 ms,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 s,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 minute,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 h,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 day,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 week,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 month,explicit_type=YearMonthInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 year,explicit_type=YearMonthInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 decade,explicit_type=YearMonthInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 century,explicit_type=YearMonthInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 millennium,explicit_type=YearMonthInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 us,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 ms,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 s,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 minute,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 h,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 day,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 week,explicit_type=DatetimeInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 month,explicit_type=YearMonthInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 year,explicit_type=YearMonthInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 decade,explicit_type=YearMonthInterval]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1 century,explicit_type=YearMonthInterval]
-  | +-SelectListItem
-  |   +-Literal
-  |     +-StringLiteral[value=1 millennium,explicit_type=YearMonthInterval]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 us,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 ms,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 s,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 minute,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 h,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 day,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 week,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 month,explicit_type=YearMonthInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 year,explicit_type=YearMonthInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 decade,explicit_type=YearMonthInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 century,explicit_type=YearMonthInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 millennium,explicit_type=YearMonthInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 us,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 ms,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 s,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 minute,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 h,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 day,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 week,explicit_type=DatetimeInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 month,explicit_type=YearMonthInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 year,explicit_type=YearMonthInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 decade,explicit_type=YearMonthInterval]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1 century,explicit_type=YearMonthInterval]
+      | +-SelectListItem
+      |   +-Literal
+      |     +-StringLiteral[value=1 millennium,explicit_type=YearMonthInterval]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT INTERVAL '4 day' (1)
@@ -1229,14 +1326,16 @@ SELECT INTERVAL '2 moth' FROM test
 
 SELECT INTERVAL '-3 year' FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-StringLiteral[value=-3 year,explicit_type=YearMonthInterval]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-StringLiteral[value=-3 year,explicit_type=YearMonthInterval]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT INTERVAL 'a 3 year' FROM test
@@ -1267,22 +1366,36 @@ SELECT INTERVAL '5-3 day to second' FROM test
 
 SELECT 1-1 FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Subtract
-  |     +-left_operand=Literal
-  |     | +-NumericLiteral[numeric_string=1,float_like=false]
-  |     +-right_operand=Literal
-  |       +-NumericLiteral[numeric_string=1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Subtract
+      |     +-left_operand=Literal
+      |     | +-NumericLiteral[numeric_string=1,float_like=false]
+      |     +-right_operand=Literal
+      |       +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1 - 1 FROM test
 --
-[same as above]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Subtract
+      |     +-left_operand=Literal
+      |     | +-NumericLiteral[numeric_string=1,float_like=false]
+      |     +-right_operand=Literal
+      |       +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1--1 FROM test
@@ -1294,50 +1407,56 @@ SELECT 1--1 FROM test
 
 SELECT 1+-1 FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Add
-  |     +-left_operand=Literal
-  |     | +-NumericLiteral[numeric_string=1,float_like=false]
-  |     +-right_operand=Literal
-  |       +-NumericLiteral[numeric_string=-1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Add
+      |     +-left_operand=Literal
+      |     | +-NumericLiteral[numeric_string=1,float_like=false]
+      |     +-right_operand=Literal
+      |       +-NumericLiteral[numeric_string=-1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1-1+-1 FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Add
-  |     +-left_operand=Subtract
-  |     | +-left_operand=Literal
-  |     | | +-NumericLiteral[numeric_string=1,float_like=false]
-  |     | +-right_operand=Literal
-  |     |   +-NumericLiteral[numeric_string=1,float_like=false]
-  |     +-right_operand=Literal
-  |       +-NumericLiteral[numeric_string=-1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Add
+      |     +-left_operand=Subtract
+      |     | +-left_operand=Literal
+      |     | | +-NumericLiteral[numeric_string=1,float_like=false]
+      |     | +-right_operand=Literal
+      |     |   +-NumericLiteral[numeric_string=1,float_like=false]
+      |     +-right_operand=Literal
+      |       +-NumericLiteral[numeric_string=-1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1+(-1) FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Add
-  |     +-left_operand=Literal
-  |     | +-NumericLiteral[numeric_string=1,float_like=false]
-  |     +-right_operand=Literal
-  |       +-NumericLiteral[numeric_string=-1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Add
+      |     +-left_operand=Literal
+      |     | +-NumericLiteral[numeric_string=1,float_like=false]
+      |     +-right_operand=Literal
+      |       +-NumericLiteral[numeric_string=-1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 # Quit statement
@@ -1349,29 +1468,31 @@ QuitStatement
 # Literals with various explicit types.
 SELECT INT '1', LONG '1', FLOAT '1', DOUBLE '1', char(42) 'foo', varchar(42) 'bar' FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1,explicit_type=Int]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1,explicit_type=Long]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1,explicit_type=Float]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=1,explicit_type=Double]
-  | +-SelectListItem
-  | | +-Literal
-  | |   +-StringLiteral[value=foo,explicit_type=Char(42)]
-  | +-SelectListItem
-  |   +-Literal
-  |     +-StringLiteral[value=bar,explicit_type=VarChar(42)]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1,explicit_type=Int]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1,explicit_type=Long]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1,explicit_type=Float]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=1,explicit_type=Double]
+      | +-SelectListItem
+      | | +-Literal
+      | |   +-StringLiteral[value=foo,explicit_type=Char(42)]
+      | +-SelectListItem
+      |   +-Literal
+      |     +-StringLiteral[value=bar,explicit_type=VarChar(42)]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 # Value needs to be quoted when giving an explicit type.
@@ -1394,50 +1515,58 @@ SELECT INT 'foo' FROM test
 # acceptable according to the SQL standard.
 SELECT 1. FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1.,float_like=true]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1.,float_like=true]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT .1 FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=.1,float_like=true]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=.1,float_like=true]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1.e1 FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1.e1,float_like=true]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1.e1,float_like=true]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT .1e1 FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=.1e1,float_like=true]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=.1e1,float_like=true]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 # EXISTS subquery
@@ -1455,57 +1584,64 @@ WHERE
     WHERE gs1.i = gs3.i
   );
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-AttributeReference[attribute_name=i]
-  +-where_clause=And
-  | +-Exists
-  | | +-subquery=SubqueryExpression
-  | |   +-Select
-  | |     +-select_clause=SelectStar
-  | |     +-where_clause=Equal
-  | |     | +-left_operand=AttributeReference[attribute_name=i,relation_name=gs1]
-  | |     | +-right_operand=AttributeReference[attribute_name=i,
-  | |     |   relation_name=gs2]
-  | |     +-from_clause=
-  | |       +-TableGenerator
-  | |         +-FunctionCall[name=generate_series]
-  | |           +-Literal
-  | |           | +-NumericLiteral[numeric_string=0,float_like=false]
-  | |           +-Literal
-  | |           | +-NumericLiteral[numeric_string=100,float_like=false]
-  | |           +-Literal
-  | |             +-NumericLiteral[numeric_string=5,float_like=false]
-  | +-Not
-  |   +-Exists
-  |     +-subquery=SubqueryExpression
-  |       +-Select
-  |         +-select_clause=SelectStar
-  |         +-where_clause=Equal
-  |         | +-left_operand=AttributeReference[attribute_name=i,
-  |         | | relation_name=gs1]
-  |         | +-right_operand=AttributeReference[attribute_name=i,
-  |         |   relation_name=gs3]
-  |         +-from_clause=
-  |           +-TableGenerator
-  |             +-FunctionCall[name=generate_series]
-  |               +-Literal
-  |               | +-NumericLiteral[numeric_string=0,float_like=false]
-  |               +-Literal
-  |               | +-NumericLiteral[numeric_string=100,float_like=false]
-  |               +-Literal
-  |                 +-NumericLiteral[numeric_string=10,float_like=false]
-  +-from_clause=
-    +-TableGenerator
-      +-FunctionCall[name=generate_series]
-        +-Literal
-        | +-NumericLiteral[numeric_string=0,float_like=false]
-        +-Literal
-        | +-NumericLiteral[numeric_string=100,float_like=false]
-        +-Literal
-          +-NumericLiteral[numeric_string=3,float_like=false]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-AttributeReference[attribute_name=i]
+      +-where_clause=And
+      | +-Exists
+      | | +-subquery=SubqueryExpression
+      | |   +-SetOperation[set_operation_type=Select]
+      | |     +-children=
+      | |       +-Select
+      | |         +-select_clause=SelectStar
+      | |         +-where_clause=Equal
+      | |         | +-left_operand=AttributeReference[attribute_name=i,
+      | |         | | relation_name=gs1]
+      | |         | +-right_operand=AttributeReference[attribute_name=i,
+      | |         |   relation_name=gs2]
+      | |         +-from_clause=
+      | |           +-TableGenerator
+      | |             +-FunctionCall[name=generate_series]
+      | |               +-Literal
+      | |               | +-NumericLiteral[numeric_string=0,float_like=false]
+      | |               +-Literal
+      | |               | +-NumericLiteral[numeric_string=100,float_like=false]
+      | |               +-Literal
+      | |                 +-NumericLiteral[numeric_string=5,float_like=false]
+      | +-Not
+      |   +-Exists
+      |     +-subquery=SubqueryExpression
+      |       +-SetOperation[set_operation_type=Select]
+      |         +-children=
+      |           +-Select
+      |             +-select_clause=SelectStar
+      |             +-where_clause=Equal
+      |             | +-left_operand=AttributeReference[attribute_name=i,
+      |             | | relation_name=gs1]
+      |             | +-right_operand=AttributeReference[attribute_name=i,
+      |             |   relation_name=gs3]
+      |             +-from_clause=
+      |               +-TableGenerator
+      |                 +-FunctionCall[name=generate_series]
+      |                   +-Literal
+      |                   | +-NumericLiteral[numeric_string=0,float_like=false]
+      |                   +-Literal
+      |                   | +-NumericLiteral[numeric_string=100,float_like=false]
+      |                   +-Literal
+      |                     +-NumericLiteral[numeric_string=10,float_like=false]
+      +-from_clause=
+        +-TableGenerator
+          +-FunctionCall[name=generate_series]
+            +-Literal
+            | +-NumericLiteral[numeric_string=0,float_like=false]
+            +-Literal
+            | +-NumericLiteral[numeric_string=100,float_like=false]
+            +-Literal
+              +-NumericLiteral[numeric_string=3,float_like=false]
 ==
 
 # CASE expressions.
@@ -1515,25 +1651,27 @@ SELECT CASE col1%2
        END
 FROM test;
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-SimpleCaseExpression
-  |     +-case_operand=Modulo
-  |     | +-left_operand=AttributeReference[attribute_name=col1]
-  |     | +-right_operand=Literal
-  |     |   +-NumericLiteral[numeric_string=2,float_like=false]
-  |     +-else_result_expression=Literal
-  |     | +-StringLiteral[value=even]
-  |     +-when_clauses=
-  |       +-SimpleWhenClause
-  |         +-condition_operand=Literal
-  |         | +-NumericLiteral[numeric_string=1,float_like=false]
-  |         +-result_expression=Literal
-  |           +-StringLiteral[value=odd]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-SimpleCaseExpression
+      |     +-case_operand=Modulo
+      |     | +-left_operand=AttributeReference[attribute_name=col1]
+      |     | +-right_operand=Literal
+      |     |   +-NumericLiteral[numeric_string=2,float_like=false]
+      |     +-else_result_expression=Literal
+      |     | +-StringLiteral[value=even]
+      |     +-when_clauses=
+      |       +-SimpleWhenClause
+      |         +-condition_operand=Literal
+      |         | +-NumericLiteral[numeric_string=1,float_like=false]
+      |         +-result_expression=Literal
+      |           +-StringLiteral[value=odd]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT *
@@ -1542,22 +1680,24 @@ WHERE CASE WHEN col1 > col2 THEN col3
            ELSE col4
       END > 0;
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-where_clause=Greater
-  | +-left_operand=SearchedCaseExpression
-  | | +-else_result_expression=AttributeReference[attribute_name=col4]
-  | | +-when_clauses=
-  | |   +-SearchedWhenClause
-  | |     +-condition_predicate=Greater
-  | |     | +-left_operand=AttributeReference[attribute_name=col1]
-  | |     | +-right_operand=AttributeReference[attribute_name=col2]
-  | |     +-result_expression=AttributeReference[attribute_name=col3]
-  | +-right_operand=Literal
-  |   +-NumericLiteral[numeric_string=0,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-where_clause=Greater
+      | +-left_operand=SearchedCaseExpression
+      | | +-else_result_expression=AttributeReference[attribute_name=col4]
+      | | +-when_clauses=
+      | |   +-SearchedWhenClause
+      | |     +-condition_predicate=Greater
+      | |     | +-left_operand=AttributeReference[attribute_name=col1]
+      | |     | +-right_operand=AttributeReference[attribute_name=col2]
+      | |     +-result_expression=AttributeReference[attribute_name=col3]
+      | +-right_operand=Literal
+      |   +-NumericLiteral[numeric_string=0,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT
@@ -1575,55 +1715,59 @@ SELECT
       END + col4)
 FROM test;
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-FunctionCall[name=FUN]
-  |     +-Add
-  |       +-left_operand=SimpleCaseExpression
-  |       | +-case_operand=AttributeReference[attribute_name=col1]
-  |       | +-else_result_expression=Literal
-  |       | | +-NumericLiteral[numeric_string=0,float_like=false]
-  |       | +-when_clauses=
-  |       |   +-SimpleWhenClause
-  |       |   | +-condition_operand=Add
-  |       |   | | +-left_operand=AttributeReference[attribute_name=col2]
-  |       |   | | +-right_operand=FunctionCall[name=FUN]
-  |       |   | |   +-Add
-  |       |   | |     +-left_operand=SearchedCaseExpression
-  |       |   | |     | +-else_result_expression=Literal
-  |       |   | |     | | +-NumericLiteral[numeric_string=1,float_like=false]
-  |       |   | |     | +-when_clauses=
-  |       |   | |     |   +-SearchedWhenClause
-  |       |   | |     |     +-condition_predicate=Less
-  |       |   | |     |     | +-left_operand=AttributeReference[
-  |       |   | |     |     | | attribute_name=col1]
-  |       |   | |     |     | +-right_operand=AttributeReference[
-  |       |   | |     |     |   attribute_name=col2]
-  |       |   | |     |     +-result_expression=Literal
-  |       |   | |     |       +-NumericLiteral[numeric_string=0,float_like=false]
-  |       |   | |     +-right_operand=SearchedCaseExpression
-  |       |   | |       +-else_result_expression=Literal
-  |       |   | |       | +-NumericLiteral[numeric_string=1,float_like=false]
-  |       |   | |       +-when_clauses=
-  |       |   | |         +-SearchedWhenClause
-  |       |   | |           +-condition_predicate=Less
-  |       |   | |           | +-left_operand=AttributeReference[
-  |       |   | |           | | attribute_name=col1]
-  |       |   | |           | +-right_operand=AttributeReference[
-  |       |   | |           |   attribute_name=col3]
-  |       |   | |           +-result_expression=Literal
-  |       |   | |             +-NumericLiteral[numeric_string=0,float_like=false]
-  |       |   | +-result_expression=Literal
-  |       |   |   +-NumericLiteral[numeric_string=1,float_like=false]
-  |       |   +-SimpleWhenClause
-  |       |     +-condition_operand=AttributeReference[attribute_name=col3]
-  |       |     +-result_expression=Literal
-  |       |       +-NumericLiteral[numeric_string=2,float_like=false]
-  |       +-right_operand=AttributeReference[attribute_name=col4]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-FunctionCall[name=FUN]
+      |     +-Add
+      |       +-left_operand=SimpleCaseExpression
+      |       | +-case_operand=AttributeReference[attribute_name=col1]
+      |       | +-else_result_expression=Literal
+      |       | | +-NumericLiteral[numeric_string=0,float_like=false]
+      |       | +-when_clauses=
+      |       |   +-SimpleWhenClause
+      |       |   | +-condition_operand=Add
+      |       |   | | +-left_operand=AttributeReference[attribute_name=col2]
+      |       |   | | +-right_operand=FunctionCall[name=FUN]
+      |       |   | |   +-Add
+      |       |   | |     +-left_operand=SearchedCaseExpression
+      |       |   | |     | +-else_result_expression=Literal
+      |       |   | |     | | +-NumericLiteral[numeric_string=1,float_like=false]
+      |       |   | |     | +-when_clauses=
+      |       |   | |     |   +-SearchedWhenClause
+      |       |   | |     |     +-condition_predicate=Less
+      |       |   | |     |     | +-left_operand=AttributeReference[
+      |       |   | |     |     | | attribute_name=col1]
+      |       |   | |     |     | +-right_operand=AttributeReference[
+      |       |   | |     |     |   attribute_name=col2]
+      |       |   | |     |     +-result_expression=Literal
+      |       |   | |     |       +-NumericLiteral[numeric_string=0,
+      |       |   | |     |         float_like=false]
+      |       |   | |     +-right_operand=SearchedCaseExpression
+      |       |   | |       +-else_result_expression=Literal
+      |       |   | |       | +-NumericLiteral[numeric_string=1,float_like=false]
+      |       |   | |       +-when_clauses=
+      |       |   | |         +-SearchedWhenClause
+      |       |   | |           +-condition_predicate=Less
+      |       |   | |           | +-left_operand=AttributeReference[
+      |       |   | |           | | attribute_name=col1]
+      |       |   | |           | +-right_operand=AttributeReference[
+      |       |   | |           |   attribute_name=col3]
+      |       |   | |           +-result_expression=Literal
+      |       |   | |             +-NumericLiteral[numeric_string=0,
+      |       |   | |               float_like=false]
+      |       |   | +-result_expression=Literal
+      |       |   |   +-NumericLiteral[numeric_string=1,float_like=false]
+      |       |   +-SimpleWhenClause
+      |       |     +-condition_operand=AttributeReference[attribute_name=col3]
+      |       |     +-result_expression=Literal
+      |       |       +-NumericLiteral[numeric_string=2,float_like=false]
+      |       +-right_operand=AttributeReference[attribute_name=col4]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 # IN predicate
@@ -1631,20 +1775,22 @@ SELECT *
 FROM test
 WHERE col1 IN (1, 3, 5);
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-where_clause=InValueList
-  | +-test_expression=AttributeReference[attribute_name=col1]
-  | +-value_list=
-  |   +-Literal
-  |   | +-NumericLiteral[numeric_string=1,float_like=false]
-  |   +-Literal
-  |   | +-NumericLiteral[numeric_string=3,float_like=false]
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=5,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-where_clause=InValueList
+      | +-test_expression=AttributeReference[attribute_name=col1]
+      | +-value_list=
+      |   +-Literal
+      |   | +-NumericLiteral[numeric_string=1,float_like=false]
+      |   +-Literal
+      |   | +-NumericLiteral[numeric_string=3,float_like=false]
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=5,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT *
@@ -1653,48 +1799,52 @@ WHERE col1 IN (FUN(1),
                col2+col3,
                CASE WHEN col4 > 0 THEN col5 ELSE col6 END);
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-where_clause=InValueList
-  | +-test_expression=AttributeReference[attribute_name=col1]
-  | +-value_list=
-  |   +-FunctionCall[name=FUN]
-  |   | +-Literal
-  |   |   +-NumericLiteral[numeric_string=1,float_like=false]
-  |   +-Add
-  |   | +-left_operand=AttributeReference[attribute_name=col2]
-  |   | +-right_operand=AttributeReference[attribute_name=col3]
-  |   +-SearchedCaseExpression
-  |     +-else_result_expression=AttributeReference[attribute_name=col6]
-  |     +-when_clauses=
-  |       +-SearchedWhenClause
-  |         +-condition_predicate=Greater
-  |         | +-left_operand=AttributeReference[attribute_name=col4]
-  |         | +-right_operand=Literal
-  |         |   +-NumericLiteral[numeric_string=0,float_like=false]
-  |         +-result_expression=AttributeReference[attribute_name=col5]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-where_clause=InValueList
+      | +-test_expression=AttributeReference[attribute_name=col1]
+      | +-value_list=
+      |   +-FunctionCall[name=FUN]
+      |   | +-Literal
+      |   |   +-NumericLiteral[numeric_string=1,float_like=false]
+      |   +-Add
+      |   | +-left_operand=AttributeReference[attribute_name=col2]
+      |   | +-right_operand=AttributeReference[attribute_name=col3]
+      |   +-SearchedCaseExpression
+      |     +-else_result_expression=AttributeReference[attribute_name=col6]
+      |     +-when_clauses=
+      |       +-SearchedWhenClause
+      |         +-condition_predicate=Greater
+      |         | +-left_operand=AttributeReference[attribute_name=col4]
+      |         | +-right_operand=Literal
+      |         |   +-NumericLiteral[numeric_string=0,float_like=false]
+      |         +-result_expression=AttributeReference[attribute_name=col5]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT *
 FROM test
 WHERE col1 NOT IN (col1, col2 + col3);
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-where_clause=Not
-  | +-InValueList
-  |   +-test_expression=AttributeReference[attribute_name=col1]
-  |   +-value_list=
-  |     +-AttributeReference[attribute_name=col1]
-  |     +-Add
-  |       +-left_operand=AttributeReference[attribute_name=col2]
-  |       +-right_operand=AttributeReference[attribute_name=col3]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-where_clause=Not
+      | +-InValueList
+      |   +-test_expression=AttributeReference[attribute_name=col1]
+      |   +-value_list=
+      |     +-AttributeReference[attribute_name=col1]
+      |     +-Add
+      |       +-left_operand=AttributeReference[attribute_name=col2]
+      |       +-right_operand=AttributeReference[attribute_name=col3]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT *
@@ -1705,25 +1855,29 @@ WHERE col1 IN (
   GROUP BY col4
 );
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-where_clause=InTableQuery
-  | +-test_expression=AttributeReference[attribute_name=col1]
-  | +-table_query=SubqueryExpression
-  |   +-Select
-  |     +-select_clause=SelectList
-  |     | +-SelectListItem
-  |     |   +-FunctionCall[name=SUM]
-  |     |     +-Add
-  |     |       +-left_operand=AttributeReference[attribute_name=col2]
-  |     |       +-right_operand=AttributeReference[attribute_name=col3]
-  |     +-group_by=GroupBy
-  |     | +-AttributeReference[attribute_name=col4]
-  |     +-from_clause=
-  |       +-TableReference[table=bar]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-where_clause=InTableQuery
+      | +-test_expression=AttributeReference[attribute_name=col1]
+      | +-table_query=SubqueryExpression
+      |   +-SetOperation[set_operation_type=Select]
+      |     +-children=
+      |       +-Select
+      |         +-select_clause=SelectList
+      |         | +-SelectListItem
+      |         |   +-FunctionCall[name=SUM]
+      |         |     +-Add
+      |         |       +-left_operand=AttributeReference[attribute_name=col2]
+      |         |       +-right_operand=AttributeReference[attribute_name=col3]
+      |         +-group_by=GroupBy
+      |         | +-AttributeReference[attribute_name=col4]
+      |         +-from_clause=
+      |           +-TableReference[table=bar]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT *
@@ -1734,26 +1888,30 @@ WHERE col1 NOT IN (
   WHERE col3 IN (col4, col5)
 );
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-where_clause=Not
-  | +-InTableQuery
-  |   +-test_expression=AttributeReference[attribute_name=col1]
-  |   +-table_query=SubqueryExpression
-  |     +-Select
-  |       +-select_clause=SelectList
-  |       | +-SelectListItem
-  |       |   +-AttributeReference[attribute_name=col2]
-  |       +-where_clause=InValueList
-  |       | +-test_expression=AttributeReference[attribute_name=col3]
-  |       | +-value_list=
-  |       |   +-AttributeReference[attribute_name=col4]
-  |       |   +-AttributeReference[attribute_name=col5]
-  |       +-from_clause=
-  |         +-TableReference[table=bar]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-where_clause=Not
+      | +-InTableQuery
+      |   +-test_expression=AttributeReference[attribute_name=col1]
+      |   +-table_query=SubqueryExpression
+      |     +-SetOperation[set_operation_type=Select]
+      |       +-children=
+      |         +-Select
+      |           +-select_clause=SelectList
+      |           | +-SelectListItem
+      |           |   +-AttributeReference[attribute_name=col2]
+      |         

<TRUNCATED>


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

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/tests/TPCH.test
----------------------------------------------------------------------
diff --git a/parser/tests/TPCH.test b/parser/tests/TPCH.test
index f0ffaa9..9ff8232 100644
--- a/parser/tests/TPCH.test
+++ b/parser/tests/TPCH.test
@@ -38,68 +38,70 @@ ORDER BY
   l_returnflag,
   l_linestatus
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=l_returnflag]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=l_linestatus]
-  | +-SelectListItem[alias=sum_qty]
-  | | +-FunctionCall[name=SUM]
-  | |   +-AttributeReference[attribute_name=l_quantity]
-  | +-SelectListItem[alias=sum_base_price]
-  | | +-FunctionCall[name=SUM]
-  | |   +-AttributeReference[attribute_name=l_extendedprice]
-  | +-SelectListItem[alias=sum_disc_price]
-  | | +-FunctionCall[name=SUM]
-  | |   +-Multiply
-  | |     +-left_operand=AttributeReference[attribute_name=l_extendedprice]
-  | |     +-right_operand=Subtract
-  | |       +-left_operand=Literal
-  | |       | +-NumericLiteral[numeric_string=1,float_like=false]
-  | |       +-right_operand=AttributeReference[attribute_name=l_discount]
-  | +-SelectListItem[alias=sum_charge]
-  | | +-FunctionCall[name=SUM]
-  | |   +-Multiply
-  | |     +-left_operand=Multiply
-  | |     | +-left_operand=AttributeReference[attribute_name=l_extendedprice]
-  | |     | +-right_operand=Subtract
-  | |     |   +-left_operand=Literal
-  | |     |   | +-NumericLiteral[numeric_string=1,float_like=false]
-  | |     |   +-right_operand=AttributeReference[attribute_name=l_discount]
-  | |     +-right_operand=Add
-  | |       +-left_operand=Literal
-  | |       | +-NumericLiteral[numeric_string=1,float_like=false]
-  | |       +-right_operand=AttributeReference[attribute_name=l_tax]
-  | +-SelectListItem[alias=avg_qty]
-  | | +-FunctionCall[name=AVG]
-  | |   +-AttributeReference[attribute_name=l_quantity]
-  | +-SelectListItem[alias=avg_price]
-  | | +-FunctionCall[name=AVG]
-  | |   +-AttributeReference[attribute_name=l_extendedprice]
-  | +-SelectListItem[alias=avg_disc]
-  | | +-FunctionCall[name=AVG]
-  | |   +-AttributeReference[attribute_name=l_discount]
-  | +-SelectListItem[alias=count_order]
-  |   +-FunctionCall[name=COUNT,is_star=true]
-  +-where_clause=LessOrEqual
-  | +-left_operand=AttributeReference[attribute_name=l_shipdate]
-  | +-right_operand=Subtract
-  |   +-left_operand=Literal
-  |   | +-StringLiteral[value=1998-12-01,explicit_type=Date]
-  |   +-right_operand=Literal
-  |     +-StringLiteral[value=96 day,explicit_type=DatetimeInterval]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=l_returnflag]
-  | +-AttributeReference[attribute_name=l_linestatus]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  | | +-AttributeReference[attribute_name=l_returnflag]
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  |   +-AttributeReference[attribute_name=l_linestatus]
-  +-from_clause=
-    +-TableReference[table=lineitem]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=l_returnflag]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=l_linestatus]
+      | +-SelectListItem[alias=sum_qty]
+      | | +-FunctionCall[name=SUM]
+      | |   +-AttributeReference[attribute_name=l_quantity]
+      | +-SelectListItem[alias=sum_base_price]
+      | | +-FunctionCall[name=SUM]
+      | |   +-AttributeReference[attribute_name=l_extendedprice]
+      | +-SelectListItem[alias=sum_disc_price]
+      | | +-FunctionCall[name=SUM]
+      | |   +-Multiply
+      | |     +-left_operand=AttributeReference[attribute_name=l_extendedprice]
+      | |     +-right_operand=Subtract
+      | |       +-left_operand=Literal
+      | |       | +-NumericLiteral[numeric_string=1,float_like=false]
+      | |       +-right_operand=AttributeReference[attribute_name=l_discount]
+      | +-SelectListItem[alias=sum_charge]
+      | | +-FunctionCall[name=SUM]
+      | |   +-Multiply
+      | |     +-left_operand=Multiply
+      | |     | +-left_operand=AttributeReference[attribute_name=l_extendedprice]
+      | |     | +-right_operand=Subtract
+      | |     |   +-left_operand=Literal
+      | |     |   | +-NumericLiteral[numeric_string=1,float_like=false]
+      | |     |   +-right_operand=AttributeReference[attribute_name=l_discount]
+      | |     +-right_operand=Add
+      | |       +-left_operand=Literal
+      | |       | +-NumericLiteral[numeric_string=1,float_like=false]
+      | |       +-right_operand=AttributeReference[attribute_name=l_tax]
+      | +-SelectListItem[alias=avg_qty]
+      | | +-FunctionCall[name=AVG]
+      | |   +-AttributeReference[attribute_name=l_quantity]
+      | +-SelectListItem[alias=avg_price]
+      | | +-FunctionCall[name=AVG]
+      | |   +-AttributeReference[attribute_name=l_extendedprice]
+      | +-SelectListItem[alias=avg_disc]
+      | | +-FunctionCall[name=AVG]
+      | |   +-AttributeReference[attribute_name=l_discount]
+      | +-SelectListItem[alias=count_order]
+      |   +-FunctionCall[name=COUNT,is_star=true]
+      +-where_clause=LessOrEqual
+      | +-left_operand=AttributeReference[attribute_name=l_shipdate]
+      | +-right_operand=Subtract
+      |   +-left_operand=Literal
+      |   | +-StringLiteral[value=1998-12-01,explicit_type=Date]
+      |   +-right_operand=Literal
+      |     +-StringLiteral[value=96 day,explicit_type=DatetimeInterval]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=l_returnflag]
+      | +-AttributeReference[attribute_name=l_linestatus]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      | | +-AttributeReference[attribute_name=l_returnflag]
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      |   +-AttributeReference[attribute_name=l_linestatus]
+      +-from_clause=
+        +-TableReference[table=lineitem]
 ==
 
 # Query 2
@@ -148,97 +150,107 @@ ORDER BY
   p_partkey
 LIMIT 100
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=s_acctbal]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=s_name]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=n_name]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=p_partkey]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=p_mfgr]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=s_address]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=s_phone]
-  | +-SelectListItem
-  |   +-AttributeReference[attribute_name=s_comment]
-  +-where_clause=And
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=p_partkey]
-  | | +-right_operand=AttributeReference[attribute_name=ps_partkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
-  | | +-right_operand=AttributeReference[attribute_name=ps_suppkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=p_size]
-  | | +-right_operand=Literal
-  | |   +-NumericLiteral[numeric_string=48,float_like=false]
-  | +-Like
-  | | +-left_operand=AttributeReference[attribute_name=p_type]
-  | | +-right_operand=Literal
-  | |   +-StringLiteral[value=%NICKEL]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
-  | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=n_regionkey]
-  | | +-right_operand=AttributeReference[attribute_name=r_regionkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=r_name]
-  | | +-right_operand=Literal
-  | |   +-StringLiteral[value=ASIA]
-  | +-Equal
-  |   +-left_operand=AttributeReference[attribute_name=ps_supplycost]
-  |   +-right_operand=SubqueryExpression
-  |     +-Select
-  |       +-select_clause=SelectList
-  |       | +-SelectListItem
-  |       |   +-FunctionCall[name=MIN]
-  |       |     +-AttributeReference[attribute_name=ps_supplycost]
-  |       +-where_clause=And
-  |       | +-Equal
-  |       | | +-left_operand=AttributeReference[attribute_name=p_partkey]
-  |       | | +-right_operand=AttributeReference[attribute_name=ps_partkey]
-  |       | +-Equal
-  |       | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
-  |       | | +-right_operand=AttributeReference[attribute_name=ps_suppkey]
-  |       | +-Equal
-  |       | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
-  |       | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
-  |       | +-Equal
-  |       | | +-left_operand=AttributeReference[attribute_name=n_regionkey]
-  |       | | +-right_operand=AttributeReference[attribute_name=r_regionkey]
-  |       | +-Equal
-  |       |   +-left_operand=AttributeReference[attribute_name=r_name]
-  |       |   +-right_operand=Literal
-  |       |     +-StringLiteral[value=ASIA]
-  |       +-from_clause=
-  |         +-TableReference[table=partsupp]
-  |         +-TableReference[table=supplier]
-  |         +-TableReference[table=nation]
-  |         +-TableReference[table=region]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=false,nulls_first=true]
-  | | +-AttributeReference[attribute_name=s_acctbal]
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  | | +-AttributeReference[attribute_name=n_name]
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  | | +-AttributeReference[attribute_name=s_name]
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  |   +-AttributeReference[attribute_name=p_partkey]
-  +-limit=LIMIT
-  | +-NumericLiteral[numeric_string=100,float_like=false]
-  +-from_clause=
-    +-TableReference[table=part]
-    +-TableReference[table=supplier]
-    +-TableReference[table=partsupp]
-    +-TableReference[table=nation]
-    +-TableReference[table=region]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=s_acctbal]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=s_name]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=n_name]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=p_partkey]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=p_mfgr]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=s_address]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=s_phone]
+      | +-SelectListItem
+      |   +-AttributeReference[attribute_name=s_comment]
+      +-where_clause=And
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=p_partkey]
+      | | +-right_operand=AttributeReference[attribute_name=ps_partkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
+      | | +-right_operand=AttributeReference[attribute_name=ps_suppkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=p_size]
+      | | +-right_operand=Literal
+      | |   +-NumericLiteral[numeric_string=48,float_like=false]
+      | +-Like
+      | | +-left_operand=AttributeReference[attribute_name=p_type]
+      | | +-right_operand=Literal
+      | |   +-StringLiteral[value=%NICKEL]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
+      | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=n_regionkey]
+      | | +-right_operand=AttributeReference[attribute_name=r_regionkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=r_name]
+      | | +-right_operand=Literal
+      | |   +-StringLiteral[value=ASIA]
+      | +-Equal
+      |   +-left_operand=AttributeReference[attribute_name=ps_supplycost]
+      |   +-right_operand=SubqueryExpression
+      |     +-SetOperation[set_operation_type=Select]
+      |       +-children=
+      |         +-Select
+      |           +-select_clause=SelectList
+      |           | +-SelectListItem
+      |           |   +-FunctionCall[name=MIN]
+      |           |     +-AttributeReference[attribute_name=ps_supplycost]
+      |           +-where_clause=And
+      |           | +-Equal
+      |           | | +-left_operand=AttributeReference[attribute_name=p_partkey]
+      |           | | +-right_operand=AttributeReference[
+      |           | |   attribute_name=ps_partkey]
+      |           | +-Equal
+      |           | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
+      |           | | +-right_operand=AttributeReference[
+      |           | |   attribute_name=ps_suppkey]
+      |           | +-Equal
+      |           | | +-left_operand=AttributeReference[
+      |           | | | attribute_name=s_nationkey]
+      |           | | +-right_operand=AttributeReference[
+      |           | |   attribute_name=n_nationkey]
+      |           | +-Equal
+      |           | | +-left_operand=AttributeReference[
+      |           | | | attribute_name=n_regionkey]
+      |           | | +-right_operand=AttributeReference[
+      |           | |   attribute_name=r_regionkey]
+      |           | +-Equal
+      |           |   +-left_operand=AttributeReference[attribute_name=r_name]
+      |           |   +-right_operand=Literal
+      |           |     +-StringLiteral[value=ASIA]
+      |           +-from_clause=
+      |             +-TableReference[table=partsupp]
+      |             +-TableReference[table=supplier]
+      |             +-TableReference[table=nation]
+      |             +-TableReference[table=region]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=false,nulls_first=true]
+      | | +-AttributeReference[attribute_name=s_acctbal]
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      | | +-AttributeReference[attribute_name=n_name]
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      | | +-AttributeReference[attribute_name=s_name]
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      |   +-AttributeReference[attribute_name=p_partkey]
+      +-limit=LIMIT
+      | +-NumericLiteral[numeric_string=100,float_like=false]
+      +-from_clause=
+        +-TableReference[table=part]
+        +-TableReference[table=supplier]
+        +-TableReference[table=partsupp]
+        +-TableReference[table=nation]
+        +-TableReference[table=region]
 ==
 
 # Query 3
@@ -266,57 +278,59 @@ ORDER BY
   o_orderdate
 LIMIT 10
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=l_orderkey]
-  | +-SelectListItem[alias=revenue]
-  | | +-FunctionCall[name=SUM]
-  | |   +-Multiply
-  | |     +-left_operand=AttributeReference[attribute_name=l_extendedprice]
-  | |     +-right_operand=Subtract
-  | |       +-left_operand=Literal
-  | |       | +-NumericLiteral[numeric_string=1,float_like=false]
-  | |       +-right_operand=AttributeReference[attribute_name=l_discount]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=o_orderdate]
-  | +-SelectListItem
-  |   +-AttributeReference[attribute_name=o_shippriority]
-  +-where_clause=And
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=c_mktsegment]
-  | | +-right_operand=Literal
-  | |   +-StringLiteral[value=AUTOMOBILE]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=c_custkey]
-  | | +-right_operand=AttributeReference[attribute_name=o_custkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=l_orderkey]
-  | | +-right_operand=AttributeReference[attribute_name=o_orderkey]
-  | +-Less
-  | | +-left_operand=AttributeReference[attribute_name=o_orderdate]
-  | | +-right_operand=Literal
-  | |   +-StringLiteral[value=1995-03-17,explicit_type=Date]
-  | +-Greater
-  |   +-left_operand=AttributeReference[attribute_name=l_shipdate]
-  |   +-right_operand=Literal
-  |     +-StringLiteral[value=1995-03-17,explicit_type=Date]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=l_orderkey]
-  | +-AttributeReference[attribute_name=o_orderdate]
-  | +-AttributeReference[attribute_name=o_shippriority]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=false,nulls_first=true]
-  | | +-AttributeReference[attribute_name=revenue]
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  |   +-AttributeReference[attribute_name=o_orderdate]
-  +-limit=LIMIT
-  | +-NumericLiteral[numeric_string=10,float_like=false]
-  +-from_clause=
-    +-TableReference[table=customer]
-    +-TableReference[table=orders]
-    +-TableReference[table=lineitem]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=l_orderkey]
+      | +-SelectListItem[alias=revenue]
+      | | +-FunctionCall[name=SUM]
+      | |   +-Multiply
+      | |     +-left_operand=AttributeReference[attribute_name=l_extendedprice]
+      | |     +-right_operand=Subtract
+      | |       +-left_operand=Literal
+      | |       | +-NumericLiteral[numeric_string=1,float_like=false]
+      | |       +-right_operand=AttributeReference[attribute_name=l_discount]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=o_orderdate]
+      | +-SelectListItem
+      |   +-AttributeReference[attribute_name=o_shippriority]
+      +-where_clause=And
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=c_mktsegment]
+      | | +-right_operand=Literal
+      | |   +-StringLiteral[value=AUTOMOBILE]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=c_custkey]
+      | | +-right_operand=AttributeReference[attribute_name=o_custkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=l_orderkey]
+      | | +-right_operand=AttributeReference[attribute_name=o_orderkey]
+      | +-Less
+      | | +-left_operand=AttributeReference[attribute_name=o_orderdate]
+      | | +-right_operand=Literal
+      | |   +-StringLiteral[value=1995-03-17,explicit_type=Date]
+      | +-Greater
+      |   +-left_operand=AttributeReference[attribute_name=l_shipdate]
+      |   +-right_operand=Literal
+      |     +-StringLiteral[value=1995-03-17,explicit_type=Date]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=l_orderkey]
+      | +-AttributeReference[attribute_name=o_orderdate]
+      | +-AttributeReference[attribute_name=o_shippriority]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=false,nulls_first=true]
+      | | +-AttributeReference[attribute_name=revenue]
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      |   +-AttributeReference[attribute_name=o_orderdate]
+      +-limit=LIMIT
+      | +-NumericLiteral[numeric_string=10,float_like=false]
+      +-from_clause=
+        +-TableReference[table=customer]
+        +-TableReference[table=orders]
+        +-TableReference[table=lineitem]
 ==
 
 # Query 4
@@ -342,45 +356,53 @@ GROUP BY
 ORDER BY
   o_orderpriority
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=o_orderpriority]
-  | +-SelectListItem[alias=order_count]
-  |   +-FunctionCall[name=COUNT,is_star=true]
-  +-where_clause=And
-  | +-GreaterOrEqual
-  | | +-left_operand=AttributeReference[attribute_name=o_orderdate]
-  | | +-right_operand=Literal
-  | |   +-StringLiteral[value=1995-08-01,explicit_type=Date]
-  | +-Less
-  | | +-left_operand=AttributeReference[attribute_name=o_orderdate]
-  | | +-right_operand=Add
-  | |   +-left_operand=Literal
-  | |   | +-StringLiteral[value=1995-08-01,explicit_type=Date]
-  | |   +-right_operand=Literal
-  | |     +-StringLiteral[value=3 month,explicit_type=YearMonthInterval]
-  | +-Exists
-  |   +-subquery=SubqueryExpression
-  |     +-Select
-  |       +-select_clause=SelectStar
-  |       +-where_clause=And
-  |       | +-Equal
-  |       | | +-left_operand=AttributeReference[attribute_name=l_orderkey]
-  |       | | +-right_operand=AttributeReference[attribute_name=o_orderkey]
-  |       | +-Less
-  |       |   +-left_operand=AttributeReference[attribute_name=l_commitdate]
-  |       |   +-right_operand=AttributeReference[attribute_name=l_receiptdate]
-  |       +-from_clause=
-  |         +-TableReference[table=lineitem]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=o_orderpriority]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  |   +-AttributeReference[attribute_name=o_orderpriority]
-  +-from_clause=
-    +-TableReference[table=orders]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=o_orderpriority]
+      | +-SelectListItem[alias=order_count]
+      |   +-FunctionCall[name=COUNT,is_star=true]
+      +-where_clause=And
+      | +-GreaterOrEqual
+      | | +-left_operand=AttributeReference[attribute_name=o_orderdate]
+      | | +-right_operand=Literal
+      | |   +-StringLiteral[value=1995-08-01,explicit_type=Date]
+      | +-Less
+      | | +-left_operand=AttributeReference[attribute_name=o_orderdate]
+      | | +-right_operand=Add
+      | |   +-left_operand=Literal
+      | |   | +-StringLiteral[value=1995-08-01,explicit_type=Date]
+      | |   +-right_operand=Literal
+      | |     +-StringLiteral[value=3 month,explicit_type=YearMonthInterval]
+      | +-Exists
+      |   +-subquery=SubqueryExpression
+      |     +-SetOperation[set_operation_type=Select]
+      |       +-children=
+      |         +-Select
+      |           +-select_clause=SelectStar
+      |           +-where_clause=And
+      |           | +-Equal
+      |           | | +-left_operand=AttributeReference[
+      |           | | | attribute_name=l_orderkey]
+      |           | | +-right_operand=AttributeReference[
+      |           | |   attribute_name=o_orderkey]
+      |           | +-Less
+      |           |   +-left_operand=AttributeReference[
+      |           |   | attribute_name=l_commitdate]
+      |           |   +-right_operand=AttributeReference[
+      |           |     attribute_name=l_receiptdate]
+      |           +-from_clause=
+      |             +-TableReference[table=lineitem]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=o_orderpriority]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      |   +-AttributeReference[attribute_name=o_orderpriority]
+      +-from_clause=
+        +-TableReference[table=orders]
 ==
 
 # Query 5
@@ -409,65 +431,67 @@ GROUP BY
 ORDER BY
   revenue DESC
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=n_name]
-  | +-SelectListItem[alias=revenue]
-  |   +-FunctionCall[name=SUM]
-  |     +-Multiply
-  |       +-left_operand=AttributeReference[attribute_name=l_extendedprice]
-  |       +-right_operand=Subtract
-  |         +-left_operand=Literal
-  |         | +-NumericLiteral[numeric_string=1,float_like=false]
-  |         +-right_operand=AttributeReference[attribute_name=l_discount]
-  +-where_clause=And
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=c_custkey]
-  | | +-right_operand=AttributeReference[attribute_name=o_custkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=l_orderkey]
-  | | +-right_operand=AttributeReference[attribute_name=o_orderkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=l_suppkey]
-  | | +-right_operand=AttributeReference[attribute_name=s_suppkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=c_nationkey]
-  | | +-right_operand=AttributeReference[attribute_name=s_nationkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
-  | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=n_regionkey]
-  | | +-right_operand=AttributeReference[attribute_name=r_regionkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=r_name]
-  | | +-right_operand=Literal
-  | |   +-StringLiteral[value=AMERICA]
-  | +-GreaterOrEqual
-  | | +-left_operand=AttributeReference[attribute_name=o_orderdate]
-  | | +-right_operand=Literal
-  | |   +-StringLiteral[value=1997-01-01,explicit_type=Date]
-  | +-Less
-  |   +-left_operand=AttributeReference[attribute_name=o_orderdate]
-  |   +-right_operand=Add
-  |     +-left_operand=Literal
-  |     | +-StringLiteral[value=1997-01-01,explicit_type=Date]
-  |     +-right_operand=Literal
-  |       +-StringLiteral[value=1 year,explicit_type=YearMonthInterval]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=n_name]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=false,nulls_first=true]
-  |   +-AttributeReference[attribute_name=revenue]
-  +-from_clause=
-    +-TableReference[table=customer]
-    +-TableReference[table=orders]
-    +-TableReference[table=lineitem]
-    +-TableReference[table=supplier]
-    +-TableReference[table=nation]
-    +-TableReference[table=region]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=n_name]
+      | +-SelectListItem[alias=revenue]
+      |   +-FunctionCall[name=SUM]
+      |     +-Multiply
+      |       +-left_operand=AttributeReference[attribute_name=l_extendedprice]
+      |       +-right_operand=Subtract
+      |         +-left_operand=Literal
+      |         | +-NumericLiteral[numeric_string=1,float_like=false]
+      |         +-right_operand=AttributeReference[attribute_name=l_discount]
+      +-where_clause=And
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=c_custkey]
+      | | +-right_operand=AttributeReference[attribute_name=o_custkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=l_orderkey]
+      | | +-right_operand=AttributeReference[attribute_name=o_orderkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=l_suppkey]
+      | | +-right_operand=AttributeReference[attribute_name=s_suppkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=c_nationkey]
+      | | +-right_operand=AttributeReference[attribute_name=s_nationkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
+      | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=n_regionkey]
+      | | +-right_operand=AttributeReference[attribute_name=r_regionkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=r_name]
+      | | +-right_operand=Literal
+      | |   +-StringLiteral[value=AMERICA]
+      | +-GreaterOrEqual
+      | | +-left_operand=AttributeReference[attribute_name=o_orderdate]
+      | | +-right_operand=Literal
+      | |   +-StringLiteral[value=1997-01-01,explicit_type=Date]
+      | +-Less
+      |   +-left_operand=AttributeReference[attribute_name=o_orderdate]
+      |   +-right_operand=Add
+      |     +-left_operand=Literal
+      |     | +-StringLiteral[value=1997-01-01,explicit_type=Date]
+      |     +-right_operand=Literal
+      |       +-StringLiteral[value=1 year,explicit_type=YearMonthInterval]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=n_name]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=false,nulls_first=true]
+      |   +-AttributeReference[attribute_name=revenue]
+      +-from_clause=
+        +-TableReference[table=customer]
+        +-TableReference[table=orders]
+        +-TableReference[table=lineitem]
+        +-TableReference[table=supplier]
+        +-TableReference[table=nation]
+        +-TableReference[table=region]
 ==
 
 # Query 6
@@ -481,44 +505,46 @@ WHERE
   AND l_discount BETWEEN 0.03 - 0.01 AND 0.03 + 0.01
   AND l_quantity < 25
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem[alias=revenue]
-  |   +-FunctionCall[name=SUM]
-  |     +-Multiply
-  |       +-left_operand=AttributeReference[attribute_name=l_extendedprice]
-  |       +-right_operand=AttributeReference[attribute_name=l_discount]
-  +-where_clause=And
-  | +-GreaterOrEqual
-  | | +-left_operand=AttributeReference[attribute_name=l_shipdate]
-  | | +-right_operand=Literal
-  | |   +-StringLiteral[value=1997-01-01,explicit_type=Date]
-  | +-Less
-  | | +-left_operand=AttributeReference[attribute_name=l_shipdate]
-  | | +-right_operand=Add
-  | |   +-left_operand=Literal
-  | |   | +-StringLiteral[value=1997-01-01,explicit_type=Date]
-  | |   +-right_operand=Literal
-  | |     +-StringLiteral[value=1 year,explicit_type=YearMonthInterval]
-  | +-Between
-  | | +-check_operand=AttributeReference[attribute_name=l_discount]
-  | | +-lower_bound_operand=Subtract
-  | | | +-left_operand=Literal
-  | | | | +-NumericLiteral[numeric_string=0.03,float_like=true]
-  | | | +-right_operand=Literal
-  | | |   +-NumericLiteral[numeric_string=0.01,float_like=true]
-  | | +-upper_bound_operand=Add
-  | |   +-left_operand=Literal
-  | |   | +-NumericLiteral[numeric_string=0.03,float_like=true]
-  | |   +-right_operand=Literal
-  | |     +-NumericLiteral[numeric_string=0.01,float_like=true]
-  | +-Less
-  |   +-left_operand=AttributeReference[attribute_name=l_quantity]
-  |   +-right_operand=Literal
-  |     +-NumericLiteral[numeric_string=25,float_like=false]
-  +-from_clause=
-    +-TableReference[table=lineitem]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem[alias=revenue]
+      |   +-FunctionCall[name=SUM]
+      |     +-Multiply
+      |       +-left_operand=AttributeReference[attribute_name=l_extendedprice]
+      |       +-right_operand=AttributeReference[attribute_name=l_discount]
+      +-where_clause=And
+      | +-GreaterOrEqual
+      | | +-left_operand=AttributeReference[attribute_name=l_shipdate]
+      | | +-right_operand=Literal
+      | |   +-StringLiteral[value=1997-01-01,explicit_type=Date]
+      | +-Less
+      | | +-left_operand=AttributeReference[attribute_name=l_shipdate]
+      | | +-right_operand=Add
+      | |   +-left_operand=Literal
+      | |   | +-StringLiteral[value=1997-01-01,explicit_type=Date]
+      | |   +-right_operand=Literal
+      | |     +-StringLiteral[value=1 year,explicit_type=YearMonthInterval]
+      | +-Between
+      | | +-check_operand=AttributeReference[attribute_name=l_discount]
+      | | +-lower_bound_operand=Subtract
+      | | | +-left_operand=Literal
+      | | | | +-NumericLiteral[numeric_string=0.03,float_like=true]
+      | | | +-right_operand=Literal
+      | | |   +-NumericLiteral[numeric_string=0.01,float_like=true]
+      | | +-upper_bound_operand=Add
+      | |   +-left_operand=Literal
+      | |   | +-NumericLiteral[numeric_string=0.03,float_like=true]
+      | |   +-right_operand=Literal
+      | |     +-NumericLiteral[numeric_string=0.01,float_like=true]
+      | +-Less
+      |   +-left_operand=AttributeReference[attribute_name=l_quantity]
+      |   +-right_operand=Literal
+      |     +-NumericLiteral[numeric_string=25,float_like=false]
+      +-from_clause=
+        +-TableReference[table=lineitem]
 ==
 
 # Query 7
@@ -562,105 +588,121 @@ ORDER BY
   cust_nation,
   l_year
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=supp_nation]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=cust_nation]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=l_year]
-  | +-SelectListItem[alias=revenue]
-  |   +-FunctionCall[name=SUM]
-  |     +-AttributeReference[attribute_name=volume]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=supp_nation]
-  | +-AttributeReference[attribute_name=cust_nation]
-  | +-AttributeReference[attribute_name=l_year]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  | | +-AttributeReference[attribute_name=supp_nation]
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  | | +-AttributeReference[attribute_name=cust_nation]
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  |   +-AttributeReference[attribute_name=l_year]
-  +-from_clause=
-    +-SubqueryTable
-      +-table_signature=TableSignature[table_alias=shipping]
-      +-SubqueryExpression
-        +-Select
-          +-select_clause=SelectList
-          | +-SelectListItem[alias=supp_nation]
-          | | +-AttributeReference[attribute_name=n_name,relation_name=n1]
-          | +-SelectListItem[alias=cust_nation]
-          | | +-AttributeReference[attribute_name=n_name,relation_name=n2]
-          | +-SelectListItem[alias=l_year]
-          | | +-Extract[unit=YEAR]
-          | |   +-date_expression=AttributeReference[attribute_name=l_shipdate]
-          | +-SelectListItem[alias=volume]
-          |   +-Multiply
-          |     +-left_operand=AttributeReference[attribute_name=l_extendedprice]
-          |     +-right_operand=Subtract
-          |       +-left_operand=Literal
-          |       | +-NumericLiteral[numeric_string=1,float_like=false]
-          |       +-right_operand=AttributeReference[attribute_name=l_discount]
-          +-where_clause=And
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
-          | | +-right_operand=AttributeReference[attribute_name=l_suppkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=o_orderkey]
-          | | +-right_operand=AttributeReference[attribute_name=l_orderkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=c_custkey]
-          | | +-right_operand=AttributeReference[attribute_name=o_custkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
-          | | +-right_operand=AttributeReference[attribute_name=n_nationkey,
-          | |   relation_name=n1]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=c_nationkey]
-          | | +-right_operand=AttributeReference[attribute_name=n_nationkey,
-          | |   relation_name=n2]
-          | +-Or
-          | | +-And
-          | | | +-Equal
-          | | | | +-left_operand=AttributeReference[attribute_name=n_name,
-          | | | | | relation_name=n1]
-          | | | | +-right_operand=Literal
-          | | | |   +-StringLiteral[value=ETHIOPIA]
-          | | | +-Equal
-          | | |   +-left_operand=AttributeReference[attribute_name=n_name,
-          | | |   | relation_name=n2]
-          | | |   +-right_operand=Literal
-          | | |     +-StringLiteral[value=UNITED STATES]
-          | | +-And
-          | |   +-Equal
-          | |   | +-left_operand=AttributeReference[attribute_name=n_name,
-          | |   | | relation_name=n1]
-          | |   | +-right_operand=Literal
-          | |   |   +-StringLiteral[value=UNITED STATES]
-          | |   +-Equal
-          | |     +-left_operand=AttributeReference[attribute_name=n_name,
-          | |     | relation_name=n2]
-          | |     +-right_operand=Literal
-          | |       +-StringLiteral[value=ETHIOPIA]
-          | +-Between
-          |   +-check_operand=AttributeReference[attribute_name=l_shipdate]
-          |   +-lower_bound_operand=Literal
-          |   | +-StringLiteral[value=1995-01-01,explicit_type=Date]
-          |   +-upper_bound_operand=Literal
-          |     +-StringLiteral[value=1996-12-31,explicit_type=Date]
-          +-from_clause=
-            +-TableReference[table=supplier]
-            +-TableReference[table=lineitem]
-            +-TableReference[table=orders]
-            +-TableReference[table=customer]
-            +-TableReference[table=nation]
-            | +-table_signature=TableSignature[table_alias=n1]
-            +-TableReference[table=nation]
-              +-table_signature=TableSignature[table_alias=n2]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=supp_nation]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=cust_nation]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=l_year]
+      | +-SelectListItem[alias=revenue]
+      |   +-FunctionCall[name=SUM]
+      |     +-AttributeReference[attribute_name=volume]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=supp_nation]
+      | +-AttributeReference[attribute_name=cust_nation]
+      | +-AttributeReference[attribute_name=l_year]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      | | +-AttributeReference[attribute_name=supp_nation]
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      | | +-AttributeReference[attribute_name=cust_nation]
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      |   +-AttributeReference[attribute_name=l_year]
+      +-from_clause=
+        +-SubqueryTable
+          +-table_signature=TableSignature[table_alias=shipping]
+          +-SubqueryExpression
+            +-SetOperation[set_operation_type=Select]
+              +-children=
+                +-Select
+                  +-select_clause=SelectList
+                  | +-SelectListItem[alias=supp_nation]
+                  | | +-AttributeReference[attribute_name=n_name,
+                  | |   relation_name=n1]
+                  | +-SelectListItem[alias=cust_nation]
+                  | | +-AttributeReference[attribute_name=n_name,
+                  | |   relation_name=n2]
+                  | +-SelectListItem[alias=l_year]
+                  | | +-Extract[unit=YEAR]
+                  | |   +-date_expression=AttributeReference[
+                  | |     attribute_name=l_shipdate]
+                  | +-SelectListItem[alias=volume]
+                  |   +-Multiply
+                  |     +-left_operand=AttributeReference[
+                  |     | attribute_name=l_extendedprice]
+                  |     +-right_operand=Subtract
+                  |       +-left_operand=Literal
+                  |       | +-NumericLiteral[numeric_string=1,float_like=false]
+                  |       +-right_operand=AttributeReference[
+                  |         attribute_name=l_discount]
+                  +-where_clause=And
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=l_suppkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[
+                  | | | attribute_name=o_orderkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=l_orderkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[attribute_name=c_custkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=o_custkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[
+                  | | | attribute_name=s_nationkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=n_nationkey,relation_name=n1]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[
+                  | | | attribute_name=c_nationkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=n_nationkey,relation_name=n2]
+                  | +-Or
+                  | | +-And
+                  | | | +-Equal
+                  | | | | +-left_operand=AttributeReference[
+                  | | | | | attribute_name=n_name,relation_name=n1]
+                  | | | | +-right_operand=Literal
+                  | | | |   +-StringLiteral[value=ETHIOPIA]
+                  | | | +-Equal
+                  | | |   +-left_operand=AttributeReference[
+                  | | |   | attribute_name=n_name,relation_name=n2]
+                  | | |   +-right_operand=Literal
+                  | | |     +-StringLiteral[value=UNITED STATES]
+                  | | +-And
+                  | |   +-Equal
+                  | |   | +-left_operand=AttributeReference[
+                  | |   | | attribute_name=n_name,relation_name=n1]
+                  | |   | +-right_operand=Literal
+                  | |   |   +-StringLiteral[value=UNITED STATES]
+                  | |   +-Equal
+                  | |     +-left_operand=AttributeReference[
+                  | |     | attribute_name=n_name,relation_name=n2]
+                  | |     +-right_operand=Literal
+                  | |       +-StringLiteral[value=ETHIOPIA]
+                  | +-Between
+                  |   +-check_operand=AttributeReference[
+                  |   | attribute_name=l_shipdate]
+                  |   +-lower_bound_operand=Literal
+                  |   | +-StringLiteral[value=1995-01-01,explicit_type=Date]
+                  |   +-upper_bound_operand=Literal
+                  |     +-StringLiteral[value=1996-12-31,explicit_type=Date]
+                  +-from_clause=
+                    +-TableReference[table=supplier]
+                    +-TableReference[table=lineitem]
+                    +-TableReference[table=orders]
+                    +-TableReference[table=customer]
+                    +-TableReference[table=nation]
+                    | +-table_signature=TableSignature[table_alias=n1]
+                    +-TableReference[table=nation]
+                      +-table_signature=TableSignature[table_alias=n2]
 ==
 
 # Query 8
@@ -702,99 +744,116 @@ GROUP BY
 ORDER BY
   o_year
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=o_year]
-  | +-SelectListItem[alias=mkt_share]
-  |   +-Divide
-  |     +-left_operand=FunctionCall[name=SUM]
-  |     | +-SearchedCaseExpression
-  |     |   +-else_result_expression=Literal
-  |     |   | +-NumericLiteral[numeric_string=0,float_like=false]
-  |     |   +-when_clauses=
-  |     |     +-SearchedWhenClause
-  |     |       +-condition_predicate=Equal
-  |     |       | +-left_operand=AttributeReference[attribute_name=nation]
-  |     |       | +-right_operand=Literal
-  |     |       |   +-StringLiteral[value=UNITED STATES]
-  |     |       +-result_expression=AttributeReference[attribute_name=volume]
-  |     +-right_operand=FunctionCall[name=SUM]
-  |       +-AttributeReference[attribute_name=volume]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=o_year]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  |   +-AttributeReference[attribute_name=o_year]
-  +-from_clause=
-    +-SubqueryTable
-      +-table_signature=TableSignature[table_alias=all_nations]
-      +-SubqueryExpression
-        +-Select
-          +-select_clause=SelectList
-          | +-SelectListItem[alias=o_year]
-          | | +-Extract[unit=YEAR]
-          | |   +-date_expression=AttributeReference[attribute_name=o_orderdate]
-          | +-SelectListItem[alias=volume]
-          | | +-Multiply
-          | |   +-left_operand=AttributeReference[attribute_name=l_extendedprice]
-          | |   +-right_operand=Subtract
-          | |     +-left_operand=Literal
-          | |     | +-NumericLiteral[numeric_string=1,float_like=false]
-          | |     +-right_operand=AttributeReference[attribute_name=l_discount]
-          | +-SelectListItem[alias=nation]
-          |   +-AttributeReference[attribute_name=n_name,relation_name=n2]
-          +-where_clause=And
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=p_partkey]
-          | | +-right_operand=AttributeReference[attribute_name=l_partkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
-          | | +-right_operand=AttributeReference[attribute_name=l_suppkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=l_orderkey]
-          | | +-right_operand=AttributeReference[attribute_name=o_orderkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=o_custkey]
-          | | +-right_operand=AttributeReference[attribute_name=c_custkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=c_nationkey]
-          | | +-right_operand=AttributeReference[attribute_name=n_nationkey,
-          | |   relation_name=n1]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=n_regionkey,
-          | | | relation_name=n1]
-          | | +-right_operand=AttributeReference[attribute_name=r_regionkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=r_name]
-          | | +-right_operand=Literal
-          | |   +-StringLiteral[value=AMERICA]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
-          | | +-right_operand=AttributeReference[attribute_name=n_nationkey,
-          | |   relation_name=n2]
-          | +-Between
-          | | +-check_operand=AttributeReference[attribute_name=o_orderdate]
-          | | +-lower_bound_operand=Literal
-          | | | +-StringLiteral[value=1995-01-01,explicit_type=Date]
-          | | +-upper_bound_operand=Literal
-          | |   +-StringLiteral[value=1996-12-31,explicit_type=Date]
-          | +-Equal
-          |   +-left_operand=AttributeReference[attribute_name=p_type]
-          |   +-right_operand=Literal
-          |     +-StringLiteral[value=MEDIUM ANODIZED NICKEL]
-          +-from_clause=
-            +-TableReference[table=part]
-            +-TableReference[table=supplier]
-            +-TableReference[table=lineitem]
-            +-TableReference[table=orders]
-            +-TableReference[table=customer]
-            +-TableReference[table=nation]
-            | +-table_signature=TableSignature[table_alias=n1]
-            +-TableReference[table=nation]
-            | +-table_signature=TableSignature[table_alias=n2]
-            +-TableReference[table=region]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=o_year]
+      | +-SelectListItem[alias=mkt_share]
+      |   +-Divide
+      |     +-left_operand=FunctionCall[name=SUM]
+      |     | +-SearchedCaseExpression
+      |     |   +-else_result_expression=Literal
+      |     |   | +-NumericLiteral[numeric_string=0,float_like=false]
+      |     |   +-when_clauses=
+      |     |     +-SearchedWhenClause
+      |     |       +-condition_predicate=Equal
+      |     |       | +-left_operand=AttributeReference[attribute_name=nation]
+      |     |       | +-right_operand=Literal
+      |     |       |   +-StringLiteral[value=UNITED STATES]
+      |     |       +-result_expression=AttributeReference[attribute_name=volume]
+      |     +-right_operand=FunctionCall[name=SUM]
+      |       +-AttributeReference[attribute_name=volume]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=o_year]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      |   +-AttributeReference[attribute_name=o_year]
+      +-from_clause=
+        +-SubqueryTable
+          +-table_signature=TableSignature[table_alias=all_nations]
+          +-SubqueryExpression
+            +-SetOperation[set_operation_type=Select]
+              +-children=
+                +-Select
+                  +-select_clause=SelectList
+                  | +-SelectListItem[alias=o_year]
+                  | | +-Extract[unit=YEAR]
+                  | |   +-date_expression=AttributeReference[
+                  | |     attribute_name=o_orderdate]
+                  | +-SelectListItem[alias=volume]
+                  | | +-Multiply
+                  | |   +-left_operand=AttributeReference[
+                  | |   | attribute_name=l_extendedprice]
+                  | |   +-right_operand=Subtract
+                  | |     +-left_operand=Literal
+                  | |     | +-NumericLiteral[numeric_string=1,float_like=false]
+                  | |     +-right_operand=AttributeReference[
+                  | |       attribute_name=l_discount]
+                  | +-SelectListItem[alias=nation]
+                  |   +-AttributeReference[attribute_name=n_name,
+                  |     relation_name=n2]
+                  +-where_clause=And
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[attribute_name=p_partkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=l_partkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=l_suppkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[
+                  | | | attribute_name=l_orderkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=o_orderkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[attribute_name=o_custkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=c_custkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[
+                  | | | attribute_name=c_nationkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=n_nationkey,relation_name=n1]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[
+                  | | | attribute_name=n_regionkey,relation_name=n1]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=r_regionkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[attribute_name=r_name]
+                  | | +-right_operand=Literal
+                  | |   +-StringLiteral[value=AMERICA]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[
+                  | | | attribute_name=s_nationkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=n_nationkey,relation_name=n2]
+                  | +-Between
+                  | | +-check_operand=AttributeReference[
+                  | | | attribute_name=o_orderdate]
+                  | | +-lower_bound_operand=Literal
+                  | | | +-StringLiteral[value=1995-01-01,explicit_type=Date]
+                  | | +-upper_bound_operand=Literal
+                  | |   +-StringLiteral[value=1996-12-31,explicit_type=Date]
+                  | +-Equal
+                  |   +-left_operand=AttributeReference[attribute_name=p_type]
+                  |   +-right_operand=Literal
+                  |     +-StringLiteral[value=MEDIUM ANODIZED NICKEL]
+                  +-from_clause=
+                    +-TableReference[table=part]
+                    +-TableReference[table=supplier]
+                    +-TableReference[table=lineitem]
+                    +-TableReference[table=orders]
+                    +-TableReference[table=customer]
+                    +-TableReference[table=nation]
+                    | +-table_signature=TableSignature[table_alias=n1]
+                    +-TableReference[table=nation]
+                    | +-table_signature=TableSignature[table_alias=n2]
+                    +-TableReference[table=region]
 ==
 
 # Query 9
@@ -831,77 +890,95 @@ ORDER BY
   nation,
   o_year DESC
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=nation]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=o_year]
-  | +-SelectListItem[alias=sum_profit]
-  |   +-FunctionCall[name=SUM]
-  |     +-AttributeReference[attribute_name=amount]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=nation]
-  | +-AttributeReference[attribute_name=o_year]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  | | +-AttributeReference[attribute_name=nation]
-  | +-OrderByItem[is_asc=false,nulls_first=true]
-  |   +-AttributeReference[attribute_name=o_year]
-  +-from_clause=
-    +-SubqueryTable
-      +-table_signature=TableSignature[table_alias=profit]
-      +-SubqueryExpression
-        +-Select
-          +-select_clause=SelectList
-          | +-SelectListItem[alias=nation]
-          | | +-AttributeReference[attribute_name=n_name]
-          | +-SelectListItem[alias=o_year]
-          | | +-Extract[unit=YEAR]
-          | |   +-date_expression=AttributeReference[attribute_name=o_orderdate]
-          | +-SelectListItem[alias=amount]
-          |   +-Subtract
-          |     +-left_operand=Multiply
-          |     | +-left_operand=AttributeReference[
-          |     | | attribute_name=l_extendedprice]
-          |     | +-right_operand=Subtract
-          |     |   +-left_operand=Literal
-          |     |   | +-NumericLiteral[numeric_string=1,float_like=false]
-          |     |   +-right_operand=AttributeReference[attribute_name=l_discount]
-          |     +-right_operand=Multiply
-          |       +-left_operand=AttributeReference[attribute_name=ps_supplycost]
-          |       +-right_operand=AttributeReference[attribute_name=l_quantity]
-          +-where_clause=And
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
-          | | +-right_operand=AttributeReference[attribute_name=l_suppkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=ps_suppkey]
-          | | +-right_operand=AttributeReference[attribute_name=l_suppkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=ps_partkey]
-          | | +-right_operand=AttributeReference[attribute_name=l_partkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=p_partkey]
-          | | +-right_operand=AttributeReference[attribute_name=l_partkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=o_orderkey]
-          | | +-right_operand=AttributeReference[attribute_name=l_orderkey]
-          | +-Equal
-          | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
-          | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
-          | +-Like
-          |   +-left_operand=AttributeReference[attribute_name=p_name]
-          |   +-right_operand=Literal
-          |     +-StringLiteral[value=%ghost%]
-          +-from_clause=
-            +-TableReference[table=part]
-            +-TableReference[table=supplier]
-            +-TableReference[table=lineitem]
-            +-TableReference[table=partsupp]
-            +-TableReference[table=orders]
-            +-TableReference[table=nation]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=nation]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=o_year]
+      | +-SelectListItem[alias=sum_profit]
+      |   +-FunctionCall[name=SUM]
+      |     +-AttributeReference[attribute_name=amount]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=nation]
+      | +-AttributeReference[attribute_name=o_year]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      | | +-AttributeReference[attribute_name=nation]
+      | +-OrderByItem[is_asc=false,nulls_first=true]
+      |   +-AttributeReference[attribute_name=o_year]
+      +-from_clause=
+        +-SubqueryTable
+          +-table_signature=TableSignature[table_alias=profit]
+          +-SubqueryExpression
+            +-SetOperation[set_operation_type=Select]
+              +-children=
+                +-Select
+                  +-select_clause=SelectList
+                  | +-SelectListItem[alias=nation]
+                  | | +-AttributeReference[attribute_name=n_name]
+                  | +-SelectListItem[alias=o_year]
+                  | | +-Extract[unit=YEAR]
+                  | |   +-date_expression=AttributeReference[
+                  | |     attribute_name=o_orderdate]
+                  | +-SelectListItem[alias=amount]
+                  |   +-Subtract
+                  |     +-left_operand=Multiply
+                  |     | +-left_operand=AttributeReference[
+                  |     | | attribute_name=l_extendedprice]
+                  |     | +-right_operand=Subtract
+                  |     |   +-left_operand=Literal
+                  |     |   | +-NumericLiteral[numeric_string=1,float_like=false]
+                  |     |   +-right_operand=AttributeReference[
+                  |     |     attribute_name=l_discount]
+                  |     +-right_operand=Multiply
+                  |       +-left_operand=AttributeReference[
+                  |       | attribute_name=ps_supplycost]
+                  |       +-right_operand=AttributeReference[
+                  |         attribute_name=l_quantity]
+                  +-where_clause=And
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[attribute_name=s_suppkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=l_suppkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[
+                  | | | attribute_name=ps_suppkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=l_suppkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[
+                  | | | attribute_name=ps_partkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=l_partkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[attribute_name=p_partkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=l_partkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[
+                  | | | attribute_name=o_orderkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=l_orderkey]
+                  | +-Equal
+                  | | +-left_operand=AttributeReference[
+                  | | | attribute_name=s_nationkey]
+                  | | +-right_operand=AttributeReference[
+                  | |   attribute_name=n_nationkey]
+                  | +-Like
+                  |   +-left_operand=AttributeReference[attribute_name=p_name]
+                  |   +-right_operand=Literal
+                  |     +-StringLiteral[value=%ghost%]
+                  +-from_clause=
+                    +-TableReference[table=part]
+                    +-TableReference[table=supplier]
+                    +-TableReference[table=lineitem]
+                    +-TableReference[table=partsupp]
+                    +-TableReference[table=orders]
+                    +-TableReference[table=nation]
 ==
 
 # Query 10
@@ -938,74 +1015,76 @@ ORDER BY
   revenue DESC
 LIMIT 20
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=c_custkey]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=c_name]
-  | +-SelectListItem[alias=revenue]
-  | | +-FunctionCall[name=SUM]
-  | |   +-Multiply
-  | |     +-left_operand=AttributeReference[attribute_name=l_extendedprice]
-  | |     +-right_operand=Subtract
-  | |       +-left_operand=Literal
-  | |       | +-NumericLiteral[numeric_string=1,float_like=false]
-  | |       +-right_operand=AttributeReference[attribute_name=l_discount]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=c_acctbal]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=n_name]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=c_address]
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=c_phone]
-  | +-SelectListItem
-  |   +-AttributeReference[attribute_name=c_comment]
-  +-where_clause=And
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=c_custkey]
-  | | +-right_operand=AttributeReference[attribute_name=o_custkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=l_orderkey]
-  | | +-right_operand=AttributeReference[attribute_name=o_orderkey]
-  | +-GreaterOrEqual
-  | | +-left_operand=AttributeReference[attribute_name=o_orderdate]
-  | | +-right_operand=Literal
-  | |   +-StringLiteral[value=1994-03-01,explicit_type=Date]
-  | +-Less
-  | | +-left_operand=AttributeReference[attribute_name=o_orderdate]
-  | | +-right_operand=Add
-  | |   +-left_operand=Literal
-  | |   | +-StringLiteral[value=1994-03-01,explicit_type=Date]
-  | |   +-right_operand=Literal
-  | |     +-StringLiteral[value=3 month,explicit_type=YearMonthInterval]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=l_returnflag]
-  | | +-right_operand=Literal
-  | |   +-StringLiteral[value=R]
-  | +-Equal
-  |   +-left_operand=AttributeReference[attribute_name=c_nationkey]
-  |   +-right_operand=AttributeReference[attribute_name=n_nationkey]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=c_custkey]
-  | +-AttributeReference[attribute_name=c_name]
-  | +-AttributeReference[attribute_name=c_acctbal]
-  | +-AttributeReference[attribute_name=c_phone]
-  | +-AttributeReference[attribute_name=n_name]
-  | +-AttributeReference[attribute_name=c_address]
-  | +-AttributeReference[attribute_name=c_comment]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=false,nulls_first=true]
-  |   +-AttributeReference[attribute_name=revenue]
-  +-limit=LIMIT
-  | +-NumericLiteral[numeric_string=20,float_like=false]
-  +-from_clause=
-    +-TableReference[table=customer]
-    +-TableReference[table=orders]
-    +-TableReference[table=lineitem]
-    +-TableReference[table=nation]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=c_custkey]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=c_name]
+      | +-SelectListItem[alias=revenue]
+      | | +-FunctionCall[name=SUM]
+      | |   +-Multiply
+      | |     +-left_operand=AttributeReference[attribute_name=l_extendedprice]
+      | |     +-right_operand=Subtract
+      | |       +-left_operand=Literal
+      | |       | +-NumericLiteral[numeric_string=1,float_like=false]
+      | |       +-right_operand=AttributeReference[attribute_name=l_discount]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=c_acctbal]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=n_name]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=c_address]
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=c_phone]
+      | +-SelectListItem
+      |   +-AttributeReference[attribute_name=c_comment]
+      +-where_clause=And
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=c_custkey]
+      | | +-right_operand=AttributeReference[attribute_name=o_custkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=l_orderkey]
+      | | +-right_operand=AttributeReference[attribute_name=o_orderkey]
+      | +-GreaterOrEqual
+      | | +-left_operand=AttributeReference[attribute_name=o_orderdate]
+      | | +-right_operand=Literal
+      | |   +-StringLiteral[value=1994-03-01,explicit_type=Date]
+      | +-Less
+      | | +-left_operand=AttributeReference[attribute_name=o_orderdate]
+      | | +-right_operand=Add
+      | |   +-left_operand=Literal
+      | |   | +-StringLiteral[value=1994-03-01,explicit_type=Date]
+      | |   +-right_operand=Literal
+      | |     +-StringLiteral[value=3 month,explicit_type=YearMonthInterval]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=l_returnflag]
+      | | +-right_operand=Literal
+      | |   +-StringLiteral[value=R]
+      | +-Equal
+      |   +-left_operand=AttributeReference[attribute_name=c_nationkey]
+      |   +-right_operand=AttributeReference[attribute_name=n_nationkey]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=c_custkey]
+      | +-AttributeReference[attribute_name=c_name]
+      | +-AttributeReference[attribute_name=c_acctbal]
+      | +-AttributeReference[attribute_name=c_phone]
+      | +-AttributeReference[attribute_name=n_name]
+      | +-AttributeReference[attribute_name=c_address]
+      | +-AttributeReference[attribute_name=c_comment]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=false,nulls_first=true]
+      |   +-AttributeReference[attribute_name=revenue]
+      +-limit=LIMIT
+      | +-NumericLiteral[numeric_string=20,float_like=false]
+      +-from_clause=
+        +-TableReference[table=customer]
+        +-TableReference[table=orders]
+        +-TableReference[table=lineitem]
+        +-TableReference[table=nation]
 ==
 
 # Query 11
@@ -1037,70 +1116,79 @@ GROUP BY
 ORDER BY
   value DESC
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=ps_partkey]
-  | +-SelectListItem[alias=value]
-  |   +-FunctionCall[name=SUM]
-  |     +-Multiply
-  |       +-left_operand=AttributeReference[attribute_name=ps_supplycost]
-  |       +-right_operand=AttributeReference[attribute_name=ps_availqty]
-  +-where_clause=And
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=ps_suppkey]
-  | | +-right_operand=AttributeReference[attribute_name=s_suppkey]
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
-  | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
-  | +-Equal
-  |   +-left_operand=AttributeReference[attribute_name=n_name]
-  |   +-right_operand=Literal
-  |     +-StringLiteral[value=INDONESIA]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=ps_partkey]
-  +-having=HAVING
-  | +-Greater
-  |   +-left_operand=FunctionCall[name=SUM]
-  |   | +-Multiply
-  |   |   +-left_operand=AttributeReference[attribute_name=ps_supplycost]
-  |   |   +-right_operand=AttributeReference[attribute_name=ps_availqty]
-  |   +-right_operand=SubqueryExpression
-  |     +-Select
-  |       +-select_clause=SelectList
-  |       | +-SelectListItem
-  |       |   +-Multiply
-  |       |     +-left_operand=FunctionCall[name=SUM]
-  |       |     | +-Multiply
-  |       |     |   +-left_operand=AttributeReference[
-  |       |     |   | attribute_name=ps_supplycost]
-  |       |     |   +-right_operand=AttributeReference[
-  |       |     |     attribute_name=ps_availqty]
-  |       |     +-right_operand=Literal
-  |       |       +-NumericLiteral[numeric_string=0.0000010000,float_like=true]
-  |       +-where_clause=And
-  |       | +-Equal
-  |       | | +-left_operand=AttributeReference[attribute_name=ps_suppkey]
-  |       | | +-right_operand=AttributeReference[attribute_name=s_suppkey]
-  |       | +-Equal
-  |       | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
-  |       | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
-  |       | +-Equal
-  |       |   +-left_operand=AttributeReference[attribute_name=n_name]
-  |       |   +-right_operand=Literal
-  |       |     +-StringLiteral[value=INDONESIA]
-  |       +-from_clause=
-  |         +-TableReference[table=partsupp]
-  |         +-TableReference[table=supplier]
-  |         +-TableReference[table=nation]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=false,nulls_first=true]
-  |   +-AttributeReference[attribute_name=value]
-  +-from_clause=
-    +-TableReference[table=partsupp]
-    +-TableReference[table=supplier]
-    +-TableReference[table=nation]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=ps_partkey]
+      | +-SelectListItem[alias=value]
+      |   +-FunctionCall[name=SUM]
+      |     +-Multiply
+      |       +-left_operand=AttributeReference[attribute_name=ps_supplycost]
+      |       +-right_operand=AttributeReference[attribute_name=ps_availqty]
+      +-where_clause=And
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=ps_suppkey]
+      | | +-right_operand=AttributeReference[attribute_name=s_suppkey]
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=s_nationkey]
+      | | +-right_operand=AttributeReference[attribute_name=n_nationkey]
+      | +-Equal
+      |   +-left_operand=AttributeReference[attribute_name=n_name]
+      |   +-right_operand=Literal
+      |     +-StringLiteral[value=INDONESIA]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=ps_partkey]
+      +-having=HAVING
+      | +-Greater
+      |   +-left_operand=FunctionCall[name=SUM]
+      |   | +-Multiply
+      |   |   +-left_operand=AttributeReference[attribute_name=ps_supplycost]
+      |   |   +-right_operand=AttributeReference[attribute_name=ps_availqty]
+      |   +-right_operand=SubqueryExpression
+      |     +-SetOperation[set_operation_type=Select]
+      |       +-children=
+      |         +-Select
+      |           +-select_clause=SelectList
+      |           | +-SelectListItem
+      |           |   +-Multiply
+      |           |     +-left_operand=FunctionCall[name=SUM]
+      |           |     | +-Multiply
+      |           |     |   +-left_operand=AttributeReference[
+      |           |     |   | attribute_name=ps_supplycost]
+      |           |     |   +-right_operand=AttributeReference[
+      |           |     |     attribute_name=ps_availqty]
+      |           |     +-right_operand=Literal
+      |           |       +-NumericLiteral[numeric_string=0.0000010000,
+      |           |         float_like=true]
+      |           +-where_clause=And
+      |           | +-Equal
+      |           | | +-left_operand=AttributeReference[
+      |           | | | attribute_name=ps_suppkey]
+      |           | | +-right_operand=AttributeReference[
+      |           | |   attribute_name=s_suppkey]
+      |           | +-Equal
+      |           | | +-left_operand=AttributeReference[
+      |           | | | attribute_name=s_nationkey]
+      |           | | +-right_operand=AttributeReference[
+      |           | |   attribute_name=n_nationkey]
+      |           | +-Equal
+      |           |   +-left_operand=AttributeReference[attribute_name=n_name]
+      |           |   +-right_operand=Literal
+      |           |     +-StringLiteral[value=INDONESIA]
+      |           +-from_clause=
+      |             +-TableReference[table=partsupp]
+      |             +-TableReference[table=supplier]
+      |             +-TableReference[table=nation]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=false,nulls_first=true]
+      |   +-AttributeReference[attribute_name=value]
+      +-from_clause=
+        +-TableReference[table=partsupp]
+        +-TableReference[table=supplier]
+        +-TableReference[table=nation]
 ==
 
 # Query 12
@@ -1133,87 +1221,89 @@ GROUP BY
 ORDER BY
   l_shipmode
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=l_shipmode]
-  | +-SelectListItem[alias=high_line_count]
-  | | +-FunctionCall[name=SUM]
-  | |   +-SearchedCaseExpression
-  | |     +-else_result_expression=Literal
-  | |     | +-NumericLiteral[numeric_string=0,float_like=false]
-  | |     +-when_clauses=
-  | |       +-SearchedWhenClause
-  | |         +-condition_predicate=Or
-  | |         | +-Equal
-  | |         | | +-left_operand=AttributeReference[
-  | |         | | | attribute_name=o_orderpriority]
-  | |         | | +-right_operand=Literal
-  | |         | |   +-StringLiteral[value=1-URGENT]
-  | |         | +-Equal
-  | |         |   +-left_operand=AttributeReference[
-  | |         |   | attribute_name=o_orderpriority]
-  | |         |   +-right_operand=Literal
-  | |         |     +-StringLiteral[value=2-HIGH]
-  | |         +-result_expression=Literal
-  | |           +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-SelectListItem[alias=low_line_count]
-  |   +-FunctionCall[name=SUM]
-  |     +-SearchedCaseExpression
-  |       +-else_result_expression=Literal
-  |       | +-NumericLiteral[numeric_string=0,float_like=false]
-  |       +-when_clauses=
-  |         +-SearchedWhenClause
-  |           +-condition_predicate=And
-  |           | +-NotEqual
-  |           | | +-left_operand=AttributeReference[
-  |           | | | attribute_name=o_orderpriority]
-  |           | | +-right_operand=Literal
-  |           | |   +-StringLiteral[value=1-URGENT]
-  |           | +-NotEqual
-  |           |   +-left_operand=AttributeReference[
-  |           |   | attribute_name=o_orderpriority]
-  |           |   +-right_operand=Literal
-  |           |     +-StringLiteral[value=2-HIGH]
-  |           +-result_expression=Literal
-  |             +-NumericLiteral[numeric_string=1,float_like=false]
-  +-where_clause=And
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=o_orderkey]
-  | | +-right_operand=AttributeReference[attribute_name=l_orderkey]
-  | +-InValueList
-  | | +-test_expression=AttributeReference[attribute_name=l_shipmode]
-  | | +-value_list=
-  | |   +-Literal
-  | |   | +-StringLiteral[value=REG AIR]
-  | |   +-Literal
-  | |     +-StringLiteral[value=RAIL]
-  | +-Less
-  | | +-left_operand=AttributeReference[attribute_name=l_commitdate]
-  | | +-right_operand=AttributeReference[attribute_name=l_receiptdate]
-  | +-Less
-  | | +-left_operand=AttributeReference[attribute_name=l_shipdate]
-  | | +-right_operand=AttributeReference[attribute_name=l_commitdate]
-  | +-GreaterOrEqual
-  | | +-left_operand=AttributeReference[attribute_name=l_receiptdate]
-  | | +-right_operand=Literal
-  | |   +-StringLiteral[value=1997-01-01,explicit_type=Date]
-  | +-Less
-  |   +-left_operand=AttributeReference[attribute_name=l_receiptdate]
-  |   +-right_operand=Add
-  |     +-left_operand=Literal
-  |     | +-StringLiteral[value=1997-01-01,explicit_type=Date]
-  |     +-right_operand=Literal
-  |       +-StringLiteral[value=1 year,explicit_type=YearMonthInterval]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=l_shipmode]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=true,nulls_first=false]
-  |   +-AttributeReference[attribute_name=l_shipmode]
-  +-from_clause=
-    +-TableReference[table=orders]
-    +-TableReference[table=lineitem]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=l_shipmode]
+      | +-SelectListItem[alias=high_line_count]
+      | | +-FunctionCall[name=SUM]
+      | |   +-SearchedCaseExpression
+      | |     +-else_result_expression=Literal
+      | |     | +-NumericLiteral[numeric_string=0,float_like=false]
+      | |     +-when_clauses=
+      | |       +-SearchedWhenClause
+      | |         +-condition_predicate=Or
+      | |         | +-Equal
+      | |         | | +-left_operand=AttributeReference[
+      | |         | | | attribute_name=o_orderpriority]
+      | |         | | +-right_operand=Literal
+      | |         | |   +-StringLiteral[value=1-URGENT]
+      | |         | +-Equal
+      | |         |   +-left_operand=AttributeReference[
+      | |         |   | attribute_name=o_orderpriority]
+      | |         |   +-right_operand=Literal
+      | |         |     +-StringLiteral[value=2-HIGH]
+      | |         +-result_expression=Literal
+      | |           +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-SelectListItem[alias=low_line_count]
+      |   +-FunctionCall[name=SUM]
+      |     +-SearchedCaseExpression
+      |       +-else_result_expression=Literal
+      |       | +-NumericLiteral[numeric_string=0,float_like=false]
+      |       +-when_clauses=
+      |         +-SearchedWhenClause
+      |           +-condition_predicate=And
+      |           | +-NotEqual
+      |           | | +-left_operand=AttributeReference[
+      |           | | | attribute_name=o_orderpriority]
+      |           | | +-right_operand=Literal
+      |           | |   +-StringLiteral[value=1-URGENT]
+      |           | +-NotEqual
+      |           |   +-left_operand=AttributeReference[
+      |           |   | attribute_name=o_orderpriority]
+      |           |   +-right_operand=Literal
+      |           |     +-StringLiteral[value=2-HIGH]
+      |           +-result_expression=Literal
+      |             +-NumericLiteral[numeric_string=1,float_like=false]
+      +-where_clause=And
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=o_orderkey]
+      | | +-right_operand=AttributeReference[attribute_name=l_orderkey]
+      | +-InValueList
+      | | +-test_expression=AttributeReference[attribute_name=l_shipmode]
+      | | +-value_list=
+      | |   +-Literal
+      | |   | +-StringLiteral[value=REG AIR]
+      | |   +-Literal
+      | |     +-StringLiteral[value=RAIL]
+      | +-Less
+      | | +-left_operand=AttributeReference[attribute_name=l_commitdate]
+      | | +-right_operand=AttributeReference[attribute_name=l_receiptdate]
+      | +-Less
+      | | +-left_operand=AttributeReference[attribute_name=l_shipdate]
+      | | +-right_operand=AttributeReference[attribute_name=l_commitdate]
+      | +-GreaterOrEqual
+      | | +-left_operand=AttributeReference[attribute_name=l_receiptdate]
+      | | +-right_operand=Literal
+      | |   +-StringLiteral[value=1997-01-01,explicit_type=Date]
+      | +-Less
+      |   +-left_operand=AttributeReference[attribute_name=l_receiptdate]
+      |   +-right_operand=Add
+      |     +-left_operand=Literal
+      |     | +-StringLiteral[value=1997-01-01,explicit_type=Date]
+      |     +-right_operand=Literal
+      |       +-StringLiteral[value=1 year,explicit_type=YearMonthInterval]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=l_shipmode]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=true,nulls_first=false]
+      |   +-AttributeReference[attribute_name=l_shipmode]
+      +-from_clause=
+        +-TableReference[table=orders]
+        +-TableReference[table=lineitem]
 ==
 
 # Query 13
@@ -1238,46 +1328,53 @@ ORDER BY
   custdist desc,
   c_count DESC
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-AttributeReference[attribute_name=c_count]
-  | +-SelectListItem[alias=custdist]
-  |   +-FunctionCall[name=COUNT,is_star=true]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=c_count]
-  +-order_by=OrderBy
-  | +-OrderByItem[is_asc=false,nulls_first=true]
-  | | +-AttributeReference[attribute_name=custdist]
-  | +-OrderByItem[is_asc=false,nulls_first=true]
-  |   +-AttributeReference[attribute_name=c_count]
-  +-from_clause=
-    +-SubqueryTable
-      +-table_signature=TableSignature[table_alias=c_orders,
-      | columns=(c_custkey, c_count)]
-      +-SubqueryExpression
-        +-Select
-          +-select_clause=SelectList
-          | +-SelectListItem
-          | | +-AttributeReference[attribute_name=c_custkey]
-          | +-SelectListItem
-          |   +-FunctionCall[name=COUNT]
-          |     +-AttributeReference[attribute_name=o_orderkey]
-          +-group_by=GroupBy
-          | +-AttributeReference[attribute_name=c_custkey]
-          +-from_clause=
-            +-JoinedTable[join_type=LeftOuterJoin]
-              +-left_table=TableReference[table=customer]
-              +-right_table=TableReference[table=orders]
-              +-join_predicate=And
-                +-Equal
-                | +-left_operand=AttributeReference[attribute_name=c_custkey]
-                | +-right_operand=AttributeReference[attribute_name=o_custkey]
-                +-NotLike
-                  +-left_operand=AttributeReference[attribute_name=o_comment]
-                  +-right_operand=Literal
-                    +-StringLiteral[value=%special%requests%]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-AttributeReference[attribute_name=c_count]
+      | +-SelectListItem[alias=custdist]
+      |   +-FunctionCall[name=COUNT,is_star=true]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=c_count]
+      +-order_by=OrderBy
+      | +-OrderByItem[is_asc=false,nulls_first=true]
+      | | +-AttributeReference[attribute_name=custdist]
+      | +-OrderByItem[is_asc=false,nulls_first=true]
+      |   +-AttributeReference[attribute_name=c_count]
+      +-from_clause=
+        +-SubqueryTable
+          +-table_signature=TableSignature[table_alias=c_orders,
+          | columns=(c_custkey, c_count)]
+          +-SubqueryExpression
+            +-SetOperation[set_operation_type=Select]
+              +-children=
+                +-Select
+                  +-select_clause=SelectList
+                  | +-SelectListItem
+                  | | +-AttributeReference[attribute_name=c_custkey]
+                  | +-SelectListItem
+                  |   +-FunctionCall[name=COUNT]
+                  |     +-AttributeReference[attribute_name=o_orderkey]
+                  +-group_by=GroupBy
+                  | +-AttributeReference[attribute_name=c_custkey]
+                  +-from_clause=
+                    +-JoinedTable[join_type=LeftOuterJoin]
+                      +-left_table=TableReference[table=customer]
+                      +-right_table=TableReference[table=orders]
+                      +-join_predicate=And
+                        +-Equal
+                        | +-left_operand=AttributeReference[
+                        | | attribute_name=c_custkey]
+                        | +-right_operand=AttributeReference[
+                        |   attribute_name=o_custkey]
+                        +-NotLike
+                          +-left_operand=AttributeReference[
+                          | attribute_name=o_comment]
+                          +-right_operand=Literal
+                            +-StringLiteral[value=%special%requests%]
 ==
 
 # Query 14
@@ -1295,57 +1392,59 @@ WHERE
   AND l_shipdate >= DATE '1994-11-01'
   AND l_shipdate < DATE '1994-11-01' + INTERVAL '1 month'
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem[alias=promo_revenue]
-  |   +-Divide
-  |     +-left_operand=Multiply
-  |     | +-left_operand=Literal
-  |     | | +-NumericLiteral[numeric_string=100.00,float_like=true]
-  |     | +-right_operand=FunctionCall[name=SUM]
-  |     |   +-SearchedCaseExpression
-  |     |     +-else_result_expression=Literal
-  |     |     | +-NumericLiteral[numeric_string=0,float_like=false]
-  |     |     +-when_clauses=
-  |     |       +-SearchedWhenClause
-  |     |         +-condition_predicate=Like
-  |     |         | +-left_operand=AttributeReference[attribute_name=p_type]
-  |     |         | +-right_operand=Literal
-  |     |         |   +-StringLiteral[value=PROMO%]
-  |     |         +-result_expression=Multiply
-  |     |           +-left_operand=AttributeReference[
-  |     |           | attribute_name=l_extendedprice]
-  |     |           +-right_operand=Subtract
-  |     |             +-left_operand=Literal
-  |     |             | +-NumericLiteral[numeric_string=1,float_like=false]
-  |     |             +-right_operand=AttributeReference[
-  |     |               attribute_name=l_discount]
-  |     +-right_operand=FunctionCall[name=SUM]
-  |       +-Multiply
-  |         +-left_operand=AttributeReference[attribute_name=l_extendedprice]
-  |         +-right_operand=Subtract
-  |           +-left_operand=Literal
-  |           | +-NumericLiteral[numeric_string=1,float_like=false]
-  |           +-right_operand=AttributeReference[attribute_name=l_discount]
-  +-where_clause=And
-  | +-Equal
-  | | +-left_operand=AttributeReference[attribute_name=l_partkey]
-  | | +-right_operand=AttributeReference[attribute_name=p_partkey]
-  | +-GreaterOrEqual
-  | | +-left_operand=AttributeReference[attribute_name=l_shipdate]
-  | | +-right_operand=Literal
-  | |   +-StringLiteral[value=1994-11-01,explicit_type=Date]
-  | +-Less
-  |   +-left_operand=AttributeReference[attribute_name=l_shipdate]
-  |   +-right_operand=Add
-  |     +-left_operand=Literal
-  |     | +-StringLiteral[value=1994-11-01,explicit_type=Date]
-  |     +-right_operand=Literal
-  |       +-StringLiteral[value=1 month,explicit_type=YearMonthInterval]
-  +-from_clause=
-    +-TableReference[table=lineitem]
-    +-TableReference[table=part]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem[alias=promo_revenue]
+      |   +-Divide
+      |     +-left_operand=Multiply
+      |     | +-left_operand=Literal
+      |     | | +-NumericLiteral[numeric_string=100.00,float_like=true]
+      |     | +-right_operand=FunctionCall[name=SUM]
+      |     |   +-SearchedCaseExpression
+      |     |     +-else_result_expression=Literal
+      |     |     | +-NumericLiteral[numeric_string=0,float_like=false]
+      |     |     +-when_clauses=
+      |     |       +-SearchedWhenClause
+      |     |         +-condition_predicate=Like
+      |     |         | +-left_operand=AttributeReference[attribute_name=p_type]
+      |     |         | +-right_operand=Literal
+      |     |         |   +-StringLiteral[value=PROMO%]
+      |     |         +-result_expression=Multiply
+      |     |           +-left_operand=AttributeReference[
+      |     |           | attribute_name=l_extendedprice]
+      |     |           +-right_operand=Subtract
+      |     |             +-left_operand=Literal
+      |     |             | +-NumericLiteral[numeric_string=1,float_like=false]
+      |     |             +-right_operand=AttributeReference[
+      |     |               attribute_name=l_discount]
+      |     +-right_operand=FunctionCall[name=SUM]
+      |       +-Multiply
+      |         +-left_operand=AttributeReference[attribute_name=l_extendedprice]
+      |         +-right_operand=Subtract
+      |           +-left_operand=Literal
+      |           | +-NumericLiteral[numeric_string=1,float_like=false]
+      |           +-right_operand=AttributeReference[attribute_name=l_discount]
+      +-where_clause=And
+      | +-Equal
+      | | +-left_operand=AttributeReference[attribute_name=l_partkey]
+      | | +-right_operand=AttributeReference[attribute_name=p_partkey]
+      | +-GreaterOrEqual
+      | | +-left_operand=AttributeReference[attribute_name=l_shipdate]
+      | | +-right_operand=Literal
+      | |   +-StringLiteral[value=1994-11-01,explicit_type=Date]
+      | +-Less
+      |   +-left_operand=AttributeReference[attribute_name=l_shipdate]
+      |   +-right_operand=Add
+      |     +-left_operand=Literal
+      |     | +-StringLiteral[value=1994-11-01,explicit_type=Date]
+      |     +-right_operand=Literal
+      |       +-StringLiteral[value=1 month,explicit_type=YearMonthInterval]
+      +-from_clause=
+        +-TableReference[table=lineitem]
+        +-TableReference[table=part]
 ==
 
 # Query 15
@@ -1379,72 +1478,81 @@ FROM
 ORDER BY
   s_suppkey
 --
-SelectStatement
-+-select_query=Select
-| +-select_clause=SelectList
-| | +-SelectListItem
-| | | +-AttributeReference[attribute_name=s_suppkey]
-| | +-SelectListItem
-| | | +-AttributeReference[attribu

<TRUNCATED>


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

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

<TRUNCATED>


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

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/preprocessed/SqlParser_gen.cpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.cpp b/parser/preprocessed/SqlParser_gen.cpp
index 23cbf1b..871053f 100644
--- a/parser/preprocessed/SqlParser_gen.cpp
+++ b/parser/preprocessed/SqlParser_gen.cpp
@@ -126,6 +126,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"
@@ -152,7 +153,7 @@ typedef struct YYLTYPE {
 // Needed for Bison 2.6 and higher, which do not automatically provide this typedef.
 typedef void* yyscan_t;
 
-#line 156 "SqlParser_gen.cpp" /* yacc.c:339  */
+#line 157 "SqlParser_gen.cpp" /* yacc.c:339  */
 
 # ifndef YY_NULLPTR
 #  if defined __cplusplus && 201103L <= __cplusplus
@@ -207,117 +208,119 @@ extern int quickstep_yydebug;
     TOKEN_IS = 275,
     UNARY_PLUS = 276,
     UNARY_MINUS = 277,
-    TOKEN_ADD = 278,
-    TOKEN_ALL = 279,
-    TOKEN_ALTER = 280,
-    TOKEN_AS = 281,
-    TOKEN_ASC = 282,
-    TOKEN_BIGINT = 283,
-    TOKEN_BIT = 284,
-    TOKEN_BITWEAVING = 285,
-    TOKEN_BLOCKPROPERTIES = 286,
-    TOKEN_BLOCKSAMPLE = 287,
-    TOKEN_BLOOM_FILTER = 288,
-    TOKEN_CSB_TREE = 289,
-    TOKEN_BY = 290,
-    TOKEN_CASE = 291,
-    TOKEN_CHARACTER = 292,
-    TOKEN_CHECK = 293,
-    TOKEN_COLUMN = 294,
-    TOKEN_CONSTRAINT = 295,
-    TOKEN_COPY = 296,
-    TOKEN_CREATE = 297,
-    TOKEN_CURRENT = 298,
-    TOKEN_DATE = 299,
-    TOKEN_DATETIME = 300,
-    TOKEN_DAY = 301,
-    TOKEN_DECIMAL = 302,
-    TOKEN_DEFAULT = 303,
-    TOKEN_DELETE = 304,
-    TOKEN_DELIMITER = 305,
-    TOKEN_DESC = 306,
-    TOKEN_DISTINCT = 307,
-    TOKEN_DOUBLE = 308,
-    TOKEN_DROP = 309,
-    TOKEN_ELSE = 310,
-    TOKEN_END = 311,
-    TOKEN_ESCAPE_STRINGS = 312,
-    TOKEN_EXISTS = 313,
-    TOKEN_EXTRACT = 314,
-    TOKEN_FALSE = 315,
-    TOKEN_FIRST = 316,
-    TOKEN_FLOAT = 317,
-    TOKEN_FOLLOWING = 318,
-    TOKEN_FOR = 319,
-    TOKEN_FOREIGN = 320,
-    TOKEN_FROM = 321,
-    TOKEN_FULL = 322,
-    TOKEN_GROUP = 323,
-    TOKEN_HASH = 324,
-    TOKEN_HAVING = 325,
-    TOKEN_HOUR = 326,
-    TOKEN_IN = 327,
-    TOKEN_INDEX = 328,
-    TOKEN_INNER = 329,
-    TOKEN_INSERT = 330,
-    TOKEN_INTEGER = 331,
-    TOKEN_INTERVAL = 332,
-    TOKEN_INTO = 333,
-    TOKEN_JOIN = 334,
-    TOKEN_KEY = 335,
-    TOKEN_LAST = 336,
-    TOKEN_LEFT = 337,
-    TOKEN_LIMIT = 338,
-    TOKEN_LONG = 339,
-    TOKEN_MINUTE = 340,
-    TOKEN_MONTH = 341,
-    TOKEN_NULL = 342,
-    TOKEN_NULLS = 343,
-    TOKEN_OFF = 344,
-    TOKEN_ON = 345,
-    TOKEN_ORDER = 346,
-    TOKEN_OUTER = 347,
-    TOKEN_OVER = 348,
-    TOKEN_PARTITION = 349,
-    TOKEN_PARTITIONS = 350,
-    TOKEN_PERCENT = 351,
-    TOKEN_PRECEDING = 352,
-    TOKEN_PRIMARY = 353,
-    TOKEN_PRIORITY = 354,
-    TOKEN_QUIT = 355,
-    TOKEN_RANGE = 356,
-    TOKEN_REAL = 357,
-    TOKEN_REFERENCES = 358,
-    TOKEN_RIGHT = 359,
-    TOKEN_ROW = 360,
-    TOKEN_ROW_DELIMITER = 361,
-    TOKEN_ROWS = 362,
-    TOKEN_SECOND = 363,
-    TOKEN_SELECT = 364,
-    TOKEN_SET = 365,
-    TOKEN_SMA = 366,
-    TOKEN_SMALLINT = 367,
-    TOKEN_SUBSTRING = 368,
-    TOKEN_TABLE = 369,
-    TOKEN_THEN = 370,
-    TOKEN_TIME = 371,
-    TOKEN_TIMESTAMP = 372,
-    TOKEN_TRUE = 373,
-    TOKEN_TUPLESAMPLE = 374,
-    TOKEN_UNBOUNDED = 375,
-    TOKEN_UNIQUE = 376,
-    TOKEN_UPDATE = 377,
-    TOKEN_USING = 378,
-    TOKEN_VALUES = 379,
-    TOKEN_VARCHAR = 380,
-    TOKEN_WHEN = 381,
-    TOKEN_WHERE = 382,
-    TOKEN_WINDOW = 383,
-    TOKEN_WITH = 384,
-    TOKEN_YEAR = 385,
-    TOKEN_YEARMONTH = 386,
-    TOKEN_EOF = 387,
-    TOKEN_LEX_ERROR = 388
+    TOKEN_ALL = 278,
+    TOKEN_UNION = 279,
+    TOKEN_INTERSECT = 280,
+    TOKEN_ADD = 281,
+    TOKEN_ALTER = 282,
+    TOKEN_AS = 283,
+    TOKEN_ASC = 284,
+    TOKEN_BIGINT = 285,
+    TOKEN_BIT = 286,
+    TOKEN_BITWEAVING = 287,
+    TOKEN_BLOCKPROPERTIES = 288,
+    TOKEN_BLOCKSAMPLE = 289,
+    TOKEN_BLOOM_FILTER = 290,
+    TOKEN_CSB_TREE = 291,
+    TOKEN_BY = 292,
+    TOKEN_CASE = 293,
+    TOKEN_CHARACTER = 294,
+    TOKEN_CHECK = 295,
+    TOKEN_COLUMN = 296,
+    TOKEN_CONSTRAINT = 297,
+    TOKEN_COPY = 298,
+    TOKEN_CREATE = 299,
+    TOKEN_CURRENT = 300,
+    TOKEN_DATE = 301,
+    TOKEN_DATETIME = 302,
+    TOKEN_DAY = 303,
+    TOKEN_DECIMAL = 304,
+    TOKEN_DEFAULT = 305,
+    TOKEN_DELETE = 306,
+    TOKEN_DELIMITER = 307,
+    TOKEN_DESC = 308,
+    TOKEN_DISTINCT = 309,
+    TOKEN_DOUBLE = 310,
+    TOKEN_DROP = 311,
+    TOKEN_ELSE = 312,
+    TOKEN_END = 313,
+    TOKEN_ESCAPE_STRINGS = 314,
+    TOKEN_EXISTS = 315,
+    TOKEN_EXTRACT = 316,
+    TOKEN_FALSE = 317,
+    TOKEN_FIRST = 318,
+    TOKEN_FLOAT = 319,
+    TOKEN_FOLLOWING = 320,
+    TOKEN_FOR = 321,
+    TOKEN_FOREIGN = 322,
+    TOKEN_FROM = 323,
+    TOKEN_FULL = 324,
+    TOKEN_GROUP = 325,
+    TOKEN_HASH = 326,
+    TOKEN_HAVING = 327,
+    TOKEN_HOUR = 328,
+    TOKEN_IN = 329,
+    TOKEN_INDEX = 330,
+    TOKEN_INNER = 331,
+    TOKEN_INSERT = 332,
+    TOKEN_INTEGER = 333,
+    TOKEN_INTERVAL = 334,
+    TOKEN_INTO = 335,
+    TOKEN_JOIN = 336,
+    TOKEN_KEY = 337,
+    TOKEN_LAST = 338,
+    TOKEN_LEFT = 339,
+    TOKEN_LIMIT = 340,
+    TOKEN_LONG = 341,
+    TOKEN_MINUTE = 342,
+    TOKEN_MONTH = 343,
+    TOKEN_NULL = 344,
+    TOKEN_NULLS = 345,
+    TOKEN_OFF = 346,
+    TOKEN_ON = 347,
+    TOKEN_ORDER = 348,
+    TOKEN_OUTER = 349,
+    TOKEN_OVER = 350,
+    TOKEN_PARTITION = 351,
+    TOKEN_PARTITIONS = 352,
+    TOKEN_PERCENT = 353,
+    TOKEN_PRECEDING = 354,
+    TOKEN_PRIMARY = 355,
+    TOKEN_PRIORITY = 356,
+    TOKEN_QUIT = 357,
+    TOKEN_RANGE = 358,
+    TOKEN_REAL = 359,
+    TOKEN_REFERENCES = 360,
+    TOKEN_RIGHT = 361,
+    TOKEN_ROW = 362,
+    TOKEN_ROW_DELIMITER = 363,
+    TOKEN_ROWS = 364,
+    TOKEN_SECOND = 365,
+    TOKEN_SELECT = 366,
+    TOKEN_SET = 367,
+    TOKEN_SMA = 368,
+    TOKEN_SMALLINT = 369,
+    TOKEN_SUBSTRING = 370,
+    TOKEN_TABLE = 371,
+    TOKEN_THEN = 372,
+    TOKEN_TIME = 373,
+    TOKEN_TIMESTAMP = 374,
+    TOKEN_TRUE = 375,
+    TOKEN_TUPLESAMPLE = 376,
+    TOKEN_UNBOUNDED = 377,
+    TOKEN_UNIQUE = 378,
+    TOKEN_UPDATE = 379,
+    TOKEN_USING = 380,
+    TOKEN_VALUES = 381,
+    TOKEN_VARCHAR = 382,
+    TOKEN_WHEN = 383,
+    TOKEN_WHERE = 384,
+    TOKEN_WINDOW = 385,
+    TOKEN_WITH = 386,
+    TOKEN_YEAR = 387,
+    TOKEN_YEARMONTH = 388,
+    TOKEN_EOF = 389,
+    TOKEN_LEX_ERROR = 390
   };
 #endif
 
@@ -326,7 +329,7 @@ extern int quickstep_yydebug;
 
 union YYSTYPE
 {
-#line 120 "../SqlParser.ypp" /* yacc.c:355  */
+#line 121 "../SqlParser.ypp" /* yacc.c:355  */
 
   quickstep::ParseString *string_value_;
 
@@ -358,6 +361,8 @@ union YYSTYPE
   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_;
@@ -385,7 +390,7 @@ union YYSTYPE
   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_;
@@ -426,7 +431,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 430 "SqlParser_gen.cpp" /* yacc.c:355  */
+#line 435 "SqlParser_gen.cpp" /* yacc.c:355  */
 };
 
 typedef union YYSTYPE YYSTYPE;
@@ -455,13 +460,13 @@ int quickstep_yyparse (yyscan_t yyscanner, quickstep::ParseStatement **parsedSta
 #endif /* !YY_QUICKSTEP_YY_SQLPARSER_GEN_HPP_INCLUDED  */
 
 /* Copy the second part of user declarations.  */
-#line 220 "../SqlParser.ypp" /* yacc.c:358  */
+#line 223 "../SqlParser.ypp" /* yacc.c:358  */
 
 /* This header needs YYSTYPE, which is defined by the %union directive above */
 #include "SqlLexer_gen.hpp"
 void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string &feature);
 
-#line 465 "SqlParser_gen.cpp" /* yacc.c:358  */
+#line 470 "SqlParser_gen.cpp" /* yacc.c:358  */
 
 #ifdef short
 # undef short
@@ -703,23 +708,23 @@ union yyalloc
 #endif /* !YYCOPY_NEEDED */
 
 /* YYFINAL -- State number of the termination state.  */
-#define YYFINAL  47
+#define YYFINAL  49
 /* YYLAST -- Last index in YYTABLE.  */
-#define YYLAST   1346
+#define YYLAST   1207
 
 /* YYNTOKENS -- Number of terminals.  */
-#define YYNTOKENS  145
+#define YYNTOKENS  147
 /* YYNNTS -- Number of nonterminals.  */
-#define YYNNTS  107
+#define YYNNTS  109
 /* YYNRULES -- Number of rules.  */
-#define YYNRULES  289
+#define YYNRULES  293
 /* YYNSTATES -- Number of states.  */
-#define YYNSTATES  536
+#define YYNSTATES  543
 
 /* YYTRANSLATE[YYX] -- Symbol number corresponding to YYX as returned
    by yylex, with out-of-bounds checking.  */
 #define YYUNDEFTOK  2
-#define YYMAXUTOK   388
+#define YYMAXUTOK   390
 
 #define YYTRANSLATE(YYX)                                                \
   ((unsigned int) (YYX) <= YYMAXUTOK ? yytranslate[YYX] : YYUNDEFTOK)
@@ -729,11 +734,11 @@ union yyalloc
 static const yytype_uint8 yytranslate[] =
 {
        0,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-     140,     2,     2,     2,     2,     2,     2,     2,     2,     2,
+     142,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,   144,     2,     2,
-     141,   142,    23,    21,   143,    22,    27,    24,     2,     2,
-       2,     2,     2,     2,     2,     2,     2,     2,     2,   139,
+       2,     2,     2,     2,     2,     2,     2,   146,     2,     2,
+     143,   144,    23,    21,   145,    22,    27,    24,     2,     2,
+       2,     2,     2,     2,     2,     2,     2,     2,     2,   141,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
        2,     2,     2,     2,     2,     2,     2,     2,     2,     2,
@@ -766,42 +771,44 @@ static const yytype_uint8 yytranslate[] =
      100,   101,   102,   103,   104,   105,   106,   107,   108,   109,
      110,   111,   112,   113,   114,   115,   116,   117,   118,   119,
      120,   121,   122,   123,   124,   125,   126,   127,   128,   129,
-     130,   131,   132,   133,   134,   135,   136,   137,   138
+     130,   131,   132,   133,   134,   135,   136,   137,   138,   139,
+     140
 };
 
 #if YYDEBUG
   /* YYRLINE[YYN] -- Source line where rule number YYN was defined.  */
 static const yytype_uint16 yyrline[] =
 {
-       0,   629,   629,   633,   637,   641,   645,   648,   655,   658,
-     661,   664,   667,   670,   673,   676,   679,   682,   688,   694,
-     701,   707,   714,   723,   728,   737,   742,   747,   751,   757,
-     762,   765,   768,   773,   776,   779,   782,   785,   788,   791,
-     794,   797,   800,   812,   815,   818,   836,   856,   859,   862,
-     867,   872,   878,   884,   893,   897,   903,   906,   911,   916,
-     921,   928,   935,   939,   945,   948,   953,   956,   961,   964,
-     969,   972,   991,   994,   999,  1003,  1009,  1012,  1015,  1020,
-    1023,  1030,  1035,  1046,  1051,  1055,  1059,  1065,  1068,  1074,
-    1082,  1085,  1088,  1094,  1099,  1102,  1107,  1111,  1115,  1119,
-    1125,  1130,  1135,  1139,  1145,  1151,  1154,  1159,  1162,  1181,
-    1186,  1190,  1196,  1202,  1208,  1211,  1215,  1221,  1224,  1229,
-    1233,  1239,  1242,  1245,  1250,  1255,  1260,  1263,  1266,  1271,
-    1274,  1277,  1280,  1283,  1286,  1289,  1292,  1297,  1300,  1305,
-    1309,  1313,  1316,  1320,  1323,  1328,  1331,  1336,  1339,  1344,
-    1348,  1354,  1357,  1362,  1365,  1370,  1373,  1378,  1381,  1400,
-    1403,  1408,  1412,  1418,  1424,  1429,  1432,  1437,  1440,  1445,
-    1448,  1453,  1456,  1461,  1462,  1465,  1470,  1471,  1474,  1479,
-    1483,  1489,  1496,  1499,  1502,  1507,  1510,  1513,  1519,  1522,
-    1527,  1532,  1541,  1546,  1555,  1560,  1563,  1568,  1571,  1576,
-    1582,  1588,  1591,  1594,  1597,  1600,  1603,  1609,  1618,  1621,
-    1626,  1629,  1634,  1637,  1642,  1645,  1648,  1651,  1655,  1659,
-    1662,  1665,  1668,  1671,  1676,  1680,  1684,  1687,  1692,  1697,
-    1701,  1707,  1710,  1715,  1719,  1725,  1730,  1734,  1740,  1745,
-    1748,  1753,  1757,  1763,  1766,  1769,  1772,  1784,  1788,  1807,
-    1820,  1835,  1838,  1841,  1844,  1847,  1850,  1855,  1859,  1865,
-    1868,  1873,  1877,  1884,  1887,  1890,  1893,  1896,  1899,  1902,
-    1905,  1908,  1911,  1916,  1927,  1930,  1935,  1938,  1941,  1947,
-    1951,  1957,  1960,  1968,  1971,  1974,  1977,  1983,  1988,  1993
+       0,   639,   639,   643,   647,   651,   655,   658,   665,   668,
+     671,   674,   677,   680,   683,   686,   689,   692,   698,   704,
+     711,   717,   724,   733,   738,   747,   752,   757,   761,   767,
+     772,   775,   778,   783,   786,   789,   792,   795,   798,   801,
+     804,   807,   810,   822,   825,   828,   846,   866,   869,   872,
+     877,   882,   888,   894,   903,   907,   913,   916,   921,   926,
+     931,   938,   945,   949,   955,   958,   963,   966,   971,   974,
+     979,   982,  1001,  1004,  1009,  1013,  1019,  1022,  1025,  1030,
+    1033,  1040,  1045,  1056,  1061,  1065,  1069,  1075,  1078,  1084,
+    1092,  1095,  1098,  1104,  1109,  1112,  1117,  1121,  1125,  1129,
+    1135,  1140,  1145,  1149,  1155,  1164,  1167,  1172,  1175,  1194,
+    1199,  1203,  1209,  1215,  1224,  1229,  1237,  1243,  1249,  1252,
+    1255,  1260,  1263,  1268,  1272,  1278,  1281,  1284,  1289,  1294,
+    1299,  1302,  1305,  1310,  1313,  1316,  1319,  1322,  1325,  1328,
+    1331,  1336,  1339,  1344,  1348,  1352,  1355,  1359,  1362,  1367,
+    1370,  1375,  1378,  1383,  1387,  1393,  1396,  1401,  1404,  1409,
+    1412,  1417,  1420,  1439,  1442,  1447,  1451,  1457,  1463,  1468,
+    1471,  1476,  1479,  1484,  1487,  1492,  1495,  1500,  1501,  1504,
+    1509,  1510,  1513,  1518,  1522,  1528,  1535,  1538,  1541,  1546,
+    1549,  1552,  1558,  1561,  1566,  1571,  1580,  1585,  1594,  1599,
+    1602,  1607,  1610,  1615,  1621,  1627,  1630,  1633,  1636,  1639,
+    1642,  1648,  1657,  1660,  1665,  1668,  1673,  1676,  1681,  1684,
+    1687,  1690,  1694,  1698,  1701,  1704,  1707,  1710,  1715,  1719,
+    1723,  1726,  1731,  1736,  1740,  1746,  1749,  1754,  1758,  1764,
+    1769,  1773,  1779,  1784,  1787,  1792,  1796,  1802,  1805,  1808,
+    1811,  1823,  1827,  1846,  1859,  1874,  1877,  1880,  1883,  1886,
+    1889,  1894,  1898,  1904,  1907,  1912,  1916,  1923,  1926,  1929,
+    1932,  1935,  1938,  1941,  1944,  1947,  1950,  1955,  1966,  1969,
+    1974,  1977,  1980,  1986,  1990,  1996,  1999,  2007,  2010,  2013,
+    2016,  2022,  2027,  2032
 };
 #endif
 
@@ -816,23 +823,24 @@ static const char *const yytname[] =
   "TOKEN_EQ", "TOKEN_LT", "TOKEN_LEQ", "TOKEN_GT", "TOKEN_GEQ",
   "TOKEN_NEQ", "TOKEN_LIKE", "TOKEN_REGEXP", "TOKEN_BETWEEN", "TOKEN_IS",
   "'+'", "'-'", "'*'", "'/'", "UNARY_PLUS", "UNARY_MINUS", "'.'",
-  "TOKEN_ADD", "TOKEN_ALL", "TOKEN_ALTER", "TOKEN_AS", "TOKEN_ASC",
-  "TOKEN_BIGINT", "TOKEN_BIT", "TOKEN_BITWEAVING", "TOKEN_BLOCKPROPERTIES",
-  "TOKEN_BLOCKSAMPLE", "TOKEN_BLOOM_FILTER", "TOKEN_CSB_TREE", "TOKEN_BY",
-  "TOKEN_CASE", "TOKEN_CHARACTER", "TOKEN_CHECK", "TOKEN_COLUMN",
-  "TOKEN_CONSTRAINT", "TOKEN_COPY", "TOKEN_CREATE", "TOKEN_CURRENT",
-  "TOKEN_DATE", "TOKEN_DATETIME", "TOKEN_DAY", "TOKEN_DECIMAL",
-  "TOKEN_DEFAULT", "TOKEN_DELETE", "TOKEN_DELIMITER", "TOKEN_DESC",
-  "TOKEN_DISTINCT", "TOKEN_DOUBLE", "TOKEN_DROP", "TOKEN_ELSE",
-  "TOKEN_END", "TOKEN_ESCAPE_STRINGS", "TOKEN_EXISTS", "TOKEN_EXTRACT",
-  "TOKEN_FALSE", "TOKEN_FIRST", "TOKEN_FLOAT", "TOKEN_FOLLOWING",
-  "TOKEN_FOR", "TOKEN_FOREIGN", "TOKEN_FROM", "TOKEN_FULL", "TOKEN_GROUP",
-  "TOKEN_HASH", "TOKEN_HAVING", "TOKEN_HOUR", "TOKEN_IN", "TOKEN_INDEX",
-  "TOKEN_INNER", "TOKEN_INSERT", "TOKEN_INTEGER", "TOKEN_INTERVAL",
-  "TOKEN_INTO", "TOKEN_JOIN", "TOKEN_KEY", "TOKEN_LAST", "TOKEN_LEFT",
-  "TOKEN_LIMIT", "TOKEN_LONG", "TOKEN_MINUTE", "TOKEN_MONTH", "TOKEN_NULL",
-  "TOKEN_NULLS", "TOKEN_OFF", "TOKEN_ON", "TOKEN_ORDER", "TOKEN_OUTER",
-  "TOKEN_OVER", "TOKEN_PARTITION", "TOKEN_PARTITIONS", "TOKEN_PERCENT",
+  "TOKEN_ALL", "TOKEN_UNION", "TOKEN_INTERSECT", "TOKEN_ADD",
+  "TOKEN_ALTER", "TOKEN_AS", "TOKEN_ASC", "TOKEN_BIGINT", "TOKEN_BIT",
+  "TOKEN_BITWEAVING", "TOKEN_BLOCKPROPERTIES", "TOKEN_BLOCKSAMPLE",
+  "TOKEN_BLOOM_FILTER", "TOKEN_CSB_TREE", "TOKEN_BY", "TOKEN_CASE",
+  "TOKEN_CHARACTER", "TOKEN_CHECK", "TOKEN_COLUMN", "TOKEN_CONSTRAINT",
+  "TOKEN_COPY", "TOKEN_CREATE", "TOKEN_CURRENT", "TOKEN_DATE",
+  "TOKEN_DATETIME", "TOKEN_DAY", "TOKEN_DECIMAL", "TOKEN_DEFAULT",
+  "TOKEN_DELETE", "TOKEN_DELIMITER", "TOKEN_DESC", "TOKEN_DISTINCT",
+  "TOKEN_DOUBLE", "TOKEN_DROP", "TOKEN_ELSE", "TOKEN_END",
+  "TOKEN_ESCAPE_STRINGS", "TOKEN_EXISTS", "TOKEN_EXTRACT", "TOKEN_FALSE",
+  "TOKEN_FIRST", "TOKEN_FLOAT", "TOKEN_FOLLOWING", "TOKEN_FOR",
+  "TOKEN_FOREIGN", "TOKEN_FROM", "TOKEN_FULL", "TOKEN_GROUP", "TOKEN_HASH",
+  "TOKEN_HAVING", "TOKEN_HOUR", "TOKEN_IN", "TOKEN_INDEX", "TOKEN_INNER",
+  "TOKEN_INSERT", "TOKEN_INTEGER", "TOKEN_INTERVAL", "TOKEN_INTO",
+  "TOKEN_JOIN", "TOKEN_KEY", "TOKEN_LAST", "TOKEN_LEFT", "TOKEN_LIMIT",
+  "TOKEN_LONG", "TOKEN_MINUTE", "TOKEN_MONTH", "TOKEN_NULL", "TOKEN_NULLS",
+  "TOKEN_OFF", "TOKEN_ON", "TOKEN_ORDER", "TOKEN_OUTER", "TOKEN_OVER",
+  "TOKEN_PARTITION", "TOKEN_PARTITIONS", "TOKEN_PERCENT",
   "TOKEN_PRECEDING", "TOKEN_PRIMARY", "TOKEN_PRIORITY", "TOKEN_QUIT",
   "TOKEN_RANGE", "TOKEN_REAL", "TOKEN_REFERENCES", "TOKEN_RIGHT",
   "TOKEN_ROW", "TOKEN_ROW_DELIMITER", "TOKEN_ROWS", "TOKEN_SECOND",
@@ -855,9 +863,10 @@ static const char *const yytname[] =
   "opt_index_properties", "insert_statement", "copy_from_statement",
   "opt_copy_from_params", "copy_from_params", "update_statement",
   "delete_statement", "assignment_list", "assignment_item",
-  "select_statement", "opt_priority_clause", "with_clause", "with_list",
-  "with_list_element", "select_query", "opt_all_distinct", "selection",
-  "selection_item_commalist", "selection_item", "from_clause",
+  "set_operation_statement", "opt_priority_clause", "with_clause",
+  "with_list", "with_list_element", "set_operation_union",
+  "set_operation_intersect", "select_query", "opt_all_distinct",
+  "selection", "selection_item_commalist", "selection_item", "from_clause",
   "subquery_expression", "opt_sample_clause", "join_type",
   "joined_table_reference", "table_reference", "table_reference_signature",
   "table_reference_signature_primary", "joined_table_reference_commalist",
@@ -899,17 +908,17 @@ static const yytype_uint16 yytoknum[] =
      350,   351,   352,   353,   354,   355,   356,   357,   358,   359,
      360,   361,   362,   363,   364,   365,   366,   367,   368,   369,
      370,   371,   372,   373,   374,   375,   376,   377,   378,   379,
-     380,   381,   382,   383,   384,   385,   386,   387,   388,    59,
-      10,    40,    41,    44,    37
+     380,   381,   382,   383,   384,   385,   386,   387,   388,   389,
+     390,    59,    10,    40,    41,    44,    37
 };
 # endif
 
-#define YYPACT_NINF -234
+#define YYPACT_NINF -244
 
 #define yypact_value_is_default(Yystate) \
-  (!!((Yystate) == (-234)))
+  (!!((Yystate) == (-244)))
 
-#define YYTABLE_NINF -130
+#define YYTABLE_NINF -134
 
 #define yytable_value_is_error(Yytable_value) \
   0
@@ -918,60 +927,61 @@ static const yytype_uint16 yytoknum[] =
      STATE-NUM.  */
 static const yytype_int16 yypact[] =
 {
-     168,  -234,  -234,   -58,   181,   -19,    40,   -37,    59,  -234,
-     123,   181,   181,  -234,   208,   127,  -234,  -234,  -234,  -234,
-    -234,  -234,  -234,  -234,  -234,  -234,    -2,    95,   -46,   233,
-     181,  -234,  -234,   113,   181,   181,   181,   181,   181,  -234,
-    -234,   667,   139,   114,  -234,   236,   128,  -234,  -234,  -234,
-     189,    95,   183,  -234,  -234,  -234,  -234,    60,   272,   196,
-     138,   191,  -234,    80,  -234,  -234,   296,   299,  -234,  -234,
-    -234,   757,   187,  -234,   242,  -234,  -234,   190,  -234,  -234,
-     310,  -234,  -234,  -234,  -234,   205,  -234,  -234,   214,   277,
-     850,   355,   290,   219,  -234,  -234,   182,    20,  -234,  -234,
-     269,  -234,  -234,  -234,  -234,  -234,  1030,    -9,   181,   181,
-     228,   181,   181,  -234,   366,    89,   174,   241,   181,   181,
-     577,  -234,  -234,   235,   181,  -234,  -234,  -234,   577,    42,
-      17,  -234,   370,  -234,   126,   126,  1120,   371,  -234,   243,
-      30,  -234,    34,   191,  1120,  -234,  -234,   181,  1120,  -234,
-    -234,  -234,  -234,  1120,    35,   299,  -234,   181,   359,   -56,
-    -234,   368,  -234,   270,  -234,   110,  -234,   270,  -234,   181,
-      -1,   181,   181,   247,  -234,   248,  -234,   141,  1158,   940,
-     228,   487,   378,   381,  -234,  -234,   545,   375,  1210,   143,
-       8,  1120,    58,  -234,  1120,  -234,   330,   254,  -234,  -234,
-    -234,  -234,  -234,  -234,   327,  -234,    50,   255,  -234,  -234,
-      16,   245,   153,  -234,   256,   245,   -14,   331,  -234,  -234,
-      20,  -234,   306,  -234,  -234,   264,  1120,  -234,   268,   151,
-     181,  -234,  1120,  -234,   181,  -234,  -234,   271,   325,   328,
-     274,  -234,  -234,  -234,    31,   181,   291,    -1,   181,  -234,
-      14,  -234,  -234,     2,   326,   577,   577,    62,  -234,  -234,
-    -234,  -234,  -234,  -234,  -234,  -234,  1120,   279,  1120,    39,
-    -234,   154,   289,  1120,    25,  -234,   360,   268,  -234,  -234,
-    1120,   415,  -234,   133,   181,  -234,  -234,   332,  -234,   334,
-     335,   340,    34,  -234,   418,   420,   245,   388,   358,   394,
-     293,   342,  -234,   156,  -234,  1120,  -234,   268,  -234,   577,
-     295,   298,   181,   437,     1,   158,  -234,   167,   416,   144,
-    -234,   301,   311,  -234,   357,   309,  1210,  -234,   362,   181,
-    -234,  -234,    14,  -234,  -234,   381,  -234,  -234,  -234,  1120,
-     312,    76,   850,  -234,   268,   363,  -234,  -234,  1210,   313,
-     268,  1120,  -234,    32,   -13,  -234,  -234,  -234,  -234,  -234,
-      34,   153,   351,   356,  -234,  1120,   577,   364,  1120,  -234,
-     419,   159,  -234,   268,     7,   181,   181,   169,  -234,  -234,
-    -234,  -234,  -234,  -234,  -234,   173,  -234,   181,  -234,  -234,
-    -234,  -234,   315,    -1,   422,   365,  -234,   577,  -234,  -234,
-     322,  -234,   237,   850,  -234,  1120,   171,  -234,  -234,  1210,
-     268,  -234,   458,  -234,   372,  -234,  -234,   329,   378,   429,
-     382,   329,  1120,  -234,  -234,  -234,   452,  -234,   175,   177,
-    -234,   469,     1,  -234,   181,  -234,  -234,   337,   435,  -234,
-       9,   181,  1120,   179,   268,  -234,   184,   341,   577,  1120,
-     472,   349,   343,  -234,   178,    22,   376,  -234,  -234,  -234,
-     192,  -234,  -234,  -234,  -234,    21,   181,   -16,  -234,   346,
-     268,  -234,  -234,  -234,   378,   343,  -234,   181,  -234,   349,
-    -234,  1120,  -234,  -234,   403,   396,   389,   400,   494,   181,
-    -234,   181,  -234,  -234,   181,  -234,   207,  -234,  -234,   373,
-    -234,   473,  -234,  -234,   115,  -234,  -234,  -234,  -234,    41,
-     374,  -234,   209,  -234,   181,   383,  -234,  -234,   438,   395,
-     439,  -234,   181,  -234,   211,   306,  -234,  -234,  -234,   215,
-     411,   377,  -234,   505,  -234,  -234
+     807,  -244,  -244,   -55,   203,   -26,    -2,    -6,    43,  -244,
+      86,   203,   203,  -244,   136,   125,  -244,  -244,  -244,  -244,
+    -244,  -244,  -244,  -244,  -244,  -244,   -22,    -3,   128,  -244,
+     -78,   182,   203,  -244,  -244,   130,   203,   203,   203,   203,
+     203,  -244,  -244,   633,   107,    89,  -244,   209,   120,  -244,
+    -244,  -244,   197,    -3,    86,   161,  -244,   137,  -244,  -244,
+    -244,    27,   265,   180,   141,   170,  -244,   -25,  -244,  -244,
+     301,   313,  -244,  -244,  -244,   660,   188,  -244,   249,  -244,
+    -244,   206,  -244,  -244,   357,  -244,  -244,  -244,  -244,   221,
+    -244,  -244,   222,   282,   783,   366,   299,   231,  -244,  -244,
+     272,     7,  -244,  -244,   277,  -244,  -244,  -244,  -244,  -244,
+     933,    -5,   203,   203,   237,   203,   203,  -244,   137,   374,
+    -244,   233,   239,   246,   203,   203,   551,  -244,  -244,   244,
+     203,  -244,  -244,  -244,   551,    47,    40,  -244,   376,  -244,
+     153,   153,   960,   381,  -244,     8,    24,  -244,    23,   170,
+     960,  -244,  -244,   203,   960,  -244,  -244,  -244,  -244,   960,
+      36,   313,  -244,   203,   363,    94,  -244,   378,  -244,   137,
+    -244,   144,  -244,   137,   128,  -244,   203,   -21,   203,   203,
+     248,  -244,   250,  -244,   147,  1069,   810,   237,   469,   387,
+     388,  -244,  -244,   329,   380,  1042,   152,    45,   960,    48,
+    -244,   960,  -244,   333,   257,  -244,  -244,  -244,  -244,  -244,
+    -244,   330,  -244,   168,   260,  -244,  -244,    19,    66,   146,
+    -244,   264,    66,    51,   335,  -244,  -244,     7,  -244,   304,
+    -244,  -244,   267,   960,  -244,   279,   158,   203,  -244,   960,
+    -244,   203,  -244,  -244,   270,   331,   334,   273,  -244,  -244,
+    -244,   165,   203,   290,   -21,   203,  -244,    77,  -244,  -244,
+       2,    64,   551,   551,   193,  -244,  -244,  -244,  -244,  -244,
+    -244,  -244,  -244,   960,   281,   960,    31,  -244,   162,   294,
+     960,    71,  -244,   364,   279,  -244,  -244,   960,   419,  -244,
+     110,   203,  -244,  -244,   332,  -244,   336,   337,   342,    23,
+    -244,   423,   426,    66,   392,   361,   397,   296,   343,  -244,
+     166,  -244,   960,  -244,   279,  -244,   551,   300,   302,   203,
+     437,   119,   169,  -244,   172,   421,    60,  -244,   305,   316,
+    -244,   355,   310,  1042,  -244,   368,   203,  -244,  -244,    77,
+    -244,  -244,   388,  -244,  -244,  -244,   960,   315,    95,   783,
+    -244,   279,   362,  -244,  -244,  1042,   317,   279,   960,  -244,
+      26,   -15,  -244,  -244,  -244,  -244,  -244,    23,   146,   356,
+     358,  -244,   960,   551,   365,   960,  -244,   420,   111,  -244,
+     279,     4,   203,   203,   174,  -244,  -244,  -244,  -244,  -244,
+    -244,  -244,   201,  -244,   203,  -244,  -244,  -244,  -244,   321,
+     -21,   427,   369,  -244,   551,  -244,  -244,   323,  -244,   238,
+     783,  -244,   960,   179,  -244,  -244,  1042,   279,  -244,   460,
+    -244,   384,  -244,  -244,   326,   387,   435,   394,   326,   960,
+    -244,  -244,  -244,   459,  -244,   181,   183,  -244,   480,   119,
+    -244,   203,  -244,  -244,   345,   447,  -244,     6,   203,   960,
+     185,   279,  -244,   192,   349,   551,   960,   489,   367,   353,
+    -244,   240,    12,   389,  -244,  -244,  -244,   208,  -244,  -244,
+    -244,  -244,    11,   203,    22,  -244,   359,   279,  -244,  -244,
+    -244,   387,   353,  -244,   203,  -244,   367,  -244,   960,  -244,
+    -244,   405,   404,   398,   407,   500,   203,  -244,   203,  -244,
+    -244,   203,  -244,   210,  -244,  -244,   371,  -244,   482,  -244,
+    -244,   129,  -244,  -244,  -244,  -244,    13,   373,  -244,   212,
+    -244,   203,   375,  -244,  -244,   449,   410,   454,  -244,   203,
+    -244,   214,   304,  -244,  -244,  -244,   216,   424,   383,  -244,
+     521,  -244,  -244
 };
 
   /* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM.
@@ -979,92 +989,93 @@ static const yytype_int16 yypact[] =
      means the default is an error.  */
 static const yytype_uint16 yydefact[] =
 {
-       0,     6,   289,     0,     0,     0,     0,     0,     0,    18,
-     114,     0,     0,     7,     0,     0,    15,     8,    10,    11,
-      13,    14,     9,    17,    12,    16,     0,   107,     0,   287,
-       0,   281,   282,     0,     0,     0,     0,     0,     0,   115,
-     116,     0,     0,   109,   110,     0,   147,     1,     3,     2,
-       0,   107,     0,   105,     5,     4,   288,     0,     0,     0,
-       0,   188,    25,     0,   247,   244,     0,   273,   117,    40,
-      29,     0,     0,    30,    31,    34,    36,     0,    37,    39,
-       0,    41,   243,    35,    38,     0,    32,    33,     0,     0,
-       0,     0,     0,   118,   119,   223,   123,   209,   211,   213,
-     216,   219,   220,   221,   215,   214,     0,   259,     0,     0,
-       0,     0,     0,   106,     0,     0,     0,    94,     0,     0,
-       0,   101,   189,     0,     0,    91,   245,   246,     0,     0,
-     239,   236,     0,    43,     0,   248,     0,     0,    44,     0,
-       0,   250,     0,   188,     0,   274,   275,     0,     0,   122,
-     277,   278,   276,     0,     0,     0,   212,     0,     0,   188,
-     103,     0,   111,     0,   112,     0,   279,     0,   108,     0,
-       0,     0,     0,     0,    93,    66,    27,     0,     0,     0,
-       0,     0,   190,   192,   194,   196,     0,   214,     0,     0,
-       0,     0,   239,   233,     0,   237,     0,     0,   253,   254,
-     255,   252,   256,   251,     0,   249,     0,     0,   125,   222,
-       0,     0,   149,   138,   124,   143,   126,   151,   120,   121,
-     208,   210,   165,   217,   260,     0,     0,   224,   241,     0,
-       0,   100,     0,   148,     0,    92,    19,     0,     0,     0,
-       0,    20,    21,    22,     0,     0,     0,    64,     0,    42,
-      56,   195,   203,     0,     0,     0,     0,     0,   263,   265,
-     266,   267,   268,   264,   269,   271,     0,     0,     0,     0,
-     257,     0,     0,     0,     0,   234,     0,   240,   232,    45,
-       0,     0,    46,   129,     0,   139,   145,   135,   130,   131,
-     133,     0,     0,   142,     0,     0,   141,     0,   153,     0,
-       0,   167,   225,     0,   226,     0,   102,   104,   280,     0,
-       0,     0,     0,     0,     0,     0,   261,     0,   259,     0,
-      63,    65,    68,    28,     0,     0,     0,    47,     0,     0,
-      49,    55,    57,    26,   202,   191,   193,   270,   272,     0,
-       0,     0,     0,   204,   201,     0,   200,    90,     0,     0,
-     238,     0,   231,     0,     0,   144,   146,   136,   132,   134,
-       0,   150,     0,     0,   140,     0,     0,   155,     0,   218,
-       0,   169,   227,   242,     0,     0,     0,     0,    96,   285,
-     286,   284,   283,    97,    95,     0,    67,     0,    83,    84,
-      85,    86,    87,     0,     0,    70,    48,     0,    51,    50,
-       0,    54,     0,     0,   206,     0,     0,   199,   258,     0,
-     235,   228,     0,   229,     0,   127,   128,   152,   154,     0,
-     157,   166,     0,   172,   171,   164,     0,    61,     0,     0,
-      58,     0,     0,   262,     0,    24,    62,     0,     0,    23,
-       0,     0,     0,     0,   197,   205,     0,     0,     0,     0,
-       0,   159,   168,   179,   182,     0,     0,    59,    98,    99,
-       0,    74,    76,    77,    78,     0,     0,     0,    52,     0,
-     198,   207,    89,   230,   137,   156,   158,     0,   113,   160,
-     161,     0,   183,   184,   185,     0,     0,     0,     0,     0,
-      88,     0,    82,    80,     0,    79,     0,    72,    73,     0,
-      53,     0,   162,   180,     0,   181,   173,   175,   174,     0,
-       0,    75,     0,    69,     0,     0,   186,   187,     0,     0,
-       0,   170,     0,    81,     0,   165,   176,   178,   177,     0,
-       0,     0,    60,     0,   163,    71
+       0,     6,   293,     0,     0,     0,     0,     0,     0,    18,
+     118,     0,     0,     7,     0,     0,    15,     8,    10,    11,
+      13,    14,     9,    17,    12,    16,     0,   107,   114,   116,
+       0,   291,     0,   285,   286,     0,     0,     0,     0,     0,
+       0,   119,   120,     0,     0,   109,   110,     0,   151,     1,
+       3,     2,     0,   107,   118,     0,   105,     0,     5,     4,
+     292,     0,     0,     0,     0,   192,    25,     0,   251,   248,
+       0,   277,   121,    40,    29,     0,     0,    30,    31,    34,
+      36,     0,    37,    39,     0,    41,   247,    35,    38,     0,
+      32,    33,     0,     0,     0,     0,     0,   122,   123,   227,
+     127,   213,   215,   217,   220,   223,   224,   225,   219,   218,
+       0,   263,     0,     0,     0,     0,     0,   106,     0,     0,
+     115,     0,     0,    94,     0,     0,     0,   101,   193,     0,
+       0,    91,   249,   250,     0,     0,   243,   240,     0,    43,
+       0,   252,     0,     0,    44,     0,     0,   254,     0,   192,
+       0,   278,   279,     0,     0,   126,   281,   282,   280,     0,
+       0,     0,   216,     0,     0,   192,   103,     0,   111,     0,
+     112,     0,   283,     0,   113,   108,     0,     0,     0,     0,
+       0,    93,    66,    27,     0,     0,     0,     0,     0,   194,
+     196,   198,   200,     0,   218,     0,     0,     0,     0,   243,
+     237,     0,   241,     0,     0,   257,   258,   259,   256,   260,
+     255,     0,   253,     0,     0,   129,   226,     0,     0,   153,
+     142,   128,   147,   130,   155,   124,   125,   212,   214,   169,
+     221,   264,     0,     0,   228,   245,     0,     0,   100,     0,
+     152,     0,    92,    19,     0,     0,     0,     0,    20,    21,
+      22,     0,     0,     0,    64,     0,    42,    56,   199,   207,
+       0,     0,     0,     0,     0,   267,   269,   270,   271,   272,
+     268,   273,   275,     0,     0,     0,     0,   261,     0,     0,
+       0,     0,   238,     0,   244,   236,    45,     0,     0,    46,
+     133,     0,   143,   149,   139,   134,   135,   137,     0,     0,
+     146,     0,     0,   145,     0,   157,     0,     0,   171,   229,
+       0,   230,     0,   102,   104,   284,     0,     0,     0,     0,
+       0,     0,     0,   265,     0,   263,     0,    63,    65,    68,
+      28,     0,     0,     0,    47,     0,     0,    49,    55,    57,
+      26,   206,   195,   197,   274,   276,     0,     0,     0,     0,
+     208,   205,     0,   204,    90,     0,     0,   242,     0,   235,
+       0,     0,   148,   150,   140,   136,   138,     0,   154,     0,
+       0,   144,     0,     0,   159,     0,   222,     0,   173,   231,
+     246,     0,     0,     0,     0,    96,   289,   290,   288,   287,
+      97,    95,     0,    67,     0,    83,    84,    85,    86,    87,
+       0,     0,    70,    48,     0,    51,    50,     0,    54,     0,
+       0,   210,     0,     0,   203,   262,     0,   239,   232,     0,
+     233,     0,   131,   132,   156,   158,     0,   161,   170,     0,
+     176,   175,   168,     0,    61,     0,     0,    58,     0,     0,
+     266,     0,    24,    62,     0,     0,    23,     0,     0,     0,
+       0,   201,   209,     0,     0,     0,     0,     0,   163,   172,
+     183,   186,     0,     0,    59,    98,    99,     0,    74,    76,
+      77,    78,     0,     0,     0,    52,     0,   202,   211,    89,
+     234,   141,   160,   162,     0,   117,   164,   165,     0,   187,
+     188,   189,     0,     0,     0,     0,     0,    88,     0,    82,
+      80,     0,    79,     0,    72,    73,     0,    53,     0,   166,
+     184,     0,   185,   177,   179,   178,     0,     0,    75,     0,
+      69,     0,     0,   190,   191,     0,     0,     0,   174,     0,
+      81,     0,   169,   180,   182,   181,     0,     0,     0,    60,
+       0,   167,    71
 };
 
   /* YYPGOTO[NTERM-NUM].  */
 static const yytype_int16 yypgoto[] =
 {
-    -234,  -234,  -234,  -234,  -234,  -234,  -234,  -234,   -94,  -234,
-     338,   185,  -234,  -234,  -233,  -234,  -234,  -234,  -234,  -234,
-    -234,    47,    36,  -234,  -234,  -234,  -234,  -234,  -234,  -234,
-    -234,  -234,  -234,  -234,  -234,   292,  -234,   474,  -234,  -234,
-     414,    11,  -234,  -234,  -234,   386,  -234,   -97,  -234,  -234,
-    -189,   166,  -180,   -10,  -234,  -234,  -234,  -234,  -234,  -234,
-    -234,    52,    10,  -234,  -234,  -234,  -234,  -234,  -234,    83,
-      53,  -234,  -234,   -75,  -234,  -116,   278,   282,   361,   -35,
-     393,   390,   436,  -137,  -234,  -234,  -234,  -234,   352,  -234,
-     417,   354,  -207,  -185,   413,   140,  -119,  -234,  -234,  -234,
-    -234,  -234,  -120,    -4,   120,  -234,  -234
+    -244,  -244,  -244,  -244,  -244,  -244,  -244,  -244,  -132,  -244,
+     346,   191,  -244,  -244,  -243,  -244,  -244,  -244,  -244,  -244,
+    -244,    59,    35,  -244,  -244,  -244,  -244,  -244,  -244,  -244,
+    -244,  -244,  -244,  -244,  -244,   303,  -244,   483,  -244,  -244,
+     428,   200,   425,    -8,   485,  -244,  -244,   395,  -244,   -98,
+    -244,  -244,  -165,   175,  -179,   -10,  -244,  -244,  -244,  -244,
+    -244,  -244,  -244,    58,     5,  -244,  -244,  -244,  -244,  -244,
+    -244,    90,    61,  -244,  -244,    39,  -244,  -133,   285,   287,
+     379,   -37,   400,   403,   438,  -144,  -244,  -244,  -244,  -244,
+     352,  -244,   430,   360,  -212,  -186,   429,   148,  -121,  -244,
+    -244,  -244,  -244,  -244,  -127,    -4,   132,  -244,  -244
 };
 
   /* YYDEFGOTO[NTERM-NUM].  */
 static const yytype_int16 yydefgoto[] =
 {
-      -1,    14,    15,    16,    17,    18,    19,    20,   176,   177,
-      91,   331,   332,   333,   241,   321,   322,   246,   395,   439,
-     499,   460,   461,   462,   463,   464,   392,   435,    21,    22,
-     174,   315,    23,    24,   159,   160,    25,    53,    26,    43,
-      44,   139,    41,    92,    93,    94,   143,    95,   296,   291,
-     212,   213,   285,   286,   214,   298,   367,   420,   451,   478,
-     479,   480,   300,   301,   371,   425,   426,   488,   521,   452,
-     453,   484,   505,   121,   122,   182,   183,   184,   185,   186,
-      97,    98,    99,   100,   101,   102,   103,   192,   193,   130,
-     131,   196,   229,   104,   204,   271,   105,   317,   268,   106,
-     148,   153,   165,   107,   383,    28,    29
+      -1,    14,    15,    16,    17,    18,    19,    20,   183,   184,
+      95,   338,   339,   340,   248,   328,   329,   253,   402,   446,
+     506,   467,   468,   469,   470,   471,   399,   442,    21,    22,
+     181,   322,    23,    24,   165,   166,    25,    56,    26,    45,
+      46,   145,    28,    29,    43,    96,    97,    98,   149,    99,
+     303,   298,   219,   220,   292,   293,   221,   305,   374,   427,
+     458,   485,   486,   487,   307,   308,   378,   432,   433,   495,
+     528,   459,   460,   491,   512,   127,   128,   189,   190,   191,
+     192,   193,   101,   102,   103,   104,   105,   106,   107,   199,
+     200,   136,   137,   203,   236,   108,   211,   278,   109,   324,
+     275,   110,   154,   159,   171,   111,   390,    30,    31
 };
 
   /* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM.  If
@@ -1072,374 +1083,348 @@ static const yytype_int16 yydefgoto[] =
      number is the opposite.  If YYTABLE_NINF, syntax error.  */
 static const yytype_int16 yytable[] =
 {
-      33,   187,    45,   270,   189,   215,    96,    42,    46,   187,
-     255,    27,   190,   164,   320,   255,   255,   255,   157,   303,
-      31,   283,    32,   294,   324,    31,    57,    32,   492,   485,
-      59,    60,    61,    62,    63,   293,   129,    51,    31,    31,
-      32,    32,   237,   150,   151,   211,   145,   146,   518,   345,
-     493,   145,   146,   145,   146,   140,   412,   325,   497,    34,
-     187,    30,   187,   145,   146,   253,   379,   326,   217,   238,
-     486,   145,   146,   215,   125,   236,   120,   194,    50,   337,
-     338,   339,    37,   252,   231,   405,   313,   230,   115,   519,
-     498,    54,   149,   314,    55,   380,   381,   145,   146,    45,
-      35,   206,   239,   361,   161,    46,   327,   166,   167,    96,
-     295,    36,    10,   211,   175,   178,   364,   328,   194,   116,
-     166,   281,   329,   228,   382,   240,   316,   158,   273,   413,
-      10,   346,   158,   169,   170,   406,   187,   187,   216,   340,
-     330,   398,    38,   219,   334,   351,   254,   487,   128,   427,
-     223,   468,    39,   224,   323,   215,   274,   210,   417,   277,
-     436,   421,   494,   408,   152,   178,   520,   242,   243,     1,
-     343,     2,   209,   191,   411,   210,   222,   198,   235,   388,
-      40,   516,   389,   390,    58,    31,    31,    32,    32,   191,
-     187,   228,   377,   374,    10,   211,   443,   307,     3,   145,
-     146,   517,   199,   145,   146,   287,   216,    46,    47,   123,
-     482,    46,   288,   147,     4,     5,   200,   201,   171,   172,
-     289,   124,     6,   215,   270,   287,   161,     7,   431,    52,
-     308,   341,   288,   344,   483,   432,    56,  -129,   350,   202,
-     289,   318,   290,   404,   178,   353,   442,   187,     8,    31,
-     418,    32,   233,   234,   108,   428,   429,   109,   145,   146,
-     391,   203,   290,   211,    48,   423,    49,   110,   433,   111,
-     373,   424,   112,     9,   356,   355,   284,   117,   187,   119,
-      46,   440,    10,   247,   248,   272,   234,   114,   216,   145,
-     146,   118,    46,   304,   305,    11,   347,   348,   372,   305,
-     384,   385,    12,   126,   402,    13,   127,   228,   166,   386,
-     387,   430,   234,   445,   305,   135,   410,   456,   234,   457,
-     234,   471,   305,   120,   133,   400,   472,   348,   132,   187,
-     228,   134,   474,   228,   490,   491,   257,   258,   259,   260,
-     261,   262,   263,   264,   265,   266,   136,   145,   146,   513,
-     491,   523,   234,   530,   234,   137,   216,   532,   234,   138,
-     141,   142,   144,    31,    64,    32,    65,   154,   228,   163,
-     444,   166,   166,   168,   512,   173,   188,   197,   207,   232,
-      66,    67,   225,   318,    10,   208,   255,   454,   244,   245,
-     256,   278,    69,    70,   524,   269,   279,   282,   280,   292,
-      71,    72,   529,   267,   297,   299,   302,   470,    73,    74,
-     310,    75,   309,   311,   454,   312,   226,    76,   349,   319,
-     342,   352,   354,    77,   360,   362,    78,   363,   365,   357,
-     465,   358,   359,   366,   368,   369,   375,   469,   370,   376,
-      79,    80,   378,   157,   393,   394,   454,   399,    81,   396,
-     397,    82,   415,   403,   409,   407,   434,   416,   437,   422,
-     419,   495,   465,   441,   438,   447,    83,   448,   209,   449,
-     450,   455,   305,   501,   458,   467,    84,    85,   466,   476,
-      86,    87,   477,   473,   489,   510,   481,   465,   500,    88,
-     166,    31,    64,    32,    65,    89,   504,   179,   506,   507,
-      90,   227,   508,   509,   515,   527,   526,   528,    66,    67,
-     166,   533,   535,   496,   514,   522,   250,   401,   166,   534,
-      69,    70,   306,   162,   525,   113,   414,   511,    71,    72,
-     218,   502,   475,   335,   503,   531,    73,    74,   336,    75,
-     251,   220,   156,   221,   275,    76,   276,   195,   205,   446,
-     180,    77,   459,     0,    78,   257,   258,   259,   260,   261,
-     262,   263,   264,   265,   266,     0,   145,   146,    79,    80,
-       0,     0,     0,     0,     0,     0,    81,     0,     0,    82,
-       0,    31,    64,    32,    65,     0,     0,   179,     0,     0,
-       0,     0,     0,     0,    83,     0,     0,     0,    66,    67,
-       0,    10,     0,     0,    84,    85,     0,     0,    86,    87,
-      69,    70,     0,     0,     0,     0,     0,    88,    71,    72,
-       0,     0,   267,    89,     0,     0,    73,    74,   181,    75,
-       0,     0,     0,     0,     0,    76,     0,     0,     0,     0,
-     180,    77,     0,     0,    78,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,    79,    80,
-       0,     0,     0,     0,     0,     0,    81,     0,     0,    82,
-       0,    31,    64,    32,    65,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    83,     0,     0,     0,    66,    67,
-      68,     0,     0,     0,    84,    85,     0,     0,    86,    87,
-      69,    70,     0,     0,     0,     0,     0,    88,    71,    72,
-       0,     0,     0,    89,     0,     0,    73,    74,   181,    75,
-       0,     0,     0,     0,     0,    76,     0,     0,     0,     0,
-       0,    77,     0,     0,    78,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,    79,    80,
-       0,     0,     0,     0,     0,     0,    81,     0,     0,    82,
-       0,    31,    64,    32,    65,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,    83,     0,     0,     0,    66,    67,
-       0,     0,     0,     0,    84,    85,     0,     0,    86,    87,
-      69,    70,     0,     0,     0,     0,     0,    88,    71,    72,
-       0,     0,     0,    89,     0,     0,    73,    74,    90,    75,
-       0,     0,     0,     0,     0,    76,     0,     0,     0,     0,
-       0,    77,     0,     0,    78,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,     0,    79,    80,
-       0,     0,     0,     0,     0,     0,    81,     0,     0,    82,
-       0,     0,     0,     0,    31,    64,    32,    65,     0,     0,
-       0,     0,     0,     0,    83,     0,     0,     0,     0,     0,
-       0,    66,    67,     0,    84,    85,     0,     0,    86,    87,
-       0,     0,     0,    69,    70,     0,     0,    88,   128,     0,
-       0,    71,    72,    89,     0,     0,     0,     0,    90,    73,
-      74,     0,    75,     0,     0,     0,     0,     0,    76,     0,
-       0,     0,     0,     0,    77,     0,     0,    78,     0,     0,
+      35,   197,    47,   196,   222,   194,   100,    44,    48,   277,
+     262,   327,   262,   194,   262,    33,   170,    34,   499,   492,
+     525,   310,   163,    33,   244,    34,    54,    33,    61,    34,
+     156,   157,    63,    64,    65,    66,    67,    54,   135,   500,
+      33,   352,    34,   300,   243,   151,   152,   151,   152,   120,
+     218,   245,   290,   262,    36,   260,   419,   146,   121,   131,
+      52,    58,   493,   526,    59,   194,    32,   194,   151,   152,
+      33,    38,    34,   222,   264,   265,   266,   267,   268,   269,
+     270,   271,   272,   273,   246,   151,   152,   331,   122,   259,
+     301,    10,   151,   152,    10,    37,   155,   395,   504,   291,
+     396,   397,   201,    47,   412,   213,   129,   247,   167,    48,
+     201,   172,   173,   100,    41,    39,   151,   152,   130,   218,
+     182,   185,   332,   330,   371,   353,   172,   235,    40,   420,
+     505,   323,   333,    55,   368,    10,    49,   413,   164,   494,
+     527,   194,   194,   274,   223,    42,   341,   405,   434,   226,
+     475,   261,   215,   158,   501,   222,   230,   443,    57,   231,
+     424,   281,   217,   428,   284,   242,   217,   280,   216,   415,
+     418,   334,   185,   134,   249,   250,   350,   302,   398,   229,
+     198,   198,   335,   381,   294,    60,   386,   336,   224,   151,
+     152,   295,   384,   358,   164,   194,   235,   523,   450,   296,
+      27,   218,   314,    62,   238,   337,   205,    33,   216,    34,
+     344,   345,   346,   223,    48,   387,   388,   524,    48,   430,
+     294,   297,   320,   222,   112,   431,    53,   295,   126,   321,
+     277,   206,  -133,   167,   113,   296,   348,   315,   351,   237,
+     425,   288,   114,   357,   389,   207,   208,   449,   325,   411,
+     360,   185,   194,    10,   362,   435,   436,   297,   438,   151,
+     152,   151,   152,   115,    50,   439,    51,   119,   209,   218,
+     123,   447,   347,   440,   489,   380,    33,   124,    34,   176,
+     177,   363,   116,   194,   125,   178,   179,    48,   240,   241,
+     210,   254,   255,   151,   152,   223,   279,   241,   490,    48,
+     151,   152,   311,   312,   126,   153,   354,   355,   132,   409,
+     379,   312,   235,   391,   392,   172,   393,   394,   437,   241,
+     133,   417,   481,   452,   312,   463,   241,   464,   241,   478,
+     312,   138,   407,   139,   194,   235,   479,   355,   235,   264,
+     265,   266,   267,   268,   269,   270,   271,   272,   273,   140,
+     151,   152,   497,   498,   520,   498,   530,   241,   537,   241,
+     539,   241,   141,   223,   142,   143,   144,    33,    68,    34,
+      69,   147,   148,   235,   519,   451,   150,   160,   172,   172,
+     169,   175,   180,   204,    70,    71,   232,   195,   214,   239,
+     325,   251,   461,   252,   531,   262,   285,   263,    73,    74,
+     276,   286,   536,   287,   289,   306,    75,    76,   274,   299,
+     304,   309,   477,   316,    77,    78,   319,    79,   317,   461,
+     326,   318,   233,    80,   349,   356,   361,   359,   367,    81,
+     369,   364,    82,   370,   372,   365,   366,   472,   373,   375,
+     376,   377,   385,   382,   476,   383,    83,    84,   163,   403,
+     400,   461,   401,   404,    85,   406,   414,    86,   410,   422,
+     416,   423,   429,   426,   441,   444,   448,   454,   502,   472,
+     445,   312,    87,    33,    68,    34,    69,   456,   462,   186,
+     508,   455,    88,    89,   457,   465,    90,    91,   473,   474,
+      70,    71,   517,   480,   472,    92,   483,   172,   488,   496,
+     511,    93,   484,   507,    73,    74,    94,   234,   513,   516,
+     514,   515,    75,    76,   521,   522,   529,   172,   532,   533,
+      77,    78,   534,    79,   535,   172,   540,   541,   542,    80,
+     408,   257,   503,   518,   187,    81,   117,   538,    82,   118,
+     313,   168,   421,   174,   509,   225,   482,   342,   162,   510,
+     343,   282,    83,    84,   227,    33,    68,    34,    69,   283,
+      85,   186,   228,    86,   453,   258,   202,     0,     0,     0,
+     212,   466,    70,    71,     0,     0,     0,     0,    87,     0,
+       0,     0,     0,     0,     0,    10,    73,    74,    88,    89,
+       0,     0,    90,    91,    75,    76,     0,     0,     0,     0,
+       0,    92,    77,    78,     0,    79,     0,    93,     0,     0,
+       0,    80,   188,     0,     0,     0,   187,    81,     0,     0,
+      82,     0,     0,     0,     0,     0,     0,     0,     0,     0,
+       0,     0,     0,     0,    83,    84,     0,    33,    68,    34,
+      69,     0,    85,     0,     0,    86,     0,     0,     0,     0,
+       0,     0,     0,     0,    70,    71,    72,     0,     0,     0,
+      87,     0,     0,     0,    33,    68,    34,    69,    73,    74,
+      88,    89,     0,     0,    90,    91,    75,    76,     0,     0,
+       0,    70,    71,    92,    77,    78,     0,    79,     0,    93,
+       0,     0,     0,    80,   188,    73,    74,     0,     0,    81,
+       0,     0,    82,    75,    76,     0,     0,     0,     0,     0,
+       0,    77,    78,     0,    79,     0,    83,    84,     0,     0,
+      80,     0,     0,     0,    85,     0,    81,    86,     0,    82,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,    79,    80,     0,     0,     0,     0,     0,     0,    81,
-       0,     0,    82,     0,    31,    64,    32,    65,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    83,     0,     0,
-       0,    66,    67,     0,    10,     0,     0,    84,    85,     0,
-       0,    86,    87,    69,    70,     0,     0,     0,     0,     0,
-      88,    71,    72,     0,     0,     0,    89,     0,     0,    73,
-      74,    90,    75,     0,     0,     0,     0,     0,    76,     0,
-       0,     0,     0,   180,    77,     0,     0,    78,     0,     0,
+       0,     0,    87,    83,    84,     0,     0,     0,     0,     0,
+       0,    85,    88,    89,    86,     0,    90,    91,     0,     0,
+       0,     0,     0,     0,     0,    92,     0,     0,     0,    87,
+       0,    93,     0,     0,     0,     0,    94,     0,     0,    88,
+      89,     0,     0,    90,    91,     0,     0,    33,    68,    34,
+      69,     0,    92,   134,     0,     0,     0,     0,    93,     0,
+       0,     0,     0,    94,    70,    71,     0,     0,     1,     0,
+       2,     0,     0,     0,    33,    68,    34,    69,    73,    74,
+       0,     0,     0,     0,     0,     0,    75,    76,     0,     0,
+       0,    70,    71,     0,    77,    78,     0,    79,     0,     3,
+       0,     0,     0,    80,     0,    73,    74,     0,     0,    81,
+       0,     0,    82,    75,    76,     4,     5,     0,     0,     0,
+       0,    77,    78,     6,    79,     0,    83,    84,     7,     0,
+      80,     0,     0,     0,    85,   187,    81,    86,     0,    82,
+       0,     0,     0,     0,     0,     0,     0,     0,     0,     8,
+       0,     0,    87,    83,    84,     0,     0,     0,     0,    10,
+       0,    85,    88,    89,    86,     0,    90,    91,     0,     0,
+       0,     0,     0,     0,     9,    92,     0,     0,     0,    87,
+       0,    93,     0,    10,     0,     0,    94,     0,     0,    88,
+      89,     0,     0,    90,    91,     0,    11,    33,    68,    34,
+      69,     0,    92,    12,     0,     0,    13,     0,    93,     0,
+       0,     0,     0,   188,    70,   161,     0,     0,     0,     0,
+       0,     0,     0,     0,    33,    68,    34,    69,    73,    74,
+       0,     0,     0,     0,     0,     0,    75,    76,     0,     0,
+       0,    70,    71,     0,    77,    78,     0,    79,     0,     0,
+       0,     0,     0,    80,     0,    73,    74,     0,     0,    81,
+       0,     0,    82,    75,    76,     0,     0,     0,     0,     0,
+       0,    77,    78,     0,    79,     0,    83,    84,     0,     0,
+      80,     0,     0,     0,    85,     0,    81,    86,     0,    82,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,    79,    80,     0,     0,     0,     0,     0,     0,    81,
-       0,     0,    82,     0,    31,    64,    32,    65,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    83,     0,     0,
-       0,    66,   155,     0,     0,     0,     0,    84,    85,     0,
-       0,    86,    87,    69,    70,     0,     0,     0,     0,     0,
-      88,    71,    72,     0,     0,     0,    89,     0,     0,    73,
-      74,   181,    75,     0,     0,     0,     0,     0,    76,     0,
-       0,     0,     0,     0,    77,     0,     0,    78,     0,     0,
+       0,     0,    87,    83,    84,     0,     0,    68,     0,    69,
+       0,    85,    88,    89,    86,     0,    90,    91,     0,     0,
+       0,     0,     0,    70,   161,    92,     0,     0,     0,    87,
+       0,    93,     0,     0,     0,     0,    94,    73,    74,    88,
+      89,     0,     0,    90,    91,     0,    76,     0,     0,     0,
+       0,     0,    92,    77,    78,     0,    79,     0,    93,     0,
+       0,     0,    80,    94,    73,    74,     0,     0,     0,     0,
+       0,    82,     0,    76,     0,     0,     0,     0,     0,     0,
+      77,    78,     0,    79,     0,    83,    84,     0,     0,    80,
+       0,     0,     0,    85,     0,     0,    86,     0,    82,     0,
        0,     0,     0,     0,     0,     0,     0,     0,     0,     0,
-       0,    79,    80,     0,     0,     0,     0,     0,     0,    81,
-       0,     0,    82,     0,    31,    64,    32,    65,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    83,     0,     0,
-       0,    66,    67,     0,     0,     0,     0,    84,    85,     0,
-       0,    86,    87,    69,    70,     0,     0,     0,     0,     0,
-      88,    71,    72,     0,     0,     0,    89,     0,     0,    73,
-      74,    90,    75,     0,     0,     0,     0,     0,    76,     0,
-       0,     0,     0,     0,    77,     0,     0,    78,     0,     0,
-       0,    69,    70,     0,     0,     0,     0,     0,     0,     0,
-      72,    79,    80,     0,     0,     0,     0,    73,    74,    81,
-      75,     0,    82,     0,     0,    64,    76,    65,     0,     0,
-       0,     0,     0,     0,     0,    78,     0,    83,     0,     0,
-       0,    66,   155,     0,     0,     0,     0,    84,    85,    79,
-     249,    86,    87,    69,    70,     0,     0,    81,     0,     0,
-      88,     0,    72,     0,     0,     0,    89,     0,     0,    73,
-      74,    90,    75,     0,     0,    83,     0,     0,    76,     0,
-       0,     0,     0,     0,     0,    84,     0,    78,     0,    86,
-      87,     0,     0,     0,     0,     0,     0,     0,    88,     0,
-       0,    79,    80,     0,    89,     0,     0,     0,     0,    81,
-       0,     0,    82,     0,     0,     0,     0,     0,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    83,     0,     0,
-       0,     0,     0,     0,     0,     0,     0,    84,     0,     0,
-       0,    86,    87,     0,     0,     0,     0,     0,     0,     0,
-      88,     0,     0,     0,     0,     0,    89
+       0,    87,    83,   256,     0,     0,     0,     0,     0,     0,
+      85,    88,     0,     0,     0,    90,    91,     0,     0,     0,
+       0,     0,     0,     0,    92,     0,     0,     0,    87,     0,
+      93,     0,     0,     0,     0,     0,     0,     0,    88,     0,
+       0,     0,    90,    91,     0,     0,     0,     0,     0,     0,
+       0,    92,     0,     0,     0,     0,     0,    93
 };
 
 static const yytype_int16 yycheck[] =
 {
-       4,   120,    12,   188,   124,   142,    41,    11,    12,   128,
-       8,     0,   128,   110,   247,     8,     8,     8,    27,   226,
-       4,   210,     6,    37,    10,     4,    30,     6,     7,     7,
-      34,    35,    36,    37,    38,   215,    71,    26,     4,     4,
-       6,     6,    43,    23,    24,   142,    21,    22,     7,    10,
-      29,    21,    22,    21,    22,    90,    69,    43,    74,    78,
-     179,   119,   181,    21,    22,   181,    65,    53,   143,    70,
-      48,    21,    22,   210,    63,   169,   132,    60,    80,    17,
-      18,    19,   119,   180,   159,     9,    55,   143,    28,    48,
-     106,   137,    96,    62,   140,    94,    95,    21,    22,   109,
-     119,   136,   103,   292,   108,   109,    92,   111,   112,   144,
-     124,    71,   114,   210,   118,   119,   296,   103,    60,    59,
-     124,    71,   108,   158,   123,   126,   245,   141,   120,   142,
-     114,    92,   141,    44,    45,   342,   255,   256,   142,    77,
-     126,   326,    83,   147,   142,   120,   181,   125,   131,   142,
-     154,   142,    29,   157,   248,   292,   191,   141,   365,   194,
-     393,   368,   141,   348,   144,   169,   125,   171,   172,     1,
-     267,     3,   142,   131,   142,   141,   141,    51,   167,    35,
-      57,    66,    38,    39,    71,     4,     4,     6,     6,   131,
-     309,   226,   312,   309,   114,   292,   403,   232,    30,    21,
-      22,    86,    76,    21,    22,    72,   210,   211,     0,   129,
-      32,   215,    79,    31,    46,    47,    90,    91,    44,    45,
-      87,   141,    54,   360,   409,    72,   230,    59,    55,   134,
-     234,   266,    79,   268,    56,    62,     3,    84,   273,   113,
-      87,   245,   109,   340,   248,   280,     9,   366,    80,     4,
-     366,     6,   142,   143,   115,   375,   376,   143,    21,    22,
-     116,   135,   109,   360,   137,   106,   139,    31,   387,   141,
-     305,   112,    83,   105,   284,   142,    31,     5,   397,   141,
-     284,   397,   114,   142,   143,   142,   143,   104,   292,    21,
-      22,    95,   296,   142,   143,   127,   142,   143,   142,   143,
-     142,   143,   134,     7,   339,   137,     7,   342,   312,   142,
-     143,   142,   143,   142,   143,     5,   351,   142,   143,   142,
-     143,   142,   143,   132,    82,   329,   142,   143,   141,   448,
-     365,   141,   448,   368,   142,   143,    10,    11,    12,    13,
-      14,    15,    16,    17,    18,    19,   141,    21,    22,   142,
-     143,   142,   143,   142,   143,   141,   360,   142,   143,    82,
-       5,    71,   143,     4,     5,     6,     7,    98,   403,   141,
-     405,   375,   376,     7,   494,   134,   141,     7,     7,    11,
-      21,    22,    23,   387,   114,   142,     8,   422,   141,   141,
-       9,    61,    33,    34,   514,    20,   142,   142,    71,   143,
-      41,    42,   522,    77,    73,    99,   142,   442,    49,    50,
-      85,    52,   141,    85,   449,   141,    57,    58,   129,   128,
-     141,    61,     7,    64,    84,     7,    67,     7,    40,    97,
-     434,    97,    97,    75,    40,   142,   141,   441,    96,   141,
-      81,    82,     5,    27,   143,   134,   481,    85,    89,    92,
-     141,    92,   101,   141,   141,    92,   141,   101,    36,    40,
-      96,   465,   466,   141,    99,     7,   107,    95,   142,    40,
-      88,    19,   143,   477,     5,    40,   117,   118,   141,     7,
-     121,   122,   133,   142,   108,   489,   143,   491,   142,   130,
-     494,     4,     5,     6,     7,   136,    93,    10,   102,   110,
-     141,   142,   102,     9,    31,   110,    68,    68,    21,    22,
-     514,   100,     7,   466,   141,   141,   178,   332,   522,   142,
-      33,    34,   230,   109,   141,    51,   360,   491,    41,    42,
-     144,   479,   449,   255,   481,   525,    49,    50,   256,    52,
-     179,   148,   106,   153,   192,    58,   192,   130,   135,   409,
-      63,    64,   432,    -1,    67,    10,    11,    12,    13,    14,
-      15,    16,    17,    18,    19,    -1,    21,    22,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
-      -1,     4,     5,     6,     7,    -1,    -1,    10,    -1,    -1,
-      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    21,    22,
-      -1,   114,    -1,    -1,   117,   118,    -1,    -1,   121,   122,
-      33,    34,    -1,    -1,    -1,    -1,    -1,   130,    41,    42,
-      -1,    -1,    77,   136,    -1,    -1,    49,    50,   141,    52,
-      -1,    -1,    -1,    -1,    -1,    58,    -1,    -1,    -1,    -1,
-      63,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
-      -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    21,    22,
-      23,    -1,    -1,    -1,   117,   118,    -1,    -1,   121,   122,
-      33,    34,    -1,    -1,    -1,    -1,    -1,   130,    41,    42,
-      -1,    -1,    -1,   136,    -1,    -1,    49,    50,   141,    52,
-      -1,    -1,    -1,    -1,    -1,    58,    -1,    -1,    -1,    -1,
-      -1,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
-      -1,     4,     5,     6,     7,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    21,    22,
-      -1,    -1,    -1,    -1,   117,   118,    -1,    -1,   121,   122,
-      33,    34,    -1,    -1,    -1,    -1,    -1,   130,    41,    42,
-      -1,    -1,    -1,   136,    -1,    -1,    49,    50,   141,    52,
-      -1,    -1,    -1,    -1,    -1,    58,    -1,    -1,    -1,    -1,
-      -1,    64,    -1,    -1,    67,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    81,    82,
-      -1,    -1,    -1,    -1,    -1,    -1,    89,    -1,    -1,    92,
-      -1,    -1,    -1,    -1,     4,     5,     6,     7,    -1,    -1,
-      -1,    -1,    -1,    -1,   107,    -1,    -1,    -1,    -1,    -1,
-      -1,    21,    22,    -1,   117,   118,    -1,    -1,   121,   122,
-      -1,    -1,    -1,    33,    34,    -1,    -1,   130,   131,    -1,
-      -1,    41,    42,   136,    -1,    -1,    -1,    -1,   141,    49,
-      50,    -1,    52,    -1,    -1,    -1,    -1,    -1,    58,    -1,
-      -1,    -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,
+       4,   134,    12,   130,   148,   126,    43,    11,    12,   195,
+       8,   254,     8,   134,     8,     4,   114,     6,     7,     7,
+       7,   233,    27,     4,    45,     6,    29,     4,    32,     6,
+      23,    24,    36,    37,    38,    39,    40,    29,    75,    28,
+       4,    10,     6,   222,   176,    21,    22,    21,    22,    57,
+     148,    72,   217,     8,    80,   188,    71,    94,    31,    67,
+      82,   139,    50,    50,   142,   186,   121,   188,    21,    22,
+       4,    73,     6,   217,    10,    11,    12,    13,    14,    15,
+      16,    17,    18,    19,   105,    21,    22,    10,    61,   187,
+      39,   116,    21,    22,   116,   121,   100,    37,    76,    33,
+      40,    41,    62,   113,     9,   142,   131,   128,   112,   113,
+      62,   115,   116,   150,    28,   121,    21,    22,   143,   217,
+     124,   125,    45,   255,   303,    94,   130,   164,    85,   144,
+     108,   252,    55,   136,   299,   116,     0,   349,   143,   127,
+     127,   262,   263,    79,   148,    59,   144,   333,   144,   153,
+     144,   188,   144,   146,   143,   299,   160,   400,    30,   163,
+     372,   198,   143,   375,   201,   173,   143,   122,   144,   355,
+     144,    94,   176,   133,   178,   179,   274,   126,   118,   143,
+     133,   133,   105,   316,    74,     3,    67,   110,   149,    21,
+      22,    81,   319,   122,   143,   316,   233,    68,   410,    89,
+       0,   299,   239,    73,   165,   128,    53,     4,   144,     6,
+      17,    18,    19,   217,   218,    96,    97,    88,   222,   108,
+      74,   111,    57,   367,   117,   114,    26,    81,   134,    64,
+     416,    78,    86,   237,   145,    89,   273,   241,   275,   145,
+     373,    73,    33,   280,   125,    92,    93,     9,   252,   347,
+     287,   255,   373,   116,   144,   382,   383,   111,    57,    21,
+      22,    21,    22,   143,   139,    64,   141,   106,   115,   367,
+       5,   404,    79,   394,    34,   312,     4,    97,     6,    46,
+      47,   291,    85,   404,   143,    46,    47,   291,   144,   145,
+     137,   144,   145,    21,    22,   299,   144,   145,    58,   303,
+      21,    22,   144,   145,   134,    33,   144,   145,     7,   346,
+     144,   145,   349,   144,   145,   319,   144,   145,   144,   145,
+       7,   358,   455,   144,   145,   144,   145,   144,   145,   144,
+     145,   143,   336,    84,   455,   372,   144,   145,   375,    10,
+      11,    12,    13,    14,    15,    16,    17,    18,    19,   143,
+      21,    22,   144,   145,   144,   145,   144,   145,   144,   145,
+     144,   145,     5,   367,   143,   143,    84,     4,     5,     6,
+       7,     5,    73,   410,   501,   412,   145,   100,   382,   383,
+     143,     7,   136,     7,    21,    22,    23,   143,     7,    11,
+     394,   143,   429,   143,   521,     8,    63,     9,    35,    36,
+      20,   144,   529,    73,   144,   101,    43,    44,    79,   145,
+      75,   144,   449,   143,    51,    52,   143,    54,    87,   456,
+     130,    87,    59,    60,   143,   131,     7,    63,    86,    66,
+       7,    99,    69,     7,    42,    99,    99,   441,    77,    42,
+     144,    98,     5,   143,   448,   143,    83,    84,    27,    94,
+     145,   488,   136,   143,    91,    87,    94,    94,   143,   103,
+     143,   103,    42,    98,   143,    38,   143,     7,   472,   473,
+     101,   145,   109,     4,     5,     6,     7,    42,    19,    10,
+     484,    97,   119,   120,    90,     5,   123,   124,   143,    42,
+      21,    22,   496,   144,   498,   132,     7,   501,   145,   110,
+      95,   138,   135,   144,    35,    36,   143,   144,   104,     9,
+     112,   104,    43,    44,   143,    33,   143,   521,   143,    70,
+      51,    52,   112,    54,    70,   529,   102,   144,     7,    60,
+     339,   185,   473,   498,    65,    66,    53,   532,    69,    54,
+     237,   113,   367,   118,   486,   150,   456,   262,   110,   488,
+     263,   199,    83,    84,   154,     4,     5,     6,     7,   199,
+      91,    10,   159,    94,   416,   186,   136,    -1,    -1,    -1,
+     141,   439,    21,    22,    -1,    -1,    -1,    -1,   109,    -1,
+      -1,    -1,    -1,    -1,    -1,   116,    35,    36,   119,   120,
+      -1,    -1,   123,   124,    43,    44,    -1,    -1,    -1,    -1,
+      -1,   132,    51,    52,    -1,    54,    -1,   138,    -1,    -1,
+      -1,    60,   143,    -1,    -1,    -1,    65,    66,    -1,    -1,
+      69,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    83,    84,    -1,     4,     5,     6,
+       7,    -1,    91,    -1,    -1,    94,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,    21,    22,    23,    -1,    -1,    -1,
+     109,    -1,    -1,    -1,     4,     5,     6,     7,    35,    36,
+     119,   120,    -1,    -1,   123,   124,    43,    44,    -1,    -1,
+      -1,    21,    22,   132,    51,    52,    -1,    54,    -1,   138,
+      -1,    -1,    -1,    60,   143,    35,    36,    -1,    -1,    66,
+      -1,    -1,    69,    43,    44,    -1,    -1,    -1,    -1,    -1,
+      -1,    51,    52,    -1,    54,    -1,    83,    84,    -1,    -1,
+      60,    -1,    -1,    -1,    91,    -1,    66,    94,    -1,    69,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,    89,
-      -1,    -1,    92,    -1,     4,     5,     6,     7,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,
-      -1,    21,    22,    -1,   114,    -1,    -1,   117,   118,    -1,
-      -1,   121,   122,    33,    34,    -1,    -1,    -1,    -1,    -1,
-     130,    41,    42,    -1,    -1,    -1,   136,    -1,    -1,    49,
-      50,   141,    52,    -1,    -1,    -1,    -1,    -1,    58,    -1,
-      -1,    -1,    -1,    63,    64,    -1,    -1,    67,    -1,    -1,
+      -1,    -1,   109,    83,    84,    -1,    -1,    -1,    -1,    -1,
+      -1,    91,   119,   120,    94,    -1,   123,   124,    -1,    -1,
+      -1,    -1,    -1,    -1,    -1,   132,    -1,    -1,    -1,   109,
+      -1,   138,    -1,    -1,    -1,    -1,   143,    -1,    -1,   119,
+     120,    -1,    -1,   123,   124,    -1,    -1,     4,     5,     6,
+       7,    -1,   132,   133,    -1,    -1,    -1,    -1,   138,    -1,
+      -1,    -1,    -1,   143,    21,    22,    -1,    -1,     1,    -1,
+       3,    -1,    -1,    -1,     4,     5,     6,     7,    35,    36,
+      -1,    -1,    -1,    -1,    -1,    -1,    43,    44,    -1,    -1,
+      -1,    21,    22,    -1,    51,    52,    -1,    54,    -1,    32,
+      -1,    -1,    -1,    60,    -1,    35,    36,    -1,    -1,    66,
+      -1,    -1,    69,    43,    44,    48,    49,    -1,    -1,    -1,
+      -1,    51,    52,    56,    54,    -1,    83,    84,    61,    -1,
+      60,    -1,    -1,    -1,    91,    65,    66,    94,    -1,    69,
+      -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    82,
+      -1,    -1,   109,    83,    84,    -1,    -1,    -1,    -1,   116,
+      -1,    91,   119,   120,    94,    -1,   123,   124,    -1,    -1,
+      -1,    -1,    -1,    -1,   107,   132,    -1,    -1,    -1,   109,
+      -1,   138,    -1,   116,    -1,    -1,   143,    -1,    -1,   119,
+     120,    -1,    -1,   123,   124,    -1,   129,     4,     5,     6,
+       7,    -1,   132,   136,    -1,    -1,   139,    -1,   138,    -1,
+      -1,    -1,    -1,   143,    21,    22,    -1,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,     4,     5,     6,     7,    35,    36,
+      -1,    -1,    -1,    -1,    -1,    -1,    43,    44,    -1,    -1,
+      -1,    21,    22,    -1,    51,    52,    -1,    54,    -1,    -1,
+      -1,    -1,    -1,    60,    -1,    35,    36,    -1,    -1,    66,
+      -1,    -1,    69,    43,    44,    -1,    -1,    -1,    -1,    -1,
+      -1,    51,    52,    -1,    54,    -1,    83,    84,    -1,    -1,
+      60,    -1,    -1,    -1,    91,    -1,    66,    94,    -1,    69,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,    89,
-      -1,    -1,    92,    -1,     4,     5,     6,     7,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,
-      -1,    21,    22,    -1,    -1,    -1,    -1,   117,   118,    -1,
-      -1,   121,   122,    33,    34,    -1,    -1,    -1,    -1,    -1,
-     130,    41,    42,    -1,    -1,    -1,   136,    -1,    -1,    49,
-      50,   141,    52,    -1,    -1,    -1,    -1,    -1,    58,    -1,
-      -1,    -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,
+      -1,    -1,   109,    83,    84,    -1,    -1,     5,    -1,     7,
+      -1,    91,   119,   120,    94,    -1,   123,   124,    -1,    -1,
+      -1,    -1,    -1,    21,    22,   132,    -1,    -1,    -1,   109,
+      -1,   138,    -1,    -1,    -1,    -1,   143,    35,    36,   119,
+     120,    -1,    -1,   123,   124,    -1,    44,    -1,    -1,    -1,
+      -1,    -1,   132,    51,    52,    -1,    54,    -1,   138,    -1,
+      -1,    -1,    60,   143,    35,    36,    -1,    -1,    -1,    -1,
+      -1,    69,    -1,    44,    -1,    -1,    -1,    -1,    -1,    -1,
+      51,    52,    -1,    54,    -1,    83,    84,    -1,    -1,    60,
+      -1,    -1,    -1,    91,    -1,    -1,    94,    -1,    69,    -1,
       -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    81,    82,    -1,    -1,    -1,    -1,    -1,    -1,    89,
-      -1,    -1,    92,    -1,     4,     5,     6,     7,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,
-      -1,    21,    22,    -1,    -1,    -1,    -1,   117,   118,    -1,
-      -1,   121,   122,    33,    34,    -1,    -1,    -1,    -1,    -1,
-     130,    41,    42,    -1,    -1,    -1,   136,    -1,    -1,    49,
-      50,   141,    52,    -1,    -1,    -1,    -1,    -1,    58,    -1,
-      -1,    -1,    -1,    -1,    64,    -1,    -1,    67,    -1,    -1,
-      -1,    33,    34,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      42,    81,    82,    -1,    -1,    -1,    -1,    49,    50,    89,
-      52,    -1,    92,    -1,    -1,     5,    58,     7,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    67,    -1,   107,    -1,    -1,
-      -1,    21,    22,    -1,    -1,    -1,    -1,   117,   118,    81,
-      82,   121,   122,    33,    34,    -1,    -1,    89,    -1,    -1,
-     130,    -1,    42,    -1,    -1,    -1,   136,    -1,    -1,    49,
-      50,   141,    52,    -1,    -1,   107,    -1,    -1,    58,    -1,
-      -1,    -1,    -1,    -1,    -1,   117,    -1,    67,    -1,   121,
-     122,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   130,    -1,
-      -1,    81,    82,    -1,   136,    -1,    -1,    -1,    -1,    89,
-      -1,    -1,    92,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,   107,    -1,    -1,
-      -1,    -1,    -1,    -1,    -1,    -1,    -1,   117,    -1,    -1,
-      -1,   121,   122,    -1,    -1,    -1,    -1,    -1,    -1,    -1,
-     130,    -1,    -1,    -1,    -1,    -1,   136
+      -1,   109,    83,    84,    -1,    -1,    -1,    -1,    -1,    -1,
+      91,   119,    -1,    -1,    -1,   123,   124,    -1,    -1,    -1,
+      -1,    -1,    -1,    -1,   132,    -1,    -1,    -1,   109,    -1,
+     138,    -1,    -1,    -1,    -1,    -1,    -1,    -1,   119,    -1,
+      -1,    -1,   123,   124,    -1,    -1,    -1,    -1,    -1,    -1,
+      -1,   132,    -1,    -1,    -1,    -1,    -1,   138
 };
 
   /* YYSTOS[STATE-NUM] -- The (internal number of the) accessing
      symbol of state STATE-NUM.  */
 static const yytype_uint8 yystos[] =
 {
-       0,     1,     3,    30,    46,    47,    54,    59,    80,   105,
-     114,   127,   134,   137,   146,   147,   148,   149,   150,   151,
-     152,   173,   174,   177,   178,   181,   183,   186,   250,   251,
-     119,     4,     6,   248,    78,   119,    71,   119,    83,    29,
-      57,   187,   248,   184,   185,   198,   248,     0,   137,   139,
-      80,   186,   134,   182,   137,   140,     3,   248,    71,   248,
-     248,   248,   248,   248,     5,     7,    21,    22,    23,    33,
-      34,    41,    42,    49,    50,    52,    58,    64,    67,    81,
-      82,    89,    92,   107,   117,   118,   121,   122,   130,   136,
-     141,   155,   188,   189,   190,   192,   224,   225,   226,   227,
-     228,   229,   230,   231,   238,   241,   244,   248,   115,   143,
-      31,   141,    83,   182,   104,    28,    59,     5,    95,   141,
-     132,   218,   219,   129,   141,   186,     7,     7,   131,   224,
-     234,   235,   141,    82,   141,     5,   141,   141,    82,   186,
-     224,     5,    71,   191,   143,    21,    22,    31,   245,   248,
-      23,    24,   144,   246,    98,    22,   227,    27,   141,   179,
-     180,   248,   185,   141,   192,   247,   248,   248,     7,    44,
-      45,    44,    45,   134,   175,   248,   153,   154,   248,    10,
-      63,   141,   220,   221,   222,   223,   224,   241,   141,   247,
-     220,   131,   232,   233,    60,   235,   236,     7,    51,    76,
-      90,    91,   113,   135,   239,   239,   224,     7,   142,   142,
-     141,   192,   195,   196,   199,   228,   248,   218,   190,   248,
-     225,   226,   141,   248,   248,    23,    57,   142,   224,   237,
-     143,   218,    11,   142,   143,   186,   153,    43,    70,   103,
-     126,   159,   248,   248,   141,   141,   162,   142,   143,    82,
-     155,   223,   192,   220,   224,     8,     9,    10,    11,    12,
-      13,    14,    15,    16,    17,    18,    19,    77,   243,    20,
-     238,   240,   142,   120,   224,   233,   236,   224,    61,   142,
-      71,    71,   142,   195,    31,   197,   198,    72,    79,    87,
-     109,   194,   143,   197,    37,   124,   193,    73,   200,    99,
-     207,   208,   142,   237,   142,   143,   180,   224,   248,   141,
-      85,    85,   141,    55,    62,   176,   241,   242,   248,   128,
-     159,   160,   161,   153,    10,    43,    53,    92,   103,   108,
-     126,   156,   157,   158,   142,   221,   222,    17,    18,    19,
-      77,   224,   141,   192,   224,    10,    92,   142,   143,   129,
-     224,   120,    61,   224,     7,   142,   198,    97,    97,    97,
-      84,   195,     7,     7,   197,    40,    75,   201,    40,   142,
-      96,   209,   142,   224,   220,   141,   141,   247,     5,    65,
-      94,    95,   123,   249,   142,   143,   142,   143,    35,    38,
-      39,   116,   171,   143,   134,   163,    92,   141,   238,    85,
-     248,   156,   224,   141,   192,     9,   237,    92,   238,   141,
-     224,   142,    69,   142,   196,   101,   101,   237,   220,    96,
-     202,   237,    40,   106,   112,   210,   211,   142,   247,   247,
-     142,    55,    62,   241,   141,   172,   159,    36,    99,   164,
-     220,   141,     9,   237,   224,   142,   240,     7,    95,    40,
-      88,   203,   214,   215,   224,    19,   142,   142,     5,   249,
-     166,   167,   168,   169,   170,   248,   141,    40,   142,   248,
-     224,   142,   142,   142,   220,   214,     7,   133,   204,   205,
-     206,   143,    32,    56,   216,     7,    48,   125,   212,   108,
-     142,   143,     7,    29,   141,   248,   166,    74,   106,   165,
-     142,   248,   206,   215,    93,   217,   102,   110,   102,     9,
-     248,   167,   247,   142,   141,    31,    66,    86,     7,    48,
-     125,   213,   141,   142,   247,   141,    68,   110,    68,   247,
-     142,   207,   142,   100,   142,     7
+       0,     1,     3,    32,    48,    49,    56,    61,    82,   107,
+     116,   129,   136,   139,   148,   149,   150,   151,   152,   153,
+     154,   175,   176,   179,   180,   183,   185,   188,   189,   190,
+     254,   255,   121,     4,     6,   252,    80,   121,    73,   121,
+      85,    28,    59,   191,   252,   186,   187,   202,   252,     0,
+     139,   141,    82,   188,    29,   136,   184,    30,   139,   142,
+       3,   252,    73,   252,   252,   252,   252,   252,     5,     7,
+      21,    22,    23,    35,    36,    43,    44,    51,    52,    54,
+      60,    66,    69,    83,    84,    91,    94,   109,   119,   120,
+     123,   124,   132,   138,   143,   157,   192,   193,   194,   196,
+     228,   229,   230,   231,   232,   233,   234,   235,   242,   245,
+     248,   252,   117,   145,    33,   143,    85,   184,   191,   106,
+     190,    31,    61,     5,    97,   143,   134,   222,   223,   131,
+     143,   190,     7,     7,   133,   228,   238,   239,   143,    84,
+     143,     5,   143,   143,    84,   188,   228,     5,    73,   195,
+     145,    21,    22,    33,   249,   252,    23,    24,   146,   250,
+     100,    22,   231,    27,   143,   181,   182,   252,   187,   143,
+     196,   251,   252,   252,   189,     7,    46,    47,    46,    47,
+     136,   177,   252,   155,   156,   252,    10,    65,   143,   224,
+     225,   226,   227,   228,   245,   143,   251,   224,   133,   236,
+     237,    62,   239,   240,     7,    53,    78,    92,    93,   115,
+     137,   243,   243,   228,     7,   144,   144,   143,   196,   199,
+     200,   203,   232,   252,   222,   194,   252,   229,   230,   143,
+     252,   252,    23,    59,   144,   228,   241,   145,   222,    11,
+     144,   145,   190,   155,    45,    72,   105,   128,   161,   252,
+     252,   143,   143,   164,   144,   145,    84,   157,   227,   196,
+     224,   228,     8,     9,    10,    11,    12,    13,    14,    15,
+      16,    17,    18,    19,    79,   247,    20,   242,   244,   144,
+     122,   228,   237,   240,   228,    63,   144,    73,    73,   144,
+     199,    33,   201,   202,    74,    81,    89,   111,   198,   145,
+     201,    39,   126,   197,    75,   204,   101,   211,   212,   144,
+     241,   144,   145,   182,   228,   252,   143,    87,    87,   143,
+      57,    64,   178,   245,   246,   252,   130,   161,   162,   163,
+     155,    10,    45,    55,    94,   105,   110,   128,   158,   159,
+     160,   144,   225,   226,    17,    18,    19,    79,   228,   143,
+     196,   228,    10,    94,   144,   145,   131,   228,   122,    63,
+     228,     7,   144,   202,    99,    99,    99,    86,   199,     7,
+       7,   201,    42,    77,   205,    42,   144,    98,   213,   144,
+     228,   224,   143,   143,   251,     5,    67,    96,    97,   125,
+     253,   144,   145,   144,   145,    37,    40,    41,   118,   173,
+     145,   136,   165,    94,   143,   242,    87,   252,   158,   228,
+     143,   196,     9,   241,    94,   242,   143,   228,   144,    71,
+     144,   200,   103,   103,   241,   224,    98,   206,   241,    42,
+     108,   114,   214,   215,   144,   251,   251,   144,    57,    64,
+     245,   143,   174,   161,    38,   101,   166,   224,   143,     9,
+     241,   228,   144,   244,     7,    97,    42,    90,   207,   218,
+     219,   228,    19,   144,   144,     5,   253,   168,   169,   170,
+     171,   172,   252,   143,    42,   144,   252,   228,   144,   144,
+     144,   224,   218,     7,   135,   208,   209,   210,   145,    34,
+      58,   220,     7,    50,   127,   216,   110,   144,   145,     7,
+      28,   143,   252,   168,    76,   108,   167,   144,   252,   210,
+     219,    95,   221,   104,   112,   104,     9,   252,   169,   251,
+     144,   143,    33,    68,    88,     7,    50,   127,   217,   143,
+     144,   251,   143,    70,   112,    70,   251,   144,   211,   144,
+     102,   144,     7
 };
 
   /* YYR1[YYN] -- Symbol number of symbol that rule YYN derives.  */
 static const yytype_uint8 yyr1[] =
 {
-       0,   145,   146,   146,   146,   146,   146,   146,   147,   147,
-     147,   147,   147,   147,   147,   147,   147,   147,   148,   149,
-     149,   149,   149,   150,   151,   152,   153,   154,   154,   155,
-     155,   155,   155,   155,   155,   155,   155,   155,   155,   155,
-     155,   155,   155,   155,   155,   155,   155,   156,   156,   156,
-     156,   156,   156,   156,   157,   157,   158,   158,   159,   159,
-     159,   159,   160,   160,   161,   161,   162,   162,   163,   163,
-     164,   164,   165,   165,   166,   166,   167,   167,   167,   168,
-     168,   169,   170,   171,   171,   171,   171,   172,   172,   173,
-     173,   173,   173,   174,   175,   175,   176,   176,   176,   176,
-     177,   178,   179,   179,   180,   181,   181,   182,   182,   183,
-     184,   184,   185,   186,   187,   187,   187,   188,   188,   189,
-     189,   190,   190,   190,   191,   192,   193,   193,   193,   194,
-     194,   194,   194,   194,   194,   194,   194,   195,   195,   196,
-     196,   196,   196,   196,   196,   197,   197,   198,   198,   199,
-     199,   200,   200,   201,   201,   202,   202,   203,   203,   204,
-     204,   205,   205,   206,   207,   208,   208,   209,   209,   210,
-     210,   211,   211,   212,   212,   212,   213,   213,   213,   214,
-     214,   215,   216,   216,   216,   217,   217,   217,   218,   218,
-     219,   220,   220,   221,   221,   222,   222,   223,   223,   223,
-     223,   223,   223,   223,   223,   223,   223,   223,   224,   224,
-     225,   225,   226,   226,   227,   227,   227,   227,   227,   227,
-     227,   227,   227,   227,   228,   228,   228,   228,   229,   230,
-     230,   231,   231,   232,   232,   233,   234,   234,   235,   236,
-     236,   237,   237,   238,   238,   238,   238,   238,   238,   238,
-     238,   239,   239,   239,   239,   239,   239,   240,   240,   241,
-     241,   242,   242,   243,   243,   243,   243,   243,   243,   243,
-     243,   243,   243,   244,   245,   245,   246,   246,   246,   247,
-     247,   248,   248,   249,   249,   249,   249,   250,   251,   251
+       0,   147,   148,   148,   148,   148,   148,   148,   149,   149,
+     149,   149,   149,   149,   149,   149,   149,   149,   150,   151,
+     151,   151,   151,   152,   153,   154,   155,   156,   156,   157,
+     157,   157,   157,   157,   157,   157,   157,   157,   157,   157,
+     157,   157,   157,   157,   157,   157,   157,   158,   158,   158,
+     158,   158,   158,   158,   159,   159,   160,   160,   161,   161,
+     161,   161,   162,   162,   163,   163,   164,   164,   165,   165,
+     166,   166,   167,   167,   168,   168,   169,   169,   169,   170,
+     170,   171,   172,   173,   173,   173,   173,   174,   174,   175,
+     175,   175,   175,   176,   177,   177,   178,   178,   178,   178,
+     179,   180,   181,   181,   182,   183,   183,   184,   184,   185,
+     186,   186,   187,   188,   188,   189,   189,   190,   191,   191,
+     191,   192,   192,   193,   193,   194,   194,   194,   195,   196,
+     197,   197,   197,   198,   198,   198,   198,   198,   198,   198,
+     198,   199,   199,   200,   200,   200,   200,   200,   200,   201,
+     201,   202,   202,   203,   203,   204,   204,   205,   205,   206,
+     206,   207,   207,   208,   208,   209,   209,   210,   211,   212,
+     212,   213,   213,   214,   214,   215,   215,   216,   216,   216,
+     217,   217,   217,   218,   218,   219,   220,   220,   220,   221,
+     221,   221,   222,   222,   223,   224,   224,   225,   225,   226,
+     226,   227,   

<TRUNCATED>


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

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/preprocessed/SqlParser_gen.hpp
----------------------------------------------------------------------
diff --git a/parser/preprocessed/SqlParser_gen.hpp b/parser/preprocessed/SqlParser_gen.hpp
index f1876b8..857eaaf 100644
--- a/parser/preprocessed/SqlParser_gen.hpp
+++ b/parser/preprocessed/SqlParser_gen.hpp
@@ -65,117 +65,119 @@ extern int quickstep_yydebug;
     TOKEN_IS = 275,
     UNARY_PLUS = 276,
     UNARY_MINUS = 277,
-    TOKEN_ADD = 278,
-    TOKEN_ALL = 279,
-    TOKEN_ALTER = 280,
-    TOKEN_AS = 281,
-    TOKEN_ASC = 282,
-    TOKEN_BIGINT = 283,
-    TOKEN_BIT = 284,
-    TOKEN_BITWEAVING = 285,
-    TOKEN_BLOCKPROPERTIES = 286,
-    TOKEN_BLOCKSAMPLE = 287,
-    TOKEN_BLOOM_FILTER = 288,
-    TOKEN_CSB_TREE = 289,
-    TOKEN_BY = 290,
-    TOKEN_CASE = 291,
-    TOKEN_CHARACTER = 292,
-    TOKEN_CHECK = 293,
-    TOKEN_COLUMN = 294,
-    TOKEN_CONSTRAINT = 295,
-    TOKEN_COPY = 296,
-    TOKEN_CREATE = 297,
-    TOKEN_CURRENT = 298,
-    TOKEN_DATE = 299,
-    TOKEN_DATETIME = 300,
-    TOKEN_DAY = 301,
-    TOKEN_DECIMAL = 302,
-    TOKEN_DEFAULT = 303,
-    TOKEN_DELETE = 304,
-    TOKEN_DELIMITER = 305,
-    TOKEN_DESC = 306,
-    TOKEN_DISTINCT = 307,
-    TOKEN_DOUBLE = 308,
-    TOKEN_DROP = 309,
-    TOKEN_ELSE = 310,
-    TOKEN_END = 311,
-    TOKEN_ESCAPE_STRINGS = 312,
-    TOKEN_EXISTS = 313,
-    TOKEN_EXTRACT = 314,
-    TOKEN_FALSE = 315,
-    TOKEN_FIRST = 316,
-    TOKEN_FLOAT = 317,
-    TOKEN_FOLLOWING = 318,
-    TOKEN_FOR = 319,
-    TOKEN_FOREIGN = 320,
-    TOKEN_FROM = 321,
-    TOKEN_FULL = 322,
-    TOKEN_GROUP = 323,
-    TOKEN_HASH = 324,
-    TOKEN_HAVING = 325,
-    TOKEN_HOUR = 326,
-    TOKEN_IN = 327,
-    TOKEN_INDEX = 328,
-    TOKEN_INNER = 329,
-    TOKEN_INSERT = 330,
-    TOKEN_INTEGER = 331,
-    TOKEN_INTERVAL = 332,
-    TOKEN_INTO = 333,
-    TOKEN_JOIN = 334,
-    TOKEN_KEY = 335,
-    TOKEN_LAST = 336,
-    TOKEN_LEFT = 337,
-    TOKEN_LIMIT = 338,
-    TOKEN_LONG = 339,
-    TOKEN_MINUTE = 340,
-    TOKEN_MONTH = 341,
-    TOKEN_NULL = 342,
-    TOKEN_NULLS = 343,
-    TOKEN_OFF = 344,
-    TOKEN_ON = 345,
-    TOKEN_ORDER = 346,
-    TOKEN_OUTER = 347,
-    TOKEN_OVER = 348,
-    TOKEN_PARTITION = 349,
-    TOKEN_PARTITIONS = 350,
-    TOKEN_PERCENT = 351,
-    TOKEN_PRECEDING = 352,
-    TOKEN_PRIMARY = 353,
-    TOKEN_PRIORITY = 354,
-    TOKEN_QUIT = 355,
-    TOKEN_RANGE = 356,
-    TOKEN_REAL = 357,
-    TOKEN_REFERENCES = 358,
-    TOKEN_RIGHT = 359,
-    TOKEN_ROW = 360,
-    TOKEN_ROW_DELIMITER = 361,
-    TOKEN_ROWS = 362,
-    TOKEN_SECOND = 363,
-    TOKEN_SELECT = 364,
-    TOKEN_SET = 365,
-    TOKEN_SMA = 366,
-    TOKEN_SMALLINT = 367,
-    TOKEN_SUBSTRING = 368,
-    TOKEN_TABLE = 369,
-    TOKEN_THEN = 370,
-    TOKEN_TIME = 371,
-    TOKEN_TIMESTAMP = 372,
-    TOKEN_TRUE = 373,
-    TOKEN_TUPLESAMPLE = 374,
-    TOKEN_UNBOUNDED = 375,
-    TOKEN_UNIQUE = 376,
-    TOKEN_UPDATE = 377,
-    TOKEN_USING = 378,
-    TOKEN_VALUES = 379,
-    TOKEN_VARCHAR = 380,
-    TOKEN_WHEN = 381,
-    TOKEN_WHERE = 382,
-    TOKEN_WINDOW = 383,
-    TOKEN_WITH = 384,
-    TOKEN_YEAR = 385,
-    TOKEN_YEARMONTH = 386,
-    TOKEN_EOF = 387,
-    TOKEN_LEX_ERROR = 388
+    TOKEN_ALL = 278,
+    TOKEN_UNION = 279,
+    TOKEN_INTERSECT = 280,
+    TOKEN_ADD = 281,
+    TOKEN_ALTER = 282,
+    TOKEN_AS = 283,
+    TOKEN_ASC = 284,
+    TOKEN_BIGINT = 285,
+    TOKEN_BIT = 286,
+    TOKEN_BITWEAVING = 287,
+    TOKEN_BLOCKPROPERTIES = 288,
+    TOKEN_BLOCKSAMPLE = 289,
+    TOKEN_BLOOM_FILTER = 290,
+    TOKEN_CSB_TREE = 291,
+    TOKEN_BY = 292,
+    TOKEN_CASE = 293,
+    TOKEN_CHARACTER = 294,
+    TOKEN_CHECK = 295,
+    TOKEN_COLUMN = 296,
+    TOKEN_CONSTRAINT = 297,
+    TOKEN_COPY = 298,
+    TOKEN_CREATE = 299,
+    TOKEN_CURRENT = 300,
+    TOKEN_DATE = 301,
+    TOKEN_DATETIME = 302,
+    TOKEN_DAY = 303,
+    TOKEN_DECIMAL = 304,
+    TOKEN_DEFAULT = 305,
+    TOKEN_DELETE = 306,
+    TOKEN_DELIMITER = 307,
+    TOKEN_DESC = 308,
+    TOKEN_DISTINCT = 309,
+    TOKEN_DOUBLE = 310,
+    TOKEN_DROP = 311,
+    TOKEN_ELSE = 312,
+    TOKEN_END = 313,
+    TOKEN_ESCAPE_STRINGS = 314,
+    TOKEN_EXISTS = 315,
+    TOKEN_EXTRACT = 316,
+    TOKEN_FALSE = 317,
+    TOKEN_FIRST = 318,
+    TOKEN_FLOAT = 319,
+    TOKEN_FOLLOWING = 320,
+    TOKEN_FOR = 321,
+    TOKEN_FOREIGN = 322,
+    TOKEN_FROM = 323,
+    TOKEN_FULL = 324,
+    TOKEN_GROUP = 325,
+    TOKEN_HASH = 326,
+    TOKEN_HAVING = 327,
+    TOKEN_HOUR = 328,
+    TOKEN_IN = 329,
+    TOKEN_INDEX = 330,
+    TOKEN_INNER = 331,
+    TOKEN_INSERT = 332,
+    TOKEN_INTEGER = 333,
+    TOKEN_INTERVAL = 334,
+    TOKEN_INTO = 335,
+    TOKEN_JOIN = 336,
+    TOKEN_KEY = 337,
+    TOKEN_LAST = 338,
+    TOKEN_LEFT = 339,
+    TOKEN_LIMIT = 340,
+    TOKEN_LONG = 341,
+    TOKEN_MINUTE = 342,
+    TOKEN_MONTH = 343,
+    TOKEN_NULL = 344,
+    TOKEN_NULLS = 345,
+    TOKEN_OFF = 346,
+    TOKEN_ON = 347,
+    TOKEN_ORDER = 348,
+    TOKEN_OUTER = 349,
+    TOKEN_OVER = 350,
+    TOKEN_PARTITION = 351,
+    TOKEN_PARTITIONS = 352,
+    TOKEN_PERCENT = 353,
+    TOKEN_PRECEDING = 354,
+    TOKEN_PRIMARY = 355,
+    TOKEN_PRIORITY = 356,
+    TOKEN_QUIT = 357,
+    TOKEN_RANGE = 358,
+    TOKEN_REAL = 359,
+    TOKEN_REFERENCES = 360,
+    TOKEN_RIGHT = 361,
+    TOKEN_ROW = 362,
+    TOKEN_ROW_DELIMITER = 363,
+    TOKEN_ROWS = 364,
+    TOKEN_SECOND = 365,
+    TOKEN_SELECT = 366,
+    TOKEN_SET = 367,
+    TOKEN_SMA = 368,
+    TOKEN_SMALLINT = 369,
+    TOKEN_SUBSTRING = 370,
+    TOKEN_TABLE = 371,
+    TOKEN_THEN = 372,
+    TOKEN_TIME = 373,
+    TOKEN_TIMESTAMP = 374,
+    TOKEN_TRUE = 375,
+    TOKEN_TUPLESAMPLE = 376,
+    TOKEN_UNBOUNDED = 377,
+    TOKEN_UNIQUE = 378,
+    TOKEN_UPDATE = 379,
+    TOKEN_USING = 380,
+    TOKEN_VALUES = 381,
+    TOKEN_VARCHAR = 382,
+    TOKEN_WHEN = 383,
+    TOKEN_WHERE = 384,
+    TOKEN_WINDOW = 385,
+    TOKEN_WITH = 386,
+    TOKEN_YEAR = 387,
+    TOKEN_YEARMONTH = 388,
+    TOKEN_EOF = 389,
+    TOKEN_LEX_ERROR = 390
   };
 #endif
 
@@ -184,7 +186,7 @@ extern int quickstep_yydebug;
 
 union YYSTYPE
 {
-#line 120 "../SqlParser.ypp" /* yacc.c:1915  */
+#line 121 "../SqlParser.ypp" /* yacc.c:1909  */
 
   quickstep::ParseString *string_value_;
 
@@ -216,6 +218,8 @@ union YYSTYPE
   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_;
@@ -243,7 +247,7 @@ union YYSTYPE
   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_;
@@ -284,7 +288,7 @@ union YYSTYPE
 
   quickstep::ParsePriority *opt_priority_clause_;
 
-#line 288 "SqlParser_gen.hpp" /* yacc.c:1915  */
+#line 292 "SqlParser_gen.hpp" /* yacc.c:1909  */
 };
 
 typedef union YYSTYPE YYSTYPE;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/tests/Aggregate.test
----------------------------------------------------------------------
diff --git a/parser/tests/Aggregate.test b/parser/tests/Aggregate.test
index e7b8945..5fc5228 100644
--- a/parser/tests/Aggregate.test
+++ b/parser/tests/Aggregate.test
@@ -17,50 +17,54 @@
 
 SELECT AGG(*), AGG(), AGG(a, b, c) FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-FunctionCall[name=AGG,is_star=true]
-  | +-SelectListItem
-  | | +-FunctionCall[name=AGG]
-  | +-SelectListItem
-  |   +-FunctionCall[name=AGG]
-  |     +-AttributeReference[attribute_name=a]
-  |     +-AttributeReference[attribute_name=b]
-  |     +-AttributeReference[attribute_name=c]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-FunctionCall[name=AGG,is_star=true]
+      | +-SelectListItem
+      | | +-FunctionCall[name=AGG]
+      | +-SelectListItem
+      |   +-FunctionCall[name=AGG]
+      |     +-AttributeReference[attribute_name=a]
+      |     +-AttributeReference[attribute_name=b]
+      |     +-AttributeReference[attribute_name=c]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT AGG()+1, AGG()*2+1+AGG(a, b)/AGG(c, d) FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-Add
-  | |   +-left_operand=FunctionCall[name=AGG]
-  | |   +-right_operand=Literal
-  | |     +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-SelectListItem
-  |   +-Add
-  |     +-left_operand=Add
-  |     | +-left_operand=Multiply
-  |     | | +-left_operand=FunctionCall[name=AGG]
-  |     | | +-right_operand=Literal
-  |     | |   +-NumericLiteral[numeric_string=2,float_like=false]
-  |     | +-right_operand=Literal
-  |     |   +-NumericLiteral[numeric_string=1,float_like=false]
-  |     +-right_operand=Divide
-  |       +-left_operand=FunctionCall[name=AGG]
-  |       | +-AttributeReference[attribute_name=a]
-  |       | +-AttributeReference[attribute_name=b]
-  |       +-right_operand=FunctionCall[name=AGG]
-  |         +-AttributeReference[attribute_name=c]
-  |         +-AttributeReference[attribute_name=d]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-Add
+      | |   +-left_operand=FunctionCall[name=AGG]
+      | |   +-right_operand=Literal
+      | |     +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-SelectListItem
+      |   +-Add
+      |     +-left_operand=Add
+      |     | +-left_operand=Multiply
+      |     | | +-left_operand=FunctionCall[name=AGG]
+      |     | | +-right_operand=Literal
+      |     | |   +-NumericLiteral[numeric_string=2,float_like=false]
+      |     | +-right_operand=Literal
+      |     |   +-NumericLiteral[numeric_string=1,float_like=false]
+      |     +-right_operand=Divide
+      |       +-left_operand=FunctionCall[name=AGG]
+      |       | +-AttributeReference[attribute_name=a]
+      |       | +-AttributeReference[attribute_name=b]
+      |       +-right_operand=FunctionCall[name=AGG]
+      |         +-AttributeReference[attribute_name=c]
+      |         +-AttributeReference[attribute_name=d]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 # Function calls as arguments of another function calls.
@@ -68,25 +72,27 @@ SelectStatement
 # the query is not valid. The query resolver will capture the error.
 SELECT AGG(AGG(AGG()+1)*2, AGG(*)/2.0) FROM test
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-FunctionCall[name=AGG]
-  |     +-Multiply
-  |     | +-left_operand=FunctionCall[name=AGG]
-  |     | | +-Add
-  |     | |   +-left_operand=FunctionCall[name=AGG]
-  |     | |   +-right_operand=Literal
-  |     | |     +-NumericLiteral[numeric_string=1,float_like=false]
-  |     | +-right_operand=Literal
-  |     |   +-NumericLiteral[numeric_string=2,float_like=false]
-  |     +-Divide
-  |       +-left_operand=FunctionCall[name=AGG,is_star=true]
-  |       +-right_operand=Literal
-  |         +-NumericLiteral[numeric_string=2.0,float_like=true]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-FunctionCall[name=AGG]
+      |     +-Multiply
+      |     | +-left_operand=FunctionCall[name=AGG]
+      |     | | +-Add
+      |     | |   +-left_operand=FunctionCall[name=AGG]
+      |     | |   +-right_operand=Literal
+      |     | |     +-NumericLiteral[numeric_string=1,float_like=false]
+      |     | +-right_operand=Literal
+      |     |   +-NumericLiteral[numeric_string=2,float_like=false]
+      |     +-Divide
+      |       +-left_operand=FunctionCall[name=AGG,is_star=true]
+      |       +-right_operand=Literal
+      |         +-NumericLiteral[numeric_string=2.0,float_like=true]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT AGG(*, a) FROM test
@@ -98,93 +104,101 @@ SELECT AGG(*, a) FROM test
 
 SELECT AGG()+1 FROM test GROUP BY a+1, AGG()+1 HAVING AGG()*2>1
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Add
-  |     +-left_operand=FunctionCall[name=AGG]
-  |     +-right_operand=Literal
-  |       +-NumericLiteral[numeric_string=1,float_like=false]
-  +-group_by=GroupBy
-  | +-Add
-  | | +-left_operand=AttributeReference[attribute_name=a]
-  | | +-right_operand=Literal
-  | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-Add
-  |   +-left_operand=FunctionCall[name=AGG]
-  |   +-right_operand=Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-having=HAVING
-  | +-Greater
-  |   +-left_operand=Multiply
-  |   | +-left_operand=FunctionCall[name=AGG]
-  |   | +-right_operand=Literal
-  |   |   +-NumericLiteral[numeric_string=2,float_like=false]
-  |   +-right_operand=Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Add
+      |     +-left_operand=FunctionCall[name=AGG]
+      |     +-right_operand=Literal
+      |       +-NumericLiteral[numeric_string=1,float_like=false]
+      +-group_by=GroupBy
+      | +-Add
+      | | +-left_operand=AttributeReference[attribute_name=a]
+      | | +-right_operand=Literal
+      | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-Add
+      |   +-left_operand=FunctionCall[name=AGG]
+      |   +-right_operand=Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-having=HAVING
+      | +-Greater
+      |   +-left_operand=Multiply
+      |   | +-left_operand=FunctionCall[name=AGG]
+      |   | +-right_operand=Literal
+      |   |   +-NumericLiteral[numeric_string=2,float_like=false]
+      |   +-right_operand=Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1 FROM test HAVING AGG() > 1 AND 1=1
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-having=HAVING
-  | +-And
-  |   +-Greater
-  |   | +-left_operand=FunctionCall[name=AGG]
-  |   | +-right_operand=Literal
-  |   |   +-NumericLiteral[numeric_string=1,float_like=false]
-  |   +-Equal
-  |     +-left_operand=Literal
-  |     | +-NumericLiteral[numeric_string=1,float_like=false]
-  |     +-right_operand=Literal
-  |       +-NumericLiteral[numeric_string=1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-having=HAVING
+      | +-And
+      |   +-Greater
+      |   | +-left_operand=FunctionCall[name=AGG]
+      |   | +-right_operand=Literal
+      |   |   +-NumericLiteral[numeric_string=1,float_like=false]
+      |   +-Equal
+      |     +-left_operand=Literal
+      |     | +-NumericLiteral[numeric_string=1,float_like=false]
+      |     +-right_operand=Literal
+      |       +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT 1 FROM test GROUP BY AGG()+1, AGG()/AGG()
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-group_by=GroupBy
-  | +-Add
-  | | +-left_operand=FunctionCall[name=AGG]
-  | | +-right_operand=Literal
-  | |   +-NumericLiteral[numeric_string=1,float_like=false]
-  | +-Divide
-  |   +-left_operand=FunctionCall[name=AGG]
-  |   +-right_operand=FunctionCall[name=AGG]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-group_by=GroupBy
+      | +-Add
+      | | +-left_operand=FunctionCall[name=AGG]
+      | | +-right_operand=Literal
+      | |   +-NumericLiteral[numeric_string=1,float_like=false]
+      | +-Divide
+      |   +-left_operand=FunctionCall[name=AGG]
+      |   +-right_operand=FunctionCall[name=AGG]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT AGG(DISTINCT x), AGG(y) FROM test GROUP BY z
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  | | +-FunctionCall[name=AGG,is_distinct=true]
-  | |   +-AttributeReference[attribute_name=x]
-  | +-SelectListItem
-  |   +-FunctionCall[name=AGG]
-  |     +-AttributeReference[attribute_name=y]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=z]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      | | +-FunctionCall[name=AGG,is_distinct=true]
+      | |   +-AttributeReference[attribute_name=x]
+      | +-SelectListItem
+      |   +-FunctionCall[name=AGG]
+      |     +-AttributeReference[attribute_name=y]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=z]
+      +-from_clause=
+        +-TableReference[table=test]
 ==
 
 SELECT AGG(DISTINCT FUN(x) + y) * AGG(z)
@@ -192,29 +206,31 @@ FROM test
 GROUP BY z
 HAVING AGG(DISTINCT w) + AGG(s) > 1
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectList
-  | +-SelectListItem
-  |   +-Multiply
-  |     +-left_operand=FunctionCall[name=AGG,is_distinct=true]
-  |     | +-Add
-  |     |   +-left_operand=FunctionCall[name=FUN]
-  |     |   | +-AttributeReference[attribute_name=x]
-  |     |   +-right_operand=AttributeReference[attribute_name=y]
-  |     +-right_operand=FunctionCall[name=AGG]
-  |       +-AttributeReference[attribute_name=z]
-  +-group_by=GroupBy
-  | +-AttributeReference[attribute_name=z]
-  +-having=HAVING
-  | +-Greater
-  |   +-left_operand=Add
-  |   | +-left_operand=FunctionCall[name=AGG,is_distinct=true]
-  |   | | +-AttributeReference[attribute_name=w]
-  |   | +-right_operand=FunctionCall[name=AGG]
-  |   |   +-AttributeReference[attribute_name=s]
-  |   +-right_operand=Literal
-  |     +-NumericLiteral[numeric_string=1,float_like=false]
-  +-from_clause=
-    +-TableReference[table=test]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectList
+      | +-SelectListItem
+      |   +-Multiply
+      |     +-left_operand=FunctionCall[name=AGG,is_distinct=true]
+      |     | +-Add
+      |     |   +-left_operand=FunctionCall[name=FUN]
+      |     |   | +-AttributeReference[attribute_name=x]
+      |     |   +-right_operand=AttributeReference[attribute_name=y]
+      |     +-right_operand=FunctionCall[name=AGG]
+      |       +-AttributeReference[attribute_name=z]
+      +-group_by=GroupBy
+      | +-AttributeReference[attribute_name=z]
+      +-having=HAVING
+      | +-Greater
+      |   +-left_operand=Add
+      |   | +-left_operand=FunctionCall[name=AGG,is_distinct=true]
+      |   | | +-AttributeReference[attribute_name=w]
+      |   | +-right_operand=FunctionCall[name=AGG]
+      |   |   +-AttributeReference[attribute_name=s]
+      |   +-right_operand=Literal
+      |     +-NumericLiteral[numeric_string=1,float_like=false]
+      +-from_clause=
+        +-TableReference[table=test]
 ==

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/parser/tests/CMakeLists.txt b/parser/tests/CMakeLists.txt
index 9b461ec..63bcd1c 100644
--- a/parser/tests/CMakeLists.txt
+++ b/parser/tests/CMakeLists.txt
@@ -68,6 +68,10 @@ add_test(quickstep_parser_tests_ParserTest_select
          quickstep_parser_tests_ParserTest
          "${CMAKE_CURRENT_SOURCE_DIR}/Select.test"
          "${CMAKE_CURRENT_BINARY_DIR}/Select.test")
+add_test(quickstep_parser_tests_ParserTest_setoperation
+         quickstep_parser_tests_ParserTest
+         "${CMAKE_CURRENT_SOURCE_DIR}/SetOperation.test"
+         "${CMAKE_CURRENT_BINARY_DIR}/SetOperation.test")
 add_test(quickstep_parser_tests_ParserTest_tpch
          quickstep_parser_tests_ParserTest
          "${CMAKE_CURRENT_SOURCE_DIR}/TPCH.test"

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/tests/Join.test
----------------------------------------------------------------------
diff --git a/parser/tests/Join.test b/parser/tests/Join.test
index 63edeb4..d8ac83f 100644
--- a/parser/tests/Join.test
+++ b/parser/tests/Join.test
@@ -20,26 +20,31 @@ FROM a JOIN b ON a.w = b.w
        JOIN c ON a.x = c.x
        JOIN d ON a.y = d.y;
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-from_clause=
-    +-JoinedTable[join_type=InnerJoin]
-      +-left_table=JoinedTable[join_type=InnerJoin]
-      | +-left_table=JoinedTable[join_type=InnerJoin]
-      | | +-left_table=TableReference[table=a]
-      | | +-right_table=TableReference[table=b]
-      | | +-join_predicate=Equal
-      | |   +-left_operand=AttributeReference[attribute_name=w,relation_name=a]
-      | |   +-right_operand=AttributeReference[attribute_name=w,relation_name=b]
-      | +-right_table=TableReference[table=c]
-      | +-join_predicate=Equal
-      |   +-left_operand=AttributeReference[attribute_name=x,relation_name=a]
-      |   +-right_operand=AttributeReference[attribute_name=x,relation_name=c]
-      +-right_table=TableReference[table=d]
-      +-join_predicate=Equal
-        +-left_operand=AttributeReference[attribute_name=y,relation_name=a]
-        +-right_operand=AttributeReference[attribute_name=y,relation_name=d]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-from_clause=
+        +-JoinedTable[join_type=InnerJoin]
+          +-left_table=JoinedTable[join_type=InnerJoin]
+          | +-left_table=JoinedTable[join_type=InnerJoin]
+          | | +-left_table=TableReference[table=a]
+          | | +-right_table=TableReference[table=b]
+          | | +-join_predicate=Equal
+          | |   +-left_operand=AttributeReference[attribute_name=w,
+          | |   | relation_name=a]
+          | |   +-right_operand=AttributeReference[attribute_name=w,
+          | |     relation_name=b]
+          | +-right_table=TableReference[table=c]
+          | +-join_predicate=Equal
+          |   +-left_operand=AttributeReference[attribute_name=x,relation_name=a]
+          |   +-right_operand=AttributeReference[attribute_name=x,
+          |     relation_name=c]
+          +-right_table=TableReference[table=d]
+          +-join_predicate=Equal
+            +-left_operand=AttributeReference[attribute_name=y,relation_name=a]
+            +-right_operand=AttributeReference[attribute_name=y,relation_name=d]
 ==
 
 SELECT *
@@ -50,30 +55,36 @@ WHERE a1.x > b1.x
   AND a1.y > c1.y
   AND a1.z > d1.z;
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-from_clause=
-    +-JoinedTable[join_type=InnerJoin]
-      +-left_table=JoinedTable[join_type=InnerJoin]
-      | +-left_table=JoinedTable[join_type=InnerJoin]
-      | | +-left_table=TableReference[table=a]
-      | | | +-table_signature=TableSignature[table_alias=a1]
-      | | +-right_table=TableReference[table=b]
-      | | | +-table_signature=TableSignature[table_alias=b1]
-      | | +-join_predicate=Equal
-      | |   +-left_operand=AttributeReference[attribute_name=w,relation_name=a1]
-      | |   +-right_operand=AttributeReference[attribute_name=w,relation_name=b1]
-      | +-right_table=TableReference[table=c]
-      | | +-table_signature=TableSignature[table_alias=c1]
-      | +-join_predicate=Equal
-      |   +-left_operand=AttributeReference[attribute_name=x,relation_name=a1]
-      |   +-right_operand=AttributeReference[attribute_name=x,relation_name=c1]
-      +-right_table=TableReference[table=d]
-      | +-table_signature=TableSignature[table_alias=d1]
-      +-join_predicate=Equal
-        +-left_operand=AttributeReference[attribute_name=y,relation_name=a1]
-        +-right_operand=AttributeReference[attribute_name=y,relation_name=d1]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-from_clause=
+        +-JoinedTable[join_type=InnerJoin]
+          +-left_table=JoinedTable[join_type=InnerJoin]
+          | +-left_table=JoinedTable[join_type=InnerJoin]
+          | | +-left_table=TableReference[table=a]
+          | | | +-table_signature=TableSignature[table_alias=a1]
+          | | +-right_table=TableReference[table=b]
+          | | | +-table_signature=TableSignature[table_alias=b1]
+          | | +-join_predicate=Equal
+          | |   +-left_operand=AttributeReference[attribute_name=w,
+          | |   | relation_name=a1]
+          | |   +-right_operand=AttributeReference[attribute_name=w,
+          | |     relation_name=b1]
+          | +-right_table=TableReference[table=c]
+          | | +-table_signature=TableSignature[table_alias=c1]
+          | +-join_predicate=Equal
+          |   +-left_operand=AttributeReference[attribute_name=x,
+          |   | relation_name=a1]
+          |   +-right_operand=AttributeReference[attribute_name=x,
+          |     relation_name=c1]
+          +-right_table=TableReference[table=d]
+          | +-table_signature=TableSignature[table_alias=d1]
+          +-join_predicate=Equal
+            +-left_operand=AttributeReference[attribute_name=y,relation_name=a1]
+            +-right_operand=AttributeReference[attribute_name=y,relation_name=d1]
 ==
 
 SELECT *
@@ -81,82 +92,97 @@ FROM a LEFT OUTER JOIN b ON a.w = b.w
        RIGHT OUTER JOIN c ON a.x = c.x
        FULL OUTER JOIN d ON a.y = d.y;
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-from_clause=
-    +-JoinedTable[join_type=FullOuterJoin]
-      +-left_table=JoinedTable[join_type=RightOuterJoin]
-      | +-left_table=JoinedTable[join_type=LeftOuterJoin]
-      | | +-left_table=TableReference[table=a]
-      | | +-right_table=TableReference[table=b]
-      | | +-join_predicate=Equal
-      | |   +-left_operand=AttributeReference[attribute_name=w,relation_name=a]
-      | |   +-right_operand=AttributeReference[attribute_name=w,relation_name=b]
-      | +-right_table=TableReference[table=c]
-      | +-join_predicate=Equal
-      |   +-left_operand=AttributeReference[attribute_name=x,relation_name=a]
-      |   +-right_operand=AttributeReference[attribute_name=x,relation_name=c]
-      +-right_table=TableReference[table=d]
-      +-join_predicate=Equal
-        +-left_operand=AttributeReference[attribute_name=y,relation_name=a]
-        +-right_operand=AttributeReference[attribute_name=y,relation_name=d]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-from_clause=
+        +-JoinedTable[join_type=FullOuterJoin]
+          +-left_table=JoinedTable[join_type=RightOuterJoin]
+          | +-left_table=JoinedTable[join_type=LeftOuterJoin]
+          | | +-left_table=TableReference[table=a]
+          | | +-right_table=TableReference[table=b]
+          | | +-join_predicate=Equal
+          | |   +-left_operand=AttributeReference[attribute_name=w,
+          | |   | relation_name=a]
+          | |   +-right_operand=AttributeReference[attribute_name=w,
+          | |     relation_name=b]
+          | +-right_table=TableReference[table=c]
+          | +-join_predicate=Equal
+          |   +-left_operand=AttributeReference[attribute_name=x,relation_name=a]
+          |   +-right_operand=AttributeReference[attribute_name=x,
+          |     relation_name=c]
+          +-right_table=TableReference[table=d]
+          +-join_predicate=Equal
+            +-left_operand=AttributeReference[attribute_name=y,relation_name=a]
+            +-right_operand=AttributeReference[attribute_name=y,relation_name=d]
 ==
 
 SELECT *
 FROM a INNER JOIN b ON (a.w = b.w OR a.x > b.y);
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-from_clause=
-    +-JoinedTable[join_type=InnerJoin]
-      +-left_table=TableReference[table=a]
-      +-right_table=TableReference[table=b]
-      +-join_predicate=Or
-        +-Equal
-        | +-left_operand=AttributeReference[attribute_name=w,relation_name=a]
-        | +-right_operand=AttributeReference[attribute_name=w,relation_name=b]
-        +-Greater
-          +-left_operand=AttributeReference[attribute_name=x,relation_name=a]
-          +-right_operand=AttributeReference[attribute_name=y,relation_name=b]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-from_clause=
+        +-JoinedTable[join_type=InnerJoin]
+          +-left_table=TableReference[table=a]
+          +-right_table=TableReference[table=b]
+          +-join_predicate=Or
+            +-Equal
+            | +-left_operand=AttributeReference[attribute_name=w,relation_name=a]
+            | +-right_operand=AttributeReference[attribute_name=w,
+            |   relation_name=b]
+            +-Greater
+              +-left_operand=AttributeReference[attribute_name=x,relation_name=a]
+              +-right_operand=AttributeReference[attribute_name=y,
+                relation_name=b]
 ==
 
 SELECT *
 FROM b LEFT JOIN c ON b.x = c.x JOIN d ON c.y = d.y;
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-from_clause=
-    +-JoinedTable[join_type=InnerJoin]
-      +-left_table=JoinedTable[join_type=LeftOuterJoin]
-      | +-left_table=TableReference[table=b]
-      | +-right_table=TableReference[table=c]
-      | +-join_predicate=Equal
-      |   +-left_operand=AttributeReference[attribute_name=x,relation_name=b]
-      |   +-right_operand=AttributeReference[attribute_name=x,relation_name=c]
-      +-right_table=TableReference[table=d]
-      +-join_predicate=Equal
-        +-left_operand=AttributeReference[attribute_name=y,relation_name=c]
-        +-right_operand=AttributeReference[attribute_name=y,relation_name=d]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-from_clause=
+        +-JoinedTable[join_type=InnerJoin]
+          +-left_table=JoinedTable[join_type=LeftOuterJoin]
+          | +-left_table=TableReference[table=b]
+          | +-right_table=TableReference[table=c]
+          | +-join_predicate=Equal
+          |   +-left_operand=AttributeReference[attribute_name=x,relation_name=b]
+          |   +-right_operand=AttributeReference[attribute_name=x,
+          |     relation_name=c]
+          +-right_table=TableReference[table=d]
+          +-join_predicate=Equal
+            +-left_operand=AttributeReference[attribute_name=y,relation_name=c]
+            +-right_operand=AttributeReference[attribute_name=y,relation_name=d]
 ==
 
 SELECT *
 FROM b LEFT JOIN (c JOIN d ON c.y = d.y) ON b.x = c.x;
 --
-SelectStatement
-+-select_query=Select
-  +-select_clause=SelectStar
-  +-from_clause=
-    +-JoinedTable[join_type=LeftOuterJoin]
-      +-left_table=TableReference[table=b]
-      +-right_table=JoinedTable[join_type=InnerJoin]
-      | +-left_table=TableReference[table=c]
-      | +-right_table=TableReference[table=d]
-      | +-join_predicate=Equal
-      |   +-left_operand=AttributeReference[attribute_name=y,relation_name=c]
-      |   +-right_operand=AttributeReference[attribute_name=y,relation_name=d]
-      +-join_predicate=Equal
-        +-left_operand=AttributeReference[attribute_name=x,relation_name=b]
-        +-right_operand=AttributeReference[attribute_name=x,relation_name=c]
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Select]
+  +-children=
+    +-Select
+      +-select_clause=SelectStar
+      +-from_clause=
+        +-JoinedTable[join_type=LeftOuterJoin]
+          +-left_table=TableReference[table=b]
+          +-right_table=JoinedTable[join_type=InnerJoin]
+          | +-left_table=TableReference[table=c]
+          | +-right_table=TableReference[table=d]
+          | +-join_predicate=Equal
+          |   +-left_operand=AttributeReference[attribute_name=y,relation_name=c]
+          |   +-right_operand=AttributeReference[attribute_name=y,
+          |     relation_name=d]
+          +-join_predicate=Equal
+            +-left_operand=AttributeReference[attribute_name=x,relation_name=b]
+            +-right_operand=AttributeReference[attribute_name=x,relation_name=c]


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

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5b7b5cb8/parser/tests/SetOperation.test
----------------------------------------------------------------------
diff --git a/parser/tests/SetOperation.test b/parser/tests/SetOperation.test
new file mode 100644
index 0000000..e86cb2f
--- /dev/null
+++ b/parser/tests/SetOperation.test
@@ -0,0 +1,235 @@
+# 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.
+
+SELECT * FROM a
+UNION ALL
+SELECT * FROM b;
+--
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=UnionAll]
+  +-children=
+    +-SetOperation[set_operation_type=Select]
+    | +-children=
+    |   +-Select
+    |     +-select_clause=SelectStar
+    |     +-from_clause=
+    |       +-TableReference[table=a]
+    +-SetOperation[set_operation_type=Select]
+      +-children=
+        +-Select
+          +-select_clause=SelectStar
+          +-from_clause=
+            +-TableReference[table=b]
+==
+
+# Currently does not support INTERSECT ALL.
+SELECT * FROM a
+INTERSECT ALL
+SELECT * FROM b;
+--
+ERROR: syntax error (2 : 11)
+INTERSECT ALL
+          ^
+==
+
+SELECT * FROM a
+UNION
+SELECT * FROM b
+UNION
+SELECT * FROM c;
+--
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Union]
+  +-children=
+    +-SetOperation[set_operation_type=Union]
+    | +-children=
+    |   +-SetOperation[set_operation_type=Select]
+    |   | +-children=
+    |   |   +-Select
+    |   |     +-select_clause=SelectStar
+    |   |     +-from_clause=
+    |   |       +-TableReference[table=a]
+    |   +-SetOperation[set_operation_type=Select]
+    |     +-children=
+    |       +-Select
+    |         +-select_clause=SelectStar
+    |         +-from_clause=
+    |           +-TableReference[table=b]
+    +-SetOperation[set_operation_type=Select]
+      +-children=
+        +-Select
+          +-select_clause=SelectStar
+          +-from_clause=
+            +-TableReference[table=c]
+==
+
+SELECT * FROM a
+INTERSECT
+SELECT * FROM b
+UNION
+SELECT * FROM c
+INTERSECT
+SELECT * FROM d;
+--
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Union]
+  +-children=
+    +-SetOperation[set_operation_type=Intersect]
+    | +-children=
+    |   +-SetOperation[set_operation_type=Select]
+    |   | +-children=
+    |   |   +-Select
+    |   |     +-select_clause=SelectStar
+    |   |     +-from_clause=
+    |   |       +-TableReference[table=a]
+    |   +-SetOperation[set_operation_type=Select]
+    |     +-children=
+    |       +-Select
+    |         +-select_clause=SelectStar
+    |         +-from_clause=
+    |           +-TableReference[table=b]
+    +-SetOperation[set_operation_type=Intersect]
+      +-children=
+        +-SetOperation[set_operation_type=Select]
+        | +-children=
+        |   +-Select
+        |     +-select_clause=SelectStar
+        |     +-from_clause=
+        |       +-TableReference[table=c]
+        +-SetOperation[set_operation_type=Select]
+          +-children=
+            +-Select
+              +-select_clause=SelectStar
+              +-from_clause=
+                +-TableReference[table=d]
+==
+
+# Currently does not support using bracket to change order,
+# Instead, use a subquery to change order of set operations.
+SELECT * FROM a
+INTERSECT
+(SELECT * FROM a
+UNION
+SELECT * FROM b);
+--
+ERROR: syntax error (3 : 1)
+(SELECT * FROM a
+^
+==
+
+SELECT * FROM a
+INTERSECT
+SELECT * FROM
+    (SELECT * FROM a
+     UNION
+     SELECT * FROM b
+    ) AS temp;
+--
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Intersect]
+  +-children=
+    +-SetOperation[set_operation_type=Select]
+    | +-children=
+    |   +-Select
+    |     +-select_clause=SelectStar
+    |     +-from_clause=
+    |       +-TableReference[table=a]
+    +-SetOperation[set_operation_type=Select]
+      +-children=
+        +-Select
+          +-select_clause=SelectStar
+          +-from_clause=
+            +-SubqueryTable
+              +-table_signature=TableSignature[table_alias=temp]
+              +-SubqueryExpression
+                +-SetOperation[set_operation_type=Union]
+                  +-children=
+                    +-SetOperation[set_operation_type=Select]
+                    | +-children=
+                    |   +-Select
+                    |     +-select_clause=SelectStar
+                    |     +-from_clause=
+                    |       +-TableReference[table=a]
+                    +-SetOperation[set_operation_type=Select]
+                      +-children=
+                        +-Select
+                          +-select_clause=SelectStar
+                          +-from_clause=
+                            +-TableReference[table=b]
+==
+
+SELECT * FROM
+	(SELECT * FROM a
+	UNION ALL
+	SELECT * FROM b
+	) AS temp1
+INTERSECT
+SELECT * FROM
+	(SELECT * FROM a
+	UNION
+	SELECT * FROM b
+	) AS temp2;
+--
+SetOperationStatement
++-set_operation_query=SetOperation[set_operation_type=Intersect]
+  +-children=
+    +-SetOperation[set_operation_type=Select]
+    | +-children=
+    |   +-Select
+    |     +-select_clause=SelectStar
+    |     +-from_clause=
+    |       +-SubqueryTable
+    |         +-table_signature=TableSignature[table_alias=temp1]
+    |         +-SubqueryExpression
+    |           +-SetOperation[set_operation_type=UnionAll]
+    |             +-children=
+    |               +-SetOperation[set_operation_type=Select]
+    |               | +-children=
+    |               |   +-Select
+    |               |     +-select_clause=SelectStar
+    |               |     +-from_clause=
+    |               |       +-TableReference[table=a]
+    |               +-SetOperation[set_operation_type=Select]
+    |                 +-children=
+    |                   +-Select
+    |                     +-select_clause=SelectStar
+    |                     +-from_clause=
+    |                       +-TableReference[table=b]
+    +-SetOperation[set_operation_type=Select]
+      +-children=
+        +-Select
+          +-select_clause=SelectStar
+          +-from_clause=
+            +-SubqueryTable
+              +-table_signature=TableSignature[table_alias=temp2]
+              +-SubqueryExpression
+                +-SetOperation[set_operation_type=Union]
+                  +-children=
+                    +-SetOperation[set_operation_type=Select]
+                    | +-children=
+                    |   +-Select
+                    |     +-select_clause=SelectStar
+                    |     +-from_clause=
+                    |       +-TableReference[table=a]
+                    +-SetOperation[set_operation_type=Select]
+                      +-children=
+                        +-Select
+                          +-select_clause=SelectStar
+                          +-from_clause=
+                            +-TableReference[table=b]
+==
+


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

Posted by ji...@apache.org.
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/master
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);
   };
 


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

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