You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@age.apache.org by de...@apache.org on 2022/04/01 18:12:45 UTC

[incubator-age] branch master updated: Fix chained unions returning unexpected results

This is an automated email from the ASF dual-hosted git repository.

dehowef pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-age.git


The following commit(s) were added to refs/heads/master by this push:
     new 72eb13a  Fix chained unions returning unexpected results
72eb13a is described below

commit 72eb13a2a176551e132757ef51eac44c7564350d
Author: Dehowe Feng <de...@gmail.com>
AuthorDate: Thu Mar 31 14:16:26 2022 -0700

    Fix chained unions returning unexpected results
    
    Rewrote the union logic to integrate union set operation values
    into the cypher_return node. Removed the cypher_union ag node to be
    more in line with the postgresql implementation.
    
    Altered the grammar to utilize the make_set_op function for future
    set operations if necessary. Also changed query_list rule to
    cypher_stmt for more clarity.
    
    Refactored logic surrounding isLeaf in transform_cypher_union_tree
    to have the function call tree be more congruent with the postgres
    function tree.
    
    Fixed a bug where chained union operations resulted in unexpected
    results.
    
    Also fixed minor grammatical errors in comments and style issues.
---
 regress/expected/cypher_union.out   | 73 +++++++++++++++-------------
 regress/sql/cypher_union.sql        | 44 +++++++++--------
 src/backend/nodes/ag_nodes.c        |  2 -
 src/backend/nodes/cypher_outfuncs.c | 15 ++----
 src/backend/parser/cypher_clause.c  | 95 ++++++++++++++++++++++++++++---------
 src/backend/parser/cypher_gram.y    | 43 ++++++++++-------
 src/include/nodes/ag_nodes.h        |  1 -
 src/include/nodes/cypher_nodes.h    | 14 ++----
 src/include/nodes/cypher_outfuncs.h |  2 -
 9 files changed, 172 insertions(+), 117 deletions(-)

diff --git a/regress/expected/cypher_union.out b/regress/expected/cypher_union.out
index 1931564..063354d 100644
--- a/regress/expected/cypher_union.out
+++ b/regress/expected/cypher_union.out
@@ -25,39 +25,39 @@ NOTICE:  graph "cypher_union" has been created
  
 (1 row)
 
-SELECT * FROM cypher('cypher_union', $$ CREATE () $$) as (a agtype);
+SELECT * FROM cypher('cypher_union', $$CREATE ()$$) as (a agtype);
  a 
 ---
 (0 rows)
 
-SELECT * FROM cypher('cypher_union', $$ MATCH (n) RETURN n UNION MATCH (n) RETURN n $$) as (a agtype);
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION MATCH (n) RETURN n$$) as (a agtype);
                                a                                
 ----------------------------------------------------------------
  {"id": 281474976710657, "label": "", "properties": {}}::vertex
 (1 row)
 
-SELECT * FROM cypher('cypher_union', $$ MATCH (n) RETURN n UNION ALL MATCH (n) RETURN n $$) as (a agtype);
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) RETURN n$$) as (a agtype);
                                a                                
 ----------------------------------------------------------------
  {"id": 281474976710657, "label": "", "properties": {}}::vertex
  {"id": 281474976710657, "label": "", "properties": {}}::vertex
 (2 rows)
 
-SELECT * FROM cypher('cypher_union', $$ MATCH (n) RETURN n UNION RETURN 1 $$) as (a agtype);
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION RETURN 1$$) as (a agtype);
                                a                                
 ----------------------------------------------------------------
  {"id": 281474976710657, "label": "", "properties": {}}::vertex
  1
 (2 rows)
 
-SELECT * FROM cypher('cypher_union', $$ MATCH (n) RETURN n UNION RETURN NULL $$) as (a agtype);
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION RETURN NULL$$) as (a agtype);
                                a                                
 ----------------------------------------------------------------
  {"id": 281474976710657, "label": "", "properties": {}}::vertex
  
 (2 rows)
 
-SELECT * FROM cypher('cypher_union', $$ RETURN [1,2,3] UNION RETURN 1 $$) as (a agtype);
+SELECT * FROM cypher('cypher_union', $$RETURN [1,2,3] UNION RETURN 1$$) as (a agtype);
      a     
 -----------
  [1, 2, 3]
@@ -65,14 +65,14 @@ SELECT * FROM cypher('cypher_union', $$ RETURN [1,2,3] UNION RETURN 1 $$) as (a
 (2 rows)
 
 /*should return 1 row*/
-SELECT * FROM cypher('cypher_union', $$return NULL UNION return NULL $$) AS (result agtype);
+SELECT * FROM cypher('cypher_union', $$RETURN NULL UNION RETURN NULL$$) AS (result agtype);
  result 
 --------
  
 (1 row)
 
 /*should return 2 rows*/
-SELECT * FROM cypher('cypher_union', $$return NULL UNION ALL return NULL $$) AS (result agtype);
+SELECT * FROM cypher('cypher_union', $$RETURN NULL UNION ALL RETURN NULL$$) AS (result agtype);
  result 
 --------
  
@@ -82,14 +82,14 @@ SELECT * FROM cypher('cypher_union', $$return NULL UNION ALL return NULL $$) AS
 /*
  *multiple unions, precedence
  */
-SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION MATCH (n) RETURN n UNION MATCH (n) RETURN n $$) AS (result agtype);
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION MATCH (n) RETURN n UNION MATCH (n) RETURN n$$) AS (result agtype);
                              result                             
 ----------------------------------------------------------------
  {"id": 281474976710657, "label": "", "properties": {}}::vertex
 (1 row)
 
 /*should return triple*/
-SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) return n UNION ALL MATCH(n) RETURN n$$) AS (result agtype);
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) RETURN n UNION ALL MATCH(n) RETURN n$$) AS (result agtype);
                              result                             
 ----------------------------------------------------------------
  {"id": 281474976710657, "label": "", "properties": {}}::vertex
@@ -97,22 +97,22 @@ SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) re
  {"id": 281474976710657, "label": "", "properties": {}}::vertex
 (3 rows)
 
-/*should return double*/
-SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) return n UNION MATCH(n) RETURN n$$) AS (result agtype);
+/*should return single*/
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) RETURN n UNION MATCH(n) RETURN n$$) AS (result agtype);
                              result                             
 ----------------------------------------------------------------
  {"id": 281474976710657, "label": "", "properties": {}}::vertex
- {"id": 281474976710657, "label": "", "properties": {}}::vertex
-(2 rows)
+(1 row)
 
-/*should return just UNION*/
-SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION MATCH (n) return n UNION ALL MATCH(n) RETURN n$$) AS (result agtype);
+/*should return just a pair*/
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION MATCH (n) RETURN n UNION ALL MATCH(n) RETURN n$$) AS (result agtype);
                              result                             
 ----------------------------------------------------------------
  {"id": 281474976710657, "label": "", "properties": {}}::vertex
-(1 row)
+ {"id": 281474976710657, "label": "", "properties": {}}::vertex
+(2 rows)
 
-/*should return 3 rows*/
+/*should return 3 null rows*/
 SELECT * FROM cypher('cypher_union', $$RETURN NULL UNION ALL RETURN NULL UNION ALL RETURN NULL$$) AS (result agtype);
  result 
 --------
@@ -121,21 +121,21 @@ SELECT * FROM cypher('cypher_union', $$RETURN NULL UNION ALL RETURN NULL UNION A
  
 (3 rows)
 
-/*should return 2 rows*/
-SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) return n UNION MATCH(n) RETURN n$$) AS (result agtype);
-                             result                             
-----------------------------------------------------------------
- {"id": 281474976710657, "label": "", "properties": {}}::vertex
- {"id": 281474976710657, "label": "", "properties": {}}::vertex
-(2 rows)
-
-/*should return 1 row*/
-SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION MATCH (n) return n UNION ALL MATCH(n) RETURN n$$) AS (result agtype);
-                             result                             
-----------------------------------------------------------------
- {"id": 281474976710657, "label": "", "properties": {}}::vertex
+/*should return 1 null row*/
+SELECT * FROM cypher('cypher_union', $$RETURN NULL UNION ALL RETURN NULL UNION RETURN NULL$$) AS (result agtype);
+ result 
+--------
+ 
 (1 row)
 
+/*should return 2 null rows*/
+SELECT * FROM cypher('cypher_union', $$RETURN NULL UNION RETURN NULL UNION ALL RETURN NULL$$) AS (result agtype);
+ result 
+--------
+ 
+ 
+(2 rows)
+
 /* scoping */
 SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (m) RETURN n$$) AS (result agtype);
 ERROR:  could not find rte for n
@@ -144,14 +144,21 @@ LINE 2: ..., $$MATCH (n) RETURN n UNION ALL MATCH (m) RETURN n$$) AS (r...
 /*
  *UNION and UNION ALL, type casting
  */
-SELECT * FROM cypher('cypher_union', $$ RETURN 1.0::int UNION return 1::float UNION ALL RETURN 2.0::float $$) AS (result agtype);
+SELECT * FROM cypher('cypher_union', $$RETURN 1.0::int UNION RETURN 1::float UNION ALL RETURN 2.0::float$$) AS (result agtype);
  result 
 --------
  1
  2.0
 (2 rows)
 
-SELECT * FROM cypher('cypher_union', $$ RETURN 1.0::float UNION return 1::int UNION RETURN 1::float $$) AS (result agtype);
+SELECT * FROM cypher('cypher_union', $$RETURN 1.0::int UNION RETURN 1.0::float UNION ALL RETURN 1::int$$) AS (result agtype);
+ result 
+--------
+ 1
+ 1
+(2 rows)
+
+SELECT * FROM cypher('cypher_union', $$RETURN 1.0::float UNION RETURN 1::int UNION RETURN 1::float$$) AS (result agtype);
  result 
 --------
  1.0
diff --git a/regress/sql/cypher_union.sql b/regress/sql/cypher_union.sql
index 3338ebb..3b278ae 100644
--- a/regress/sql/cypher_union.sql
+++ b/regress/sql/cypher_union.sql
@@ -22,47 +22,47 @@ SET search_path TO ag_catalog;
 
 SELECT create_graph('cypher_union');
 
-SELECT * FROM cypher('cypher_union', $$ CREATE () $$) as (a agtype);
+SELECT * FROM cypher('cypher_union', $$CREATE ()$$) as (a agtype);
 
 
-SELECT * FROM cypher('cypher_union', $$ MATCH (n) RETURN n UNION MATCH (n) RETURN n $$) as (a agtype);
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION MATCH (n) RETURN n$$) as (a agtype);
 
-SELECT * FROM cypher('cypher_union', $$ MATCH (n) RETURN n UNION ALL MATCH (n) RETURN n $$) as (a agtype);
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) RETURN n$$) as (a agtype);
 
-SELECT * FROM cypher('cypher_union', $$ MATCH (n) RETURN n UNION RETURN 1 $$) as (a agtype);
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION RETURN 1$$) as (a agtype);
 
-SELECT * FROM cypher('cypher_union', $$ MATCH (n) RETURN n UNION RETURN NULL $$) as (a agtype);
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION RETURN NULL$$) as (a agtype);
 
-SELECT * FROM cypher('cypher_union', $$ RETURN [1,2,3] UNION RETURN 1 $$) as (a agtype);
+SELECT * FROM cypher('cypher_union', $$RETURN [1,2,3] UNION RETURN 1$$) as (a agtype);
 
 /*should return 1 row*/
-SELECT * FROM cypher('cypher_union', $$return NULL UNION return NULL $$) AS (result agtype);
+SELECT * FROM cypher('cypher_union', $$RETURN NULL UNION RETURN NULL$$) AS (result agtype);
 
 /*should return 2 rows*/
-SELECT * FROM cypher('cypher_union', $$return NULL UNION ALL return NULL $$) AS (result agtype);
+SELECT * FROM cypher('cypher_union', $$RETURN NULL UNION ALL RETURN NULL$$) AS (result agtype);
 
 /*
  *multiple unions, precedence
  */
-SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION MATCH (n) RETURN n UNION MATCH (n) RETURN n $$) AS (result agtype);
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION MATCH (n) RETURN n UNION MATCH (n) RETURN n$$) AS (result agtype);
 
 /*should return triple*/
-SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) return n UNION ALL MATCH(n) RETURN n$$) AS (result agtype);
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) RETURN n UNION ALL MATCH(n) RETURN n$$) AS (result agtype);
 
-/*should return double*/
-SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) return n UNION MATCH(n) RETURN n$$) AS (result agtype);
+/*should return single*/
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) RETURN n UNION MATCH(n) RETURN n$$) AS (result agtype);
 
-/*should return just UNION*/
-SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION MATCH (n) return n UNION ALL MATCH(n) RETURN n$$) AS (result agtype);
+/*should return just a pair*/
+SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION MATCH (n) RETURN n UNION ALL MATCH(n) RETURN n$$) AS (result agtype);
 
-/*should return 3 rows*/
+/*should return 3 null rows*/
 SELECT * FROM cypher('cypher_union', $$RETURN NULL UNION ALL RETURN NULL UNION ALL RETURN NULL$$) AS (result agtype);
 
-/*should return 2 rows*/
-SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (n) return n UNION MATCH(n) RETURN n$$) AS (result agtype);
+/*should return 1 null row*/
+SELECT * FROM cypher('cypher_union', $$RETURN NULL UNION ALL RETURN NULL UNION RETURN NULL$$) AS (result agtype);
 
-/*should return 1 row*/
-SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION MATCH (n) return n UNION ALL MATCH(n) RETURN n$$) AS (result agtype);
+/*should return 2 null rows*/
+SELECT * FROM cypher('cypher_union', $$RETURN NULL UNION RETURN NULL UNION ALL RETURN NULL$$) AS (result agtype);
 
 /* scoping */
 SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (m) RETURN n$$) AS (result agtype);
@@ -70,9 +70,11 @@ SELECT * FROM cypher('cypher_union', $$MATCH (n) RETURN n UNION ALL MATCH (m) RE
 /*
  *UNION and UNION ALL, type casting
  */
-SELECT * FROM cypher('cypher_union', $$ RETURN 1.0::int UNION return 1::float UNION ALL RETURN 2.0::float $$) AS (result agtype);
+SELECT * FROM cypher('cypher_union', $$RETURN 1.0::int UNION RETURN 1::float UNION ALL RETURN 2.0::float$$) AS (result agtype);
+
+SELECT * FROM cypher('cypher_union', $$RETURN 1.0::int UNION RETURN 1.0::float UNION ALL RETURN 1::int$$) AS (result agtype);
 
-SELECT * FROM cypher('cypher_union', $$ RETURN 1.0::float UNION return 1::int UNION RETURN 1::float $$) AS (result agtype);
+SELECT * FROM cypher('cypher_union', $$RETURN 1.0::float UNION RETURN 1::int UNION RETURN 1::float$$) AS (result agtype);
 
 
 SELECT drop_graph('cypher_union', true);
diff --git a/src/backend/nodes/ag_nodes.c b/src/backend/nodes/ag_nodes.c
index 95d2b2d..d65bb50 100644
--- a/src/backend/nodes/ag_nodes.c
+++ b/src/backend/nodes/ag_nodes.c
@@ -39,7 +39,6 @@ const char *node_names[] = {
     "cypher_set",
     "cypher_set_item",
     "cypher_delete",
-    "cypher_union",
     "cypher_unwind",
     "cypher_merge",
     "cypher_path",
@@ -102,7 +101,6 @@ const ExtensibleNodeMethods node_methods[] = {
     DEFINE_NODE_METHODS(cypher_set),
     DEFINE_NODE_METHODS(cypher_set_item),
     DEFINE_NODE_METHODS(cypher_delete),
-    DEFINE_NODE_METHODS(cypher_union),
     DEFINE_NODE_METHODS(cypher_unwind),
     DEFINE_NODE_METHODS(cypher_merge),
     DEFINE_NODE_METHODS(cypher_path),
diff --git a/src/backend/nodes/cypher_outfuncs.c b/src/backend/nodes/cypher_outfuncs.c
index b97a247..05c0f7f 100644
--- a/src/backend/nodes/cypher_outfuncs.c
+++ b/src/backend/nodes/cypher_outfuncs.c
@@ -107,6 +107,11 @@ void out_cypher_return(StringInfo str, const ExtensibleNode *node)
     WRITE_NODE_FIELD(order_by);
     WRITE_NODE_FIELD(skip);
     WRITE_NODE_FIELD(limit);
+
+    WRITE_BOOL_FIELD(all_or_distinct);
+    WRITE_ENUM_FIELD(op, SetOperation);
+    WRITE_NODE_FIELD(larg);
+    WRITE_NODE_FIELD(rarg);
 }
 
 // serialization function for the cypher_with ExtensibleNode.
@@ -378,16 +383,6 @@ void out_cypher_delete_item(StringInfo str, const ExtensibleNode *node)
     WRITE_STRING_FIELD(var_name);
 }
 
-void out_cypher_union(StringInfo str, const ExtensibleNode *node)
-{
-    DEFINE_AG_NODE(cypher_union);
-
-    WRITE_BOOL_FIELD(all_or_distinct);
-    WRITE_ENUM_FIELD(op, SetOperation);
-    WRITE_NODE_FIELD(larg);
-    WRITE_NODE_FIELD(rarg);
-}
-
 // serialization function for the cypher_merge_information ExtensibleNode.
 void out_cypher_merge_information(StringInfo str, const ExtensibleNode *node)
 {
diff --git a/src/backend/parser/cypher_clause.c b/src/backend/parser/cypher_clause.c
index fb6502d..45f28ef 100644
--- a/src/backend/parser/cypher_clause.c
+++ b/src/backend/parser/cypher_clause.c
@@ -394,7 +394,20 @@ Query *transform_cypher_clause(cypher_parsestate *cpstate,
     // examine the type of clause and call the transform logic for it
     if (is_ag_node(self, cypher_return))
     {
-        result = transform_cypher_return(cpstate, clause);
+        cypher_return *n = (cypher_return *) self;
+
+        if (n->op == SETOP_NONE)
+        {
+            result = transform_cypher_return(cpstate, clause);
+        }
+        else if (n->op == SETOP_UNION)
+        {
+            result = transform_cypher_union(cpstate, clause);
+        }
+        else
+        {
+            ereport(ERROR, (errmsg_internal("unexpected Node for cypher_return")));
+        }
     }
     else if (is_ag_node(self, cypher_with))
     {
@@ -424,10 +437,6 @@ Query *transform_cypher_clause(cypher_parsestate *cpstate,
     {
         result = transform_cypher_sub_pattern(cpstate, clause);
     }
-    else if (is_ag_node(self, cypher_union))
-    {
-        result = transform_cypher_union(cpstate, clause);
-    }
     else if (is_ag_node(self, cypher_unwind))
     {
         result = transform_cypher_unwind(cpstate, clause);
@@ -480,7 +489,7 @@ static cypher_clause *make_cypher_clause(List *stmt)
 /*
  * transform_cypher_union -
  *    transforms a union tree, derived from postgresql's
- *    transformSetOperationStmt.A lot of the general logic is similar,
+ *    transformSetOperationStmt. A lot of the general logic is similar,
  *    with adjustments made for AGE.
  *
  * A union tree is just a return, but with UNION structure to it.
@@ -500,8 +509,9 @@ static Query *transform_cypher_union(cypher_parsestate *cpstate,
     SetOperationStmt *cypher_union_statement;
     Node *skip = NULL; /* equivalent to postgres limitOffset */
     Node *limit = NULL; /* equivalent to postgres limitCount */
+    List *order_by = NIL;
     Node *node;
-
+    cypher_return *self = (cypher_return *)clause->self;
     ListCell *left_tlist, *lct, *lcm, *lcc;
     List *targetvars, *targetnames, *sv_namespace;
     int sv_rtable_length;
@@ -523,6 +533,14 @@ static Query *transform_cypher_union(cypher_parsestate *cpstate,
                         parser_errposition(&cpstate->pstate, 0)));
     }
 
+    order_by = self->order_by;
+    skip = self->skip;
+    limit = self->limit;
+
+    self->order_by = NIL;
+    self->skip = NULL;
+    self->limit = NULL;
+
     /*
      * Recursively transform the components of the tree.
      */
@@ -538,7 +556,7 @@ static Query *transform_cypher_union(cypher_parsestate *cpstate,
     node = cypher_union_statement->larg;
     while (node && IsA(node, SetOperationStmt))
     {
-        node = ((SetOperationStmt *) cypher_union_statement)->larg;
+        node = ((SetOperationStmt *) node)->larg;
     }
     Assert(node && IsA(node, RangeTblRef));
     leftmostRTI = ((RangeTblRef *) node)->rtindex;
@@ -619,6 +637,12 @@ static Query *transform_cypher_union(cypher_parsestate *cpstate,
 
     tllen = list_length(qry->targetList);
 
+    qry->sortClause = transformSortClause(pstate,
+                                          order_by,
+                                          &qry->targetList,
+                                          EXPR_KIND_ORDER_BY,
+                                          false /* allow SQL92 rules */ );
+
     /* restore namespace, remove jrte from rtable */
     pstate->p_namespace = sv_namespace;
     pstate->p_rtable = list_truncate(pstate->p_rtable, sv_rtable_length);
@@ -634,7 +658,7 @@ static Query *transform_cypher_union(cypher_parsestate *cpstate,
     }
 
     qry->limitOffset = transform_cypher_limit(cpstate, skip,
-                                              EXPR_KIND_OFFSET, "SKIP");
+                                              EXPR_KIND_OFFSET, "OFFSET");
     qry->limitCount = transform_cypher_limit(cpstate, limit,
                                               EXPR_KIND_LIMIT, "LIMIT");
 
@@ -678,24 +702,51 @@ transform_cypher_union_tree(cypher_parsestate *cpstate, cypher_clause *clause,
     bool isLeaf;
 
     ParseState *pstate = (ParseState *)cpstate;
+    cypher_return *cmp;
+
+    /* Guard against stack overflow due to overly complex set-expressions */
+    check_stack_depth();
 
     if (IsA(clause, List))
     {
-        isLeaf = true;
+        clause = make_cypher_clause((List *)clause);
     }
-    else if (is_ag_node(clause->self, cypher_union))
+
+    if (is_ag_node(clause->self, cypher_return))
     {
-        isLeaf = false;
+        cmp = (cypher_return *) clause->self;
     }
     else
     {
         ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
-                errmsg("Cypher union found a clause type it does not support"),
+                errmsg("Cypher found an unsupported node"),
                 parser_errposition(pstate, 0)));
     }
 
-    /* Guard against stack overflow due to overly complex set-expressions */
-    check_stack_depth();
+
+    if (cmp->op == SETOP_NONE)
+    {
+        Assert(cmp->larg == NULL && cmp->rarg == NULL);
+        isLeaf = true;
+    }
+    else if (cmp->op == SETOP_UNION)
+    {
+        Assert(cmp->larg != NULL && cmp->rarg != NULL);
+        if (cmp->order_by || cmp->limit || cmp->skip)
+        {
+            isLeaf = true;
+        }
+        else
+        {
+            isLeaf = false;
+        }
+    }
+    else
+    {
+        ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+                errmsg("Cypher found an unsupported SETOP"),
+                parser_errposition(pstate, 0)));
+    }
 
     if (isLeaf)
     {
@@ -705,7 +756,6 @@ transform_cypher_union_tree(cypher_parsestate *cpstate, cypher_clause *clause,
         RangeTblEntry *rte PG_USED_FOR_ASSERTS_ONLY;
         RangeTblRef *rtr;
         ListCell *tl;
-        cypher_clause *leaf_clause;
 
         /*
          * Transform SelectStmt into a Query.
@@ -727,10 +777,9 @@ transform_cypher_union_tree(cypher_parsestate *cpstate, cypher_clause *clause,
          * cypher_analyze doesn't do this because the cypher_union clause
          * is hiding it.
          */
-        leaf_clause = make_cypher_clause((List *)clause);
 
-        returnQuery = cypher_parse_sub_analyze_union( (cypher_clause *) leaf_clause, cpstate,
-                                               NULL, false, false);
+        returnQuery = cypher_parse_sub_analyze_union((cypher_clause *) clause, cpstate,
+                                                     NULL, false, false);
         /*
          * Check for bogus references to Vars on the current query level (but
          * upper-level references are okay). Normally this can't happen
@@ -791,7 +840,7 @@ transform_cypher_union_tree(cypher_parsestate *cpstate, cypher_clause *clause,
         List *rtargetlist;
         ListCell *ltl;
         ListCell *rtl;
-        cypher_union *self = (cypher_union *) clause->self;
+        cypher_return *self = (cypher_return *) clause->self;
         const char *context;
 
         context = "UNION";
@@ -802,7 +851,8 @@ transform_cypher_union_tree(cypher_parsestate *cpstate, cypher_clause *clause,
         /*
          * Recursively transform the left child node.
          */
-        op->larg = transform_cypher_union_tree(cpstate ,(cypher_clause *) self->larg,
+        op->larg = transform_cypher_union_tree(cpstate,
+                                               (cypher_clause *) self->larg,
                                                false,
                                                &ltargetlist);
 
@@ -823,7 +873,8 @@ transform_cypher_union_tree(cypher_parsestate *cpstate, cypher_clause *clause,
         /*
          * Recursively transform the right child node.
          */
-        op->rarg = transform_cypher_union_tree(cpstate, (cypher_clause *) self->rarg,
+        op->rarg = transform_cypher_union_tree(cpstate,
+                                               (cypher_clause *) self->rarg,
                                                false,
                                                &rtargetlist);
 
diff --git a/src/backend/parser/cypher_gram.y b/src/backend/parser/cypher_gram.y
index fd4c21e..c08f01d 100644
--- a/src/backend/parser/cypher_gram.y
+++ b/src/backend/parser/cypher_gram.y
@@ -98,7 +98,7 @@
 
 /* query */
 %type <node> stmt
-%type <list> single_query query_part_init query_part_last query_list
+%type <list> single_query query_part_init query_part_last cypher_stmt
              reading_clause_list updating_clause_list_0 updating_clause_list_1
 %type <node> reading_clause updating_clause
 
@@ -208,8 +208,10 @@ static Node *make_typecast_expr(Node *expr, char *typecast, int location);
 
 // functions
 static Node *make_function_expr(List *func_name, List *exprs, int location);
-%}
 
+// setops
+static Node *make_set_op(SetOperation op, bool all_or_distinct, List *larg, List *rarg);
+%}
 %%
 
 /*
@@ -217,7 +219,7 @@ static Node *make_function_expr(List *func_name, List *exprs, int location);
  */
 
 stmt:
-    query_list semicolon_opt
+    cypher_stmt semicolon_opt
         {
             /*
              * If there is no transition for the lookahead token and the
@@ -238,7 +240,7 @@ stmt:
             extra->result = $1;
             extra->extra = NULL;
         }
-    | EXPLAIN query_list semicolon_opt
+    | EXPLAIN cypher_stmt semicolon_opt
         {
             ExplainStmt *estmt = NULL;
 
@@ -252,7 +254,7 @@ stmt:
             estmt->options = NIL;
             extra->extra = (Node *)estmt;
         }
-    | EXPLAIN VERBOSE query_list semicolon_opt
+    | EXPLAIN VERBOSE cypher_stmt semicolon_opt
         {
             ExplainStmt *estmt = NULL;
 
@@ -266,7 +268,7 @@ stmt:
             estmt->options = list_make1(makeDefElem("verbose", NULL, @2));;
             extra->extra = (Node *)estmt;
         }
-    | EXPLAIN ANALYZE query_list semicolon_opt
+    | EXPLAIN ANALYZE cypher_stmt semicolon_opt
         {
             ExplainStmt *estmt = NULL;
 
@@ -280,7 +282,7 @@ stmt:
             estmt->options = list_make1(makeDefElem("analyze", NULL, @2));;
             extra->extra = (Node *)estmt;
         }
-    | EXPLAIN ANALYZE VERBOSE query_list semicolon_opt
+    | EXPLAIN ANALYZE VERBOSE cypher_stmt semicolon_opt
         {
             ExplainStmt *estmt = NULL;
 
@@ -297,21 +299,14 @@ stmt:
         }
     ;
 
-query_list:
+cypher_stmt:
     single_query
         {
             $$ = $1;
         }
-    | single_query UNION all_or_distinct query_list
+    | cypher_stmt UNION all_or_distinct cypher_stmt
         {
-            cypher_union *u = make_ag_node(cypher_union);
-
-            u->all_or_distinct = $3;
-            u->op = SETOP_UNION;
-            u->larg = $1;
-            u->rarg = $4;
-
-            $$ = list_make1((Node *) u);
+            $$ = list_make1(make_set_op(SETOP_UNION, $3, $1, $4));
         }
     ;
 
@@ -2200,3 +2195,17 @@ static unsigned long get_a_unique_number(void)
 
     return unique_counter++;
 }
+
+/*set operation function node to make a set op node*/
+
+static Node *
+make_set_op(SetOperation op, bool all_or_distinct, List *larg, List *rarg)
+{
+    cypher_return *n = make_ag_node(cypher_return);
+
+    n->op = op;
+    n->all_or_distinct = all_or_distinct;
+    n->larg = (List *) larg;
+    n->rarg = (List *) rarg;
+    return (Node *) n;
+}
diff --git a/src/include/nodes/ag_nodes.h b/src/include/nodes/ag_nodes.h
index 8585b33..15ff409 100644
--- a/src/include/nodes/ag_nodes.h
+++ b/src/include/nodes/ag_nodes.h
@@ -40,7 +40,6 @@ typedef enum ag_node_tag
     cypher_set_t,
     cypher_set_item_t,
     cypher_delete_t,
-    cypher_union_t,
     cypher_unwind_t,
     cypher_merge_t,
     // pattern
diff --git a/src/include/nodes/cypher_nodes.h b/src/include/nodes/cypher_nodes.h
index e41574c..fe9f6bd 100644
--- a/src/include/nodes/cypher_nodes.h
+++ b/src/include/nodes/cypher_nodes.h
@@ -55,6 +55,11 @@ typedef struct cypher_return
     List *order_by;
     Node *skip;
     Node *limit;
+
+    bool all_or_distinct;
+    SetOperation op;
+    List *larg; /* lefthand argument of the unions */
+    List *rarg; /*righthand argument of the unions */
 } cypher_return;
 
 typedef struct cypher_with
@@ -107,15 +112,6 @@ typedef struct cypher_delete
     int location;
 } cypher_delete;
 
-typedef struct cypher_union
-{
-    ExtensibleNode extensible;
-    bool all_or_distinct;
-    SetOperation op;
-    List *larg; /* lefthand argument of the unions */
-    List *rarg; /*righthand argument of the unions */
-} cypher_union;
-
 typedef struct cypher_unwind
 {
     ExtensibleNode extensible;
diff --git a/src/include/nodes/cypher_outfuncs.h b/src/include/nodes/cypher_outfuncs.h
index b1129f9..20ef73f 100644
--- a/src/include/nodes/cypher_outfuncs.h
+++ b/src/include/nodes/cypher_outfuncs.h
@@ -39,8 +39,6 @@ void out_cypher_create(StringInfo str, const ExtensibleNode *node);
 void out_cypher_set(StringInfo str, const ExtensibleNode *node);
 void out_cypher_set_item(StringInfo str, const ExtensibleNode *node);
 void out_cypher_delete(StringInfo str, const ExtensibleNode *node);
-void out_cypher_union(StringInfo str, const ExtensibleNode *node);
-void out_cypher_union_stmt(StringInfo str, const ExtensibleNode *node);
 void out_cypher_unwind(StringInfo str, const ExtensibleNode *node);
 void out_cypher_merge(StringInfo str, const ExtensibleNode *node);