You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@quickstep.apache.org by Terranlee <gi...@git.apache.org> on 2017/04/06 22:50:12 UTC

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

GitHub user Terranlee opened a pull request:

    https://github.com/apache/incubator-quickstep/pull/229

    Implement parser and resolver for UNION and INTERSECT operators

    This pull request implements the parser and resolver parts of set operations (`UNION` and `INTERSECT`).
    
    It also supports composite set operations, e.g.
    ```
    SELECT * FROM a
    UNION ALL
    SELECT * FROM b
    UNION ALL
    SELECT * FROM c
    INTERSECT
    SELECT * FROM d;
    ``` 
    
    Note: will add parser and resolver tests, so do not merge yet.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/Terranlee/incubator-quickstep union-operator-parser

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-quickstep/pull/229.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #229
    
----
commit ddce20354ab70df1bda53d65f6e2266f50bc5e44
Author: TianrunLi <tl...@wisc.edu>
Date:   2016-12-20T15:23:24Z

    Implement parser and resolver for UNION and INTERSECT.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by Terranlee <gi...@git.apache.org>.
Github user Terranlee commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110910643
  
    --- Diff: parser/SqlParser.ypp ---
    @@ -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
    --- End diff --
    
    This is from the original ypp file, since we do not implement the select query like 'SELECT ALL ...' or 'SELECT DISTINCT ...'


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110303184
  
    --- Diff: query_optimizer/resolver/Resolver.cpp ---
    @@ -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) {
    +          resolveSetOperationDispatcher(*set_operation_statement.set_operation_query(),
    +                              "", /* set operation name */
    --- End diff --
    
    Change to `/* set_operation_name */`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110296317
  
    --- Diff: parser/SqlParser.ypp ---
    @@ -604,8 +615,6 @@ void NotSupported(const YYLTYPE *location, yyscan_t yyscanner, const std::string
       command_argument_list
     
     /*
    -%type <int_val>
    --- End diff --
    
    Keep it, as it is not related to this PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110300978
  
    --- Diff: 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,
    --- End diff --
    
    Please change to `kIntersect = 0,`. Also, I'm curious about `kSingle`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110305172
  
    --- Diff: query_optimizer/resolver/Resolver.hpp ---
    @@ -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 resolveSetOperationDispatcher(
    --- End diff --
    
    How about renaming to `resolveSetOperation`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110302971
  
    --- Diff: 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,
    +    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);
    +    }
    +  }
    +
    +  SetOperationType set_operation_type_;
    +  std::vector<LogicalPtr> operands_;
    +  std::vector<expressions::AttributeReferencePtr> project_attributes_;
    --- End diff --
    
    Please mark both `set_operation_type_` and `project_attributes_` `const`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110305426
  
    --- Diff: query_optimizer/resolver/Resolver.cpp ---
    @@ -1350,16 +1351,172 @@ 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 =
    +      resolveSetOperationDispatcher(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.
    +  ++iter;
    +  for (; iter != operands.end(); ++iter) {
    +    const ParseSetOperation &current_operation =
    +        static_cast<const ParseSetOperation&>(**iter);
    +    L::LogicalPtr current_logical =
    +        resolveSetOperationDispatcher(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();
    +  }
    --- End diff --
    
    Add `return nullptr;` to fix the error in Travis CI.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep issue #229: Implement parser and resolver for UNION and ...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on the issue:

    https://github.com/apache/incubator-quickstep/pull/229
  
    @jianqiao We need to wait for squashing commits into one before merge.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110300377
  
    --- Diff: parser/tests/SetOperation.test ---
    @@ -0,0 +1,38 @@
    +# 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=UnionAll[set_operation_type=UnionAll]
    +  +-children=
    +    +-Single[set_operation_type=Single]
    +    | +-children=
    +    |   +-Select
    +    |     +-select_clause=SelectStar
    +    |     +-from_clause=
    +    |       +-TableReference[table=a]
    +    +-Single[set_operation_type=Single]
    +      +-children=
    +        +-Select
    +          +-select_clause=SelectStar
    +          +-from_clause=
    +            +-TableReference[table=b]
    +==
    +
    --- End diff --
    
    We need more tests to cover all set operation types, and some parser failure queries.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110304392
  
    --- Diff: query_optimizer/resolver/Resolver.cpp ---
    @@ -1350,16 +1351,172 @@ 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 =
    +      resolveSetOperationDispatcher(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.
    +  ++iter;
    +  for (; iter != operands.end(); ++iter) {
    +    const ParseSetOperation &current_operation =
    +        static_cast<const ParseSetOperation&>(**iter);
    +    L::LogicalPtr current_logical =
    +        resolveSetOperationDispatcher(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];
    --- End diff --
    
    Mark `const`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110295083
  
    --- Diff: parser/ParseSetOperation.hpp ---
    @@ -0,0 +1,136 @@
    +/**
    + * 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"
    +
    +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,
    +    kSingle,
    +    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 {
    +    switch (set_operation_type_) {
    +      case kIntersect:
    +        return "Intersect";
    +      case kUnion:
    +        return "Union";
    +      case kUnionAll:
    +        return "UnionAll";
    +      case kSingle:
    +        return "Single";
    +      default:
    +        return "Unknown";
    +    }
    +  }
    +
    +  /**
    +   * @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");
    +    inline_field_values->push_back(getName());
    +
    +    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_;
    +  SetOperationType set_operation_type_;
    --- End diff --
    
    Please mark as `const`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110303143
  
    --- Diff: query_optimizer/resolver/Resolver.cpp ---
    @@ -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) {
    +          resolveSetOperationDispatcher(*set_operation_statement.set_operation_query(),
    +                              "", /* set operation name */
    +                              nullptr /* type hints */,
    +                              nullptr /* parent resolver*/);
    --- End diff --
    
    Reset the changes regarding `parent_resolver`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep issue #229: Implement parser and resolver for UNION and ...

Posted by jianqiao <gi...@git.apache.org>.
Github user jianqiao commented on the issue:

    https://github.com/apache/incubator-quickstep/pull/229
  
    LGTM! Merging.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110295617
  
    --- Diff: parser/ParseStatement.hpp ---
    @@ -479,49 +480,49 @@ class ParseStatementDropTable : public ParseStatement {
     };
     
     /**
    - * @brief The parsed representation of a SELECT statement.
    + * @brief The parsed representation of a UNION/INTERSECT/SELECT statement.
    --- End diff --
    
    Typo: `a` to `an`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110295303
  
    --- Diff: parser/ParseSetOperation.hpp ---
    @@ -0,0 +1,136 @@
    +/**
    + * 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"
    +
    +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,
    +    kSingle,
    --- End diff --
    
    Question: does `kSingle` has an extra meaning beyond representing a `SELECT` query?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by jianqiao <gi...@git.apache.org>.
Github user jianqiao commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110523124
  
    --- Diff: parser/SqlParser.ypp ---
    @@ -676,7 +686,7 @@ sql_statement:
       | quit_statement {
         $$ = $1;
       }
    -  | select_statement {
    +  | set_operation_statement {
    --- End diff --
    
    That will leads to a conflict -- the parser cannot decide whether a `SELECT` query should be reduced to a select statement or a set operation statement.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110304313
  
    --- Diff: query_optimizer/resolver/Resolver.cpp ---
    @@ -1350,16 +1351,172 @@ 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 =
    +      resolveSetOperationDispatcher(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.
    +  ++iter;
    +  for (; iter != operands.end(); ++iter) {
    +    const ParseSetOperation &current_operation =
    +        static_cast<const ParseSetOperation&>(**iter);
    +    L::LogicalPtr current_logical =
    +        resolveSetOperationDispatcher(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)
    --- End diff --
    
    Reduce to two-whitespace-indentation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110303831
  
    --- Diff: query_optimizer/resolver/Resolver.cpp ---
    @@ -1350,16 +1351,172 @@ 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 =
    +      resolveSetOperationDispatcher(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.
    +  ++iter;
    +  for (; iter != operands.end(); ++iter) {
    --- End diff --
    
    We could merge `++iter;` in Line 1377 inside the for-loop.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110304705
  
    --- Diff: query_optimizer/resolver/Resolver.cpp ---
    @@ -1350,16 +1351,172 @@ 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 =
    +      resolveSetOperationDispatcher(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.
    +  ++iter;
    +  for (; iter != operands.end(); ++iter) {
    +    const ParseSetOperation &current_operation =
    +        static_cast<const ParseSetOperation&>(**iter);
    +    L::LogicalPtr current_logical =
    +        resolveSetOperationDispatcher(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*/,
    --- End diff --
    
    Change to `"" /* relation_name */,`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110294613
  
    --- Diff: parser/ParseSetOperation.hpp ---
    @@ -0,0 +1,136 @@
    +/**
    + * 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"
    +
    +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,
    --- End diff --
    
    Please change to `kIntersect = 0,`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110294927
  
    --- Diff: parser/ParseSetOperation.hpp ---
    @@ -0,0 +1,136 @@
    +/**
    + * 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"
    +
    +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,
    +    kSingle,
    +    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 {
    +    switch (set_operation_type_) {
    +      case kIntersect:
    +        return "Intersect";
    +      case kUnion:
    +        return "Union";
    +      case kUnionAll:
    +        return "UnionAll";
    +      case kSingle:
    +        return "Single";
    +      default:
    +        return "Unknown";
    --- End diff --
    
    Suggest to change to `LOG(FATAL) << "Unknown set operation type.";`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110511246
  
    --- Diff: parser/SqlParser.ypp ---
    @@ -676,7 +686,7 @@ sql_statement:
       | quit_statement {
         $$ = $1;
       }
    -  | select_statement {
    +  | set_operation_statement {
    --- End diff --
    
    What if we add `set_operation_statement`, but still keep `select_statement` so that we don't need to modify test files with `SELECT`-only queries?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110304131
  
    --- Diff: query_optimizer/resolver/Resolver.cpp ---
    @@ -1350,16 +1351,172 @@ 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 =
    +      resolveSetOperationDispatcher(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.
    +  ++iter;
    +  for (; iter != operands.end(); ++iter) {
    +    const ParseSetOperation &current_operation =
    +        static_cast<const ParseSetOperation&>(**iter);
    +    L::LogicalPtr current_logical =
    +        resolveSetOperationDispatcher(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)
    --- End diff --
    
    Reduce to two-whitespace-indentation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110817047
  
    --- Diff: parser/SqlParser.ypp ---
    @@ -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
    --- End diff --
    
    I observed the following warning:
    
    ```
    SqlParser.ypp:1243.16-31: warning: unused value: $2 [-Wother]
       TOKEN_SELECT opt_all_distinct selection from_clause opt_where_clause opt_group_by_clause opt_having_clause
                    ^^^^^^^^^^^^^^^^
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/incubator-quickstep/pull/229


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #229: Implement parser and resolver for UNI...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/229#discussion_r110300461
  
    --- Diff: query_optimizer/LogicalGenerator.cpp ---
    @@ -51,12 +51,12 @@ L::LogicalPtr LogicalGenerator::generatePlan(
         const CatalogDatabase &catalog_database,
         const ParseStatement &parse_statement) {
       resolver::Resolver resolver(catalog_database, optimizer_context_);
    -  DVLOG(4) << "Parse tree:\n" << parse_statement.toString();
    +  std::cout << "Parse tree:\n" << parse_statement.toString();
       logical_plan_ = resolver.resolve(parse_statement);
    -  DVLOG(4) << "Initial logical plan:\n" << logical_plan_->toString();
    +  std::cout << "Initial logical plan:\n" << logical_plan_->toString();
     
       optimizePlan();
    -  DVLOG(4) << "Optimized logical plan:\n" << logical_plan_->toString();
    +  std::cout << "Optimized logical plan:\n" << logical_plan_->toString();
    --- End diff --
    
    Please reset all changes in this file.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---