You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2020/07/31 17:24:00 UTC

[impala] branch master updated (b29cb4c -> ea3f073)

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

tarmstrong pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git.


    from b29cb4c  IMPALA-10006: handle non-writable /opt/impala/logs
     new 033a460  IMPALA-9959: Implement ds_kll_sketch() and ds_kll_quantile() functions
     new ea3f073  IMPALA-9943,IMPALA-4974: INTERSECT/EXCEPT [DISTINCT]

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/exprs/aggregate-functions-ir.cc             |  95 ++-
 be/src/exprs/aggregate-functions.h                 |   7 +
 be/src/exprs/datasketches-common.cc                |  18 +-
 be/src/exprs/datasketches-common.h                 |  16 +-
 be/src/exprs/datasketches-functions-ir.cc          |  28 +-
 be/src/exprs/datasketches-functions.h              |  16 +-
 common/function-registry/impala_functions.py       |   5 +-
 fe/src/main/cup/sql-parser.cup                     | 135 ++--
 .../apache/impala/analysis/AnalysisContext.java    |  28 +-
 .../java/org/apache/impala/analysis/Analyzer.java  |  10 +-
 .../org/apache/impala/analysis/InsertStmt.java     |   4 +-
 .../java/org/apache/impala/analysis/QueryStmt.java |   2 +-
 .../{UnionStmt.java => SetOperationStmt.java}      | 704 ++++++++++++-------
 .../org/apache/impala/analysis/StmtRewriter.java   | 271 +++++++-
 .../java/org/apache/impala/analysis/UnionStmt.java | 608 +----------------
 .../org/apache/impala/analysis/ValuesStmt.java     |   5 +-
 .../java/org/apache/impala/catalog/BuiltinsDb.java |  10 +
 .../org/apache/impala/planner/PlanFragment.java    |   2 +-
 .../apache/impala/planner/SingleNodePlanner.java   |  29 +-
 .../java/org/apache/impala/planner/UnionNode.java  |   2 +-
 fe/src/main/jflex/sql-scanner.flex                 |   3 +
 .../apache/impala/analysis/AnalyzeStmtsTest.java   | 193 +++++-
 .../org/apache/impala/analysis/ParserTest.java     | 275 +++++---
 .../org/apache/impala/planner/PlannerTest.java     |   6 +
 testdata/data/README                               |  10 +
 testdata/data/kll_sketches_from_hive.parquet       | Bin 0 -> 2501 bytes
 .../queries/PlannerTest/empty.test                 |  70 ++
 .../queries/PlannerTest/setoperation-rewrite.test  | 758 +++++++++++++++++++++
 .../queries/PlannerTest/tpcds-all.test             | 546 +++++++++++++++
 .../queries/QueryTest/datasketches-kll.test        | 146 ++++
 .../functional-query/queries/QueryTest/except.test | 628 +++++++++++++++++
 .../queries/QueryTest/intersect.test               | 536 +++++++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q14-1.test      | 207 ++++++
 .../tpcds/queries/tpcds-decimal_v2-q14-2.test      | 231 +++++++
 testdata/workloads/tpcds/queries/tpcds-q14-1.test  | 207 ++++++
 testdata/workloads/tpcds/queries/tpcds-q14-2.test  | 231 +++++++
 testdata/workloads/tpcds/queries/tpcds-q38.test    |  35 +
 testdata/workloads/tpcds/queries/tpcds-q87.test    |  37 +
 tests/query_test/test_datasketches.py              |   4 +
 tests/query_test/test_queries.py                   |   6 +
 tests/query_test/test_tpcds_queries.py             |  18 +
 tests/util/parse_util.py                           |   2 +-
 42 files changed, 5059 insertions(+), 1085 deletions(-)
 copy fe/src/main/java/org/apache/impala/analysis/{UnionStmt.java => SetOperationStmt.java} (54%)
 create mode 100644 testdata/data/kll_sketches_from_hive.parquet
 create mode 100644 testdata/workloads/functional-planner/queries/PlannerTest/setoperation-rewrite.test
 create mode 100644 testdata/workloads/functional-query/queries/QueryTest/datasketches-kll.test
 create mode 100644 testdata/workloads/functional-query/queries/QueryTest/except.test
 create mode 100644 testdata/workloads/functional-query/queries/QueryTest/intersect.test
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-decimal_v2-q14-1.test
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-decimal_v2-q14-2.test
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-q14-1.test
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-q14-2.test
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-q38.test
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-q87.test


[impala] 02/02: IMPALA-9943, IMPALA-4974: INTERSECT/EXCEPT [DISTINCT]

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tarmstrong pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit ea3f073881783baed17cea2d8bb718038cdfba8a
Author: Shant Hovsepian <sh...@cloudera.com>
AuthorDate: Mon Jun 8 19:31:12 2020 -0400

    IMPALA-9943,IMPALA-4974: INTERSECT/EXCEPT [DISTINCT]
    
    INTERSECT and EXCEPT set operations are implemented as rewrites to
    joins. Currently only the DISTINCT qualified operators are implemented,
    not ALL qualified. The operator MINUS is supported as an alias for
    EXCEPT.
    
    We mimic Oracle and Hive's non-standard implementation which treats all
    operators with the same precedence, as opposed to the SQL Standard of
    giving INTERSECT higher precedence.
    
    A new class SetOperationStmt was created to encompass the previous
    UnionStmt behavior. UnionStmt is preserved as a special case of union
    only operands to ensure compatibility with previous union planning
    behavior.
    
    Tests:
    * Added parser and analyzer tests.
    * Ensured no test failures or plan changes for union tests.
    * Added TPC-DS queries 14,38,87 to functional and planner tests.
    * Added functional tests test_intersect test_except
    * New planner testSetOperationStmt
    
    Change-Id: I5be46f824217218146ad48b30767af0fc7edbc0f
    Reviewed-on: http://gerrit.cloudera.org:8080/16123
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Aman Sinha <am...@cloudera.com>
    Reviewed-by: Tim Armstrong <ta...@cloudera.com>
---
 fe/src/main/cup/sql-parser.cup                     | 135 ++--
 .../apache/impala/analysis/AnalysisContext.java    |  28 +-
 .../java/org/apache/impala/analysis/Analyzer.java  |  10 +-
 .../org/apache/impala/analysis/InsertStmt.java     |   4 +-
 .../java/org/apache/impala/analysis/QueryStmt.java |   2 +-
 .../{UnionStmt.java => SetOperationStmt.java}      | 704 ++++++++++++-------
 .../org/apache/impala/analysis/StmtRewriter.java   | 271 +++++++-
 .../java/org/apache/impala/analysis/UnionStmt.java | 608 +----------------
 .../org/apache/impala/analysis/ValuesStmt.java     |   5 +-
 .../org/apache/impala/planner/PlanFragment.java    |   2 +-
 .../apache/impala/planner/SingleNodePlanner.java   |  29 +-
 .../java/org/apache/impala/planner/UnionNode.java  |   2 +-
 fe/src/main/jflex/sql-scanner.flex                 |   3 +
 .../apache/impala/analysis/AnalyzeStmtsTest.java   | 193 +++++-
 .../org/apache/impala/analysis/ParserTest.java     | 275 +++++---
 .../org/apache/impala/planner/PlannerTest.java     |   6 +
 .../queries/PlannerTest/empty.test                 |  70 ++
 .../queries/PlannerTest/setoperation-rewrite.test  | 758 +++++++++++++++++++++
 .../queries/PlannerTest/tpcds-all.test             | 546 +++++++++++++++
 .../functional-query/queries/QueryTest/except.test | 628 +++++++++++++++++
 .../queries/QueryTest/intersect.test               | 536 +++++++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q14-1.test      | 207 ++++++
 .../tpcds/queries/tpcds-decimal_v2-q14-2.test      | 231 +++++++
 testdata/workloads/tpcds/queries/tpcds-q14-1.test  | 207 ++++++
 testdata/workloads/tpcds/queries/tpcds-q14-2.test  | 231 +++++++
 testdata/workloads/tpcds/queries/tpcds-q38.test    |  35 +
 testdata/workloads/tpcds/queries/tpcds-q87.test    |  37 +
 tests/query_test/test_queries.py                   |   6 +
 tests/query_test/test_tpcds_queries.py             |  18 +
 tests/util/parse_util.py                           |   2 +-
 30 files changed, 4726 insertions(+), 1063 deletions(-)

diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index bc124a4..08903f9 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -30,8 +30,9 @@ import java_cup.runtime.Symbol;
 
 import org.apache.impala.analysis.ColumnDef;
 import org.apache.impala.analysis.ColumnDef.Option;
-import org.apache.impala.analysis.UnionStmt.Qualifier;
-import org.apache.impala.analysis.UnionStmt.UnionOperand;
+import org.apache.impala.analysis.SetOperationStmt.Qualifier;
+import org.apache.impala.analysis.SetOperationStmt.SetOperand;
+import org.apache.impala.analysis.SetOperationStmt.SetOperator;
 import org.apache.impala.analysis.RangePartition;
 import org.apache.impala.analysis.TableSampleClause;
 import org.apache.impala.analysis.AlterTableAddDropRangePartitionStmt.Operation;
@@ -287,15 +288,15 @@ terminal
   KW_COMPUTE, KW_CONSTRAINT, KW_COPY, KW_CREATE, KW_CROSS, KW_CUBE, KW_CURRENT, KW_DATA,
   KW_DATABASE, KW_DATABASES, KW_DATE, KW_DATETIME, KW_DECIMAL, KW_DEFAULT, KW_DELETE,
   KW_DELIMITED, KW_DESC, KW_DESCRIBE, KW_DISABLE, KW_DISTINCT, KW_DIV, KW_DOUBLE,
-  KW_DROP, KW_ELSE, KW_ENABLE, KW_ENCODING, KW_END, KW_ESCAPED, KW_EXISTS, KW_EXPLAIN,
+  KW_DROP, KW_ELSE, KW_ENABLE, KW_ENCODING, KW_END, KW_ESCAPED, KW_EXCEPT, KW_EXISTS, KW_EXPLAIN,
   KW_EXTENDED, KW_EXTERNAL, KW_FALSE, KW_FIELDS, KW_FILEFORMAT, KW_FILES, KW_FINALIZE_FN,
   KW_FIRST, KW_FLOAT, KW_FOLLOWING, KW_FOR, KW_FOREIGN, KW_FORMAT, KW_FORMATTED,
   KW_FROM, KW_FULL, KW_FUNCTION, KW_FUNCTIONS, KW_GRANT, KW_GROUP, KW_GROUPING, KW_HASH,
   KW_HUDIPARQUET,
   KW_IGNORE, KW_HAVING, KW_ICEBERG, KW_IF, KW_ILIKE, KW_IN, KW_INCREMENTAL, KW_INIT_FN, KW_INNER,
-  KW_INPATH, KW_INSERT, KW_INT, KW_INTERMEDIATE, KW_INTERVAL, KW_INTO, KW_INVALIDATE, KW_IREGEXP,
+  KW_INPATH, KW_INSERT, KW_INT, KW_INTERMEDIATE, KW_INTERSECT, KW_INTERVAL, KW_INTO, KW_INVALIDATE, KW_IREGEXP,
   KW_IS, KW_JOIN,  KW_KUDU, KW_LAST, KW_LEFT, KW_LEXICAL, KW_LIKE, KW_LIMIT, KW_LINES,
-  KW_LOAD, KW_LOCATION, KW_LOGICAL_OR, KW_MAP, KW_MERGE_FN, KW_METADATA, KW_NORELY, KW_NOT,
+  KW_LOAD, KW_LOCATION, KW_LOGICAL_OR, KW_MAP, KW_MERGE_FN, KW_METADATA, KW_MINUS, KW_NORELY, KW_NOT,
   KW_NOVALIDATE, KW_NULL, KW_NULLS, KW_OFFSET, KW_ON, KW_OR, KW_ORC, KW_ORDER, KW_OUTER,
   KW_OVER, KW_OVERWRITE, KW_PARQUET, KW_PARQUETFILE, KW_PARTITION, KW_PARTITIONED,
   KW_PARTITIONS, KW_PRECEDING, KW_PREPARE_FN, KW_PRIMARY, KW_PRODUCED, KW_PURGE,
@@ -342,15 +343,15 @@ nonterminal StatementBase stmt;
 nonterminal SelectStmt select_stmt;
 // Single values statement.
 nonterminal ValuesStmt values_stmt;
-// Select or union statement.
+// Select or Set Operation statement.
 nonterminal QueryStmt query_stmt;
 nonterminal QueryStmt opt_query_stmt;
 // Single select_stmt or parenthesized query_stmt.
-nonterminal QueryStmt union_operand;
-// List of select or union blocks connected by UNION operators or a single select block.
-nonterminal List<UnionOperand> union_operand_list;
-// List of union operands consisting of constant selects.
-nonterminal List<UnionOperand> values_operand_list;
+nonterminal QueryStmt set_operand;
+// List of select or set operation blocks connected by set operators or a single select.
+nonterminal List<SetOperand> set_operand_list;
+// List of set operands consisting of constant selects.
+nonterminal List<SetOperand> values_operand_list;
 // USE stmt
 nonterminal UseStmt use_stmt;
 nonterminal SetStmt set_stmt;
@@ -369,8 +370,8 @@ nonterminal TDescribeOutputStyle describe_output_style;
 nonterminal LoadDataStmt load_stmt;
 nonterminal TruncateStmt truncate_stmt;
 nonterminal ResetMetadataStmt reset_metadata_stmt;
-// List of select blocks connected by UNION operators, with order by or limit.
-nonterminal QueryStmt union_with_order_by_or_limit;
+// List of select blocks connected by set operands, with order by or limit.
+nonterminal QueryStmt set_operation_with_order_by_or_limit;
 nonterminal SelectList select_clause;
 nonterminal SelectList select_list;
 nonterminal SelectListItem select_list_item;
@@ -446,7 +447,8 @@ nonterminal List<PartitionKeyValue> static_partition_key_value_list;
 nonterminal List<PartitionKeyValue> partition_key_value_list;
 nonterminal PartitionKeyValue partition_key_value;
 nonterminal PartitionKeyValue static_partition_key_value;
-nonterminal Qualifier union_op;
+nonterminal Qualifier opt_set_op_qualifier;
+nonterminal SetOperator set_op;
 
 // For ALTER DATABASE.
 nonterminal AlterDbStmt alter_db_stmt;
@@ -2482,21 +2484,21 @@ function_def_arg_key ::=
 // even if the union has order by and limit.
 // ORDER BY and LIMIT bind to the preceding select statement by default.
 query_stmt ::=
-  opt_with_clause:w union_operand_list:operands
+  opt_with_clause:w set_operand_list:operands
   {:
     QueryStmt queryStmt = null;
     if (operands.size() == 1) {
       queryStmt = operands.get(0).getQueryStmt();
     } else {
-      queryStmt = new UnionStmt(operands, null, null);
+      queryStmt = SetOperationStmt.createUnionOrSetOperation(operands, null, null);
     }
     queryStmt.setWithClause(w);
     RESULT = queryStmt;
   :}
-  | opt_with_clause:w union_with_order_by_or_limit:union
+  | opt_with_clause:w set_operation_with_order_by_or_limit:set_operation
   {:
-    union.setWithClause(w);
-    RESULT = union;
+    set_operation.setWithClause(w);
+    RESULT = set_operation;
   :}
   ;
 
@@ -2534,50 +2536,51 @@ with_view_def_list ::=
   :}
   ;
 
-// We must have a non-empty order by or limit for them to bind to the union.
-// We cannot reuse the existing opt_order_by_clause or
-// limit_clause because they would introduce conflicts with EOF,
-// which, unfortunately, cannot be accessed in the parser as a nonterminal
-// making this issue unresolvable.
-// We rely on the left precedence of KW_ORDER, KW_BY, and KW_LIMIT,
-// to resolve the ambiguity with select_stmt in favor of select_stmt
-// (i.e., ORDER BY and LIMIT bind to the select_stmt by default, and not the union).
-// There must be at least two union operands for ORDER BY or LIMIT to bind to a union,
-// and we manually throw a parse error if we reach this production
-// with only a single operand.
-union_with_order_by_or_limit ::=
-    union_operand_list:operands
+// IMPALA-4741: The behavior below is non-standard, order by should only bind to a nested
+// statement if it is enclosed in parentheses.
+// We must have a non-empty order by or limit for them to bind to the set op.  We cannot
+// reuse the existing opt_order_by_clause or limit_clause because they would introduce
+// conflicts with EOF, which, unfortunately, cannot be accessed in the parser as a
+// nonterminal making this issue unresolvable.  We rely on the left precedence of
+// KW_ORDER, KW_BY, and KW_LIMIT, to resolve the ambiguity with select_stmt in favor of
+// select_stmt (i.e., ORDER BY and LIMIT bind to the select_stmt by default, and not the
+// set op).  There must be at least two set operands for ORDER BY or LIMIT to bind, and we
+// manually throw a parse error if we reach this production with only a single operand.
+set_operation_with_order_by_or_limit ::=
+    set_operand_list:operands
     KW_ORDER KW_BY order_by_elements:orderByClause
     opt_offset_param:offsetExpr
   {:
     if (operands.size() == 1) {
       parser.parseError("order", SqlParserSymbols.KW_ORDER);
     }
-    RESULT = new UnionStmt(operands, orderByClause, new LimitElement(null, offsetExpr));
+    RESULT = SetOperationStmt.createUnionOrSetOperation(operands, orderByClause,
+        new LimitElement(null, offsetExpr));
   :}
   |
-    union_operand_list:operands
+    set_operand_list:operands
     KW_LIMIT expr:limitExpr
   {:
     if (operands.size() == 1) {
       parser.parseError("limit", SqlParserSymbols.KW_LIMIT);
     }
-    RESULT = new UnionStmt(operands, null, new LimitElement(limitExpr, null));
+    RESULT = SetOperationStmt.createUnionOrSetOperation(operands, null,
+        new LimitElement(limitExpr, null));
   :}
   |
-    union_operand_list:operands
+    set_operand_list:operands
     KW_ORDER KW_BY order_by_elements:orderByClause
     KW_LIMIT expr:limitExpr opt_offset_param:offsetExpr
   {:
     if (operands.size() == 1) {
       parser.parseError("order", SqlParserSymbols.KW_ORDER);
     }
-    RESULT = new UnionStmt(operands, orderByClause,
+    RESULT = SetOperationStmt.createUnionOrSetOperation(operands, orderByClause,
         new LimitElement(limitExpr, offsetExpr));
   :}
   ;
 
-union_operand ::=
+set_operand ::=
   select_stmt:select
   {: RESULT = select; :}
   | values_stmt:values
@@ -2586,26 +2589,44 @@ union_operand ::=
   {: RESULT = query; :}
   ;
 
-union_operand_list ::=
-  union_operand:operand
+set_operand_list ::=
+  set_operand:operand
   {:
-    List<UnionOperand> operands = new ArrayList<>();
-    operands.add(new UnionOperand(operand, null));
+    List<SetOperand> operands = new ArrayList<>();
+    operands.add(new SetOperand(operand, null, null));
     RESULT = operands;
   :}
-  | union_operand_list:operands union_op:op union_operand:operand
+  | set_operand_list:operands set_op:op set_operand:operand
   {:
-    operands.add(new UnionOperand(operand, op));
+    operands.add(new SetOperand(operand, op, Qualifier.DISTINCT));
+    RESULT = operands;
+  :}
+  | set_operand_list:operands set_op:op KW_DISTINCT set_operand:operand
+  {:
+    operands.add(new SetOperand(operand, op, Qualifier.DISTINCT));
+    RESULT = operands;
+  :}
+  | set_operand_list:operands KW_UNION opt_set_op_qualifier:qual set_operand:operand
+  {:
+    operands.add(new SetOperand(operand, SetOperator.UNION, qual));
     RESULT = operands;
   :}
   ;
 
-union_op ::=
-  KW_UNION
+set_op ::=
+  KW_EXCEPT
+  {: RESULT = SetOperator.EXCEPT; :}
+  | KW_INTERSECT
+  {: RESULT = SetOperator.INTERSECT; :}
+  | KW_MINUS
+  {: RESULT = SetOperator.EXCEPT; :}
+  ;
+
+opt_set_op_qualifier ::=
   {: RESULT = Qualifier.DISTINCT; :}
-  | KW_UNION KW_DISTINCT
+  | KW_DISTINCT
   {: RESULT = Qualifier.DISTINCT; :}
-  | KW_UNION KW_ALL
+  | KW_ALL
   {: RESULT = Qualifier.ALL; :}
   ;
 
@@ -2627,15 +2648,17 @@ values_stmt ::=
 values_operand_list ::=
   LPAREN select_list:selectList RPAREN
   {:
-    List<UnionOperand> operands = new ArrayList<>();
-    operands.add(new UnionOperand(
-        new SelectStmt(selectList, null, null, null, null, null, null), null));
+    List<SetOperand> operands = new ArrayList<>();
+    operands.add(new SetOperand(
+        new SelectStmt(selectList, null, null, null, null, null, null),
+                       SetOperator.UNION, null));
     RESULT = operands;
   :}
   | values_operand_list:operands COMMA LPAREN select_list:selectList RPAREN
   {:
-    operands.add(new UnionOperand(
-        new SelectStmt(selectList, null, null, null, null, null, null), Qualifier.ALL));
+    operands.add(new SetOperand(
+        new SelectStmt(selectList, null, null, null, null, null, null),
+                       SetOperator.UNION, Qualifier.ALL));
     RESULT = operands;
   :}
   ;
@@ -3878,6 +3901,8 @@ word ::=
   {: RESULT = r.toString(); :}
   | KW_ESCAPED:r
   {: RESULT = r.toString(); :}
+  | KW_EXCEPT:r
+  {: RESULT = r.toString(); :}
   | KW_EXISTS:r
   {: RESULT = r.toString(); :}
   | KW_EXPLAIN:r
@@ -3954,6 +3979,8 @@ word ::=
   {: RESULT = r.toString(); :}
   | KW_INTERMEDIATE:r
   {: RESULT = r.toString(); :}
+  | KW_INTERSECT:r
+  {: RESULT = r.toString(); :}
   | KW_INTERVAL:r
   {: RESULT = r.toString(); :}
   | KW_INTO:r
@@ -3992,6 +4019,8 @@ word ::=
   {: RESULT = r.toString(); :}
   | KW_METADATA:r
   {: RESULT = r.toString(); :}
+  | KW_MINUS:r
+  {: RESULT = r.toString(); :}
   | KW_NORELY:r
   {: RESULT = r.toString(); :}
   | KW_NOT:r
diff --git a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java b/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
index c2c09cc..71389b8 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
@@ -93,6 +93,7 @@ public class AnalysisContext {
     public boolean isAlterViewStmt() { return stmt_ instanceof AlterViewStmt; }
     public boolean isComputeStatsStmt() { return stmt_ instanceof ComputeStatsStmt; }
     public boolean isQueryStmt() { return stmt_ instanceof QueryStmt; }
+    public boolean isSetOperationStmt() { return stmt_ instanceof SetOperationStmt; }
     public boolean isInsertStmt() { return stmt_ instanceof InsertStmt; }
     public boolean isDropDbStmt() { return stmt_ instanceof DropDbStmt; }
     public boolean isDropTableOrViewStmt() {
@@ -375,13 +376,17 @@ public class AnalysisContext {
     public Analyzer getAnalyzer() { return analyzer_; }
     public Set<TAccessEvent> getAccessEvents() { return analyzer_.getAccessEvents(); }
     public boolean requiresSubqueryRewrite() {
-      return analyzer_.containsSubquery() && !(stmt_ instanceof CreateViewStmt)
-          && !(stmt_ instanceof AlterViewStmt) && !(stmt_ instanceof ShowCreateTableStmt);
+      return analyzer_.containsSubquery() && !isCreateViewStmt() && !isAlterViewStmt()
+          && !isShowCreateTableStmt();
     }
     public boolean requiresExprRewrite() {
       return isQueryStmt() || isInsertStmt() || isCreateTableAsSelectStmt()
           || isUpdateStmt() || isDeleteStmt();
     }
+    public boolean requiresSetOperationRewrite() {
+      return analyzer_.containsSetOperation() && !isCreateViewStmt() && !isAlterViewStmt()
+          && !isShowCreateTableStmt();
+    }
     public TLineageGraph getThriftLineageGraph() {
       return analyzer_.getThriftSerializedLineageGraph();
     }
@@ -470,7 +475,7 @@ public class AnalysisContext {
     analysisResult_.analyzer_.checkStmtExprLimit();
     boolean isExplain = analysisResult_.isExplainStmt();
 
-    // Apply expr and subquery rewrites.
+    // Apply expr, setop, and subquery rewrites.
     boolean reAnalyze = false;
     ExprRewriter rewriter = analysisResult_.analyzer_.getExprRewriter();
     if (analysisResult_.requiresExprRewrite()) {
@@ -481,7 +486,10 @@ public class AnalysisContext {
     if (analysisResult_.requiresSubqueryRewrite()) {
       new StmtRewriter.SubqueryRewriter().rewrite(analysisResult_);
       reAnalyze = true;
-      LOG.info("Re-analyze the rewritten query.");
+    }
+    if (analysisResult_.requiresSetOperationRewrite()) {
+      new StmtRewriter().rewrite(analysisResult_);
+      reAnalyze = true;
     }
     if (!reAnalyze) return;
 
@@ -503,11 +511,23 @@ public class AnalysisContext {
     ImmutableList<PrivilegeRequest> origPrivReqs =
         analysisResult_.analyzer_.getPrivilegeReqs();
 
+    // For SetOperationStmt we must replace the query statement with the rewritten version
+    // before re-analysis.
+    if (analysisResult_.requiresSetOperationRewrite()) {
+      if (analysisResult_.isSetOperationStmt()) {
+        if (((SetOperationStmt) analysisResult_.getStmt()).hasRewrittenStmt()) {
+          analysisResult_.stmt_ =
+            ((SetOperationStmt) analysisResult_.getStmt()).getRewrittenStmt();
+        }
+      }
+    }
+
     // Re-analyze the stmt with a new analyzer.
     analysisResult_.analyzer_ = createAnalyzer(stmtTableCache, authzCtx);
     // We restore the privileges collected in the first pass below. So, no point in
     // collecting them again.
     analysisResult_.analyzer_.setEnablePrivChecks(false);
+
     analysisResult_.stmt_.reset();
     try {
       analysisResult_.stmt_.analyze(analysisResult_.analyzer_);
diff --git a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
index 958e47b..929a551 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
@@ -216,6 +216,7 @@ public class Analyzer {
   public boolean hasPlanHints() { return globalState_.hasPlanHints; }
   public void setHasWithClause() { hasWithClause_ = true; }
   public boolean hasWithClause() { return hasWithClause_; }
+  public void setSetOpNeedsRewrite() { globalState_.setOperationNeedsRewrite = true; }
 
   /**
    * @param table Table whose properties need to be checked.
@@ -342,6 +343,10 @@ public class Analyzer {
     // True if at least one of the analyzers belongs to a subquery.
     public boolean containsSubquery = false;
 
+    // True if one of the analyzers belongs to a set operand of type EXCEPT or INTERSECT
+    // which needs to be rewritten using joins.
+    public boolean setOperationNeedsRewrite = false;
+
     // all registered conjuncts (map from expr id to conjunct). We use a LinkedHashMap to
     // preserve the order in which conjuncts are added.
     public final Map<ExprId, Expr> conjuncts = new LinkedHashMap<>();
@@ -492,6 +497,7 @@ public class Analyzer {
   private final GlobalState globalState_;
 
   public boolean containsSubquery() { return globalState_.containsSubquery; }
+  public boolean containsSetOperation() { return globalState_.setOperationNeedsRewrite; }
 
   // An analyzer stores analysis state for a single select block. A select block can be
   // a top level select statement, or an inline view select block.
@@ -900,7 +906,7 @@ public class Analyzer {
 
   /**
    * Return true if this analyzer has no ancestors. (i.e. false for the analyzer created
-   * for inline views/ union operands, etc.)
+   * for inline views/ set operands (except/intersect/union), etc.)
    */
   public boolean isRootAnalyzer() { return ancestors_.isEmpty(); }
 
@@ -2695,7 +2701,7 @@ public class Analyzer {
    * Returns null if an empty expression list or null is passed to it.
    * Throw an AnalysisException if the types are incompatible.
    */
-  public List<Expr> castToUnionCompatibleTypes(List<List<Expr>> exprLists)
+  public List<Expr> castToSetOpCompatibleTypes(List<List<Expr>> exprLists)
       throws AnalysisException {
     if (exprLists == null || exprLists.size() == 0) return null;
     if (exprLists.size() == 1) return exprLists.get(0);
diff --git a/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java b/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
index 6d1c037..8d24187 100644
--- a/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
@@ -703,8 +703,8 @@ public class InsertStmt extends StatementBase {
       kuduPartitionColumnNames = getKuduPartitionColumnNames((FeKuduTable) table_);
     }
 
-    UnionStmt unionStmt =
-        (queryStmt_ instanceof UnionStmt) ? (UnionStmt) queryStmt_ : null;
+    SetOperationStmt unionStmt =
+        (queryStmt_ instanceof SetOperationStmt) ? (SetOperationStmt) queryStmt_ : null;
     List<Expr> widestTypeExprList = null;
     if (unionStmt != null && unionStmt.getWidestExprs() != null
         && unionStmt.getWidestExprs().size() > 0) {
diff --git a/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java b/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java
index 0d1e6d7..a303794 100644
--- a/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java
@@ -38,7 +38,7 @@ import com.google.common.collect.Lists;
 /**
  * Abstract base class for any statement that returns results
  * via a list of result expressions, for example a
- * SelectStmt or UnionStmt. Also maintains a map of expression substitutions
+ * SelectStmt or SetOperationStmt. Also maintains a map of expression substitutions
  * for replacing expressions from ORDER BY or GROUP BY clauses with
  * their corresponding result expressions.
  * Used for sharing members/methods and some of the analysis code, in particular the
diff --git a/fe/src/main/java/org/apache/impala/analysis/UnionStmt.java b/fe/src/main/java/org/apache/impala/analysis/SetOperationStmt.java
similarity index 54%
copy from fe/src/main/java/org/apache/impala/analysis/UnionStmt.java
copy to fe/src/main/java/org/apache/impala/analysis/SetOperationStmt.java
index bf4c0bf..864a1a3 100644
--- a/fe/src/main/java/org/apache/impala/analysis/UnionStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SetOperationStmt.java
@@ -20,7 +20,6 @@ package org.apache.impala.analysis;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
-
 import org.apache.impala.catalog.ColumnStats;
 import org.apache.impala.catalog.FeView;
 import org.apache.impala.common.AnalysisException;
@@ -31,59 +30,74 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Preconditions;
 
 /**
- * Representation of a union with its list of operands, and optional order by and limit.
+ * Representation of a set operation with its list of operands, and optional order by and
+ * limit. A child class UnionStmt encapsulates instances that contain only Union operands.
+ * EXCEPT, INTERSECT, and any combination of the previous with UNION is rewritten in
+ * StmtRewriter to use joins in place of native operators.
+ *
+ * TODO: This class contains many methods and members that should be pushed down to
+ * UnionStmt.
+ *
  * A union materializes its results, and its resultExprs are SlotRefs into a new
- * materialized tuple.
- * During analysis, the operands are normalized (separated into a single sequence of
- * DISTINCT followed by a single sequence of ALL operands) and unnested to the extent
- * possible. This also creates the AggregationInfo for DISTINCT operands.
+ * materialized tuple. During analysis, the operands are normalized (separated into a
+ * single sequence of DISTINCT followed by a single sequence of ALL operands) and unnested
+ * to the extent possible. This also creates the AggregationInfo for UNION DISTINCT
+ * operands. EXCEPT and INTERSECT are unnested during analysis, then rewritten discarding
+ * the original instance of SetOperationStmt.
  *
- * Use of resultExprs vs. baseTblResultExprs:
- * We consistently use/cast the resultExprs of union operands because the final expr
- * substitution happens during planning. The only place where baseTblResultExprs are
- * used is in materializeRequiredSlots() because that is called before plan generation
- * and we need to mark the slots of resolved exprs as materialized.
+ * Use of resultExprs vs. baseTblResultExprs: We consistently use/cast the resultExprs of
+ * union operands because the final expr substitution happens during planning. The only
+ * place where baseTblResultExprs are used is in materializeRequiredSlots() because that
+ * is called before plan generation and we need to mark the slots of resolved exprs as
+ * materialized.
  */
-public class UnionStmt extends QueryStmt {
-  private final static Logger LOG = LoggerFactory.getLogger(UnionStmt.class);
+public class SetOperationStmt extends QueryStmt {
+  private final static Logger LOG = LoggerFactory.getLogger(SetOperationStmt.class);
 
-  public static enum Qualifier {
-    ALL,
-    DISTINCT
-  }
+  public static enum Qualifier { ALL, DISTINCT }
+
+  public static enum SetOperator { EXCEPT, INTERSECT, UNION }
 
   /**
-   * Represents an operand to a union. It consists of a query statement and its left
-   * all/distinct qualifier (null for the first operand).
+   * Represents an operand to a set operation. It consists of a query statement, the set
+   * operation to perform (except/intersect/union) and its left all/distinct qualifier
+   * (null for the first operand).
    */
-  public static class UnionOperand {
+  public static class SetOperand {
     // Effective qualifier. Should not be reset() to preserve changes made during
     // distinct propagation and unnesting that are needed after rewriting Subqueries.
     private Qualifier qualifier_;
 
+    private SetOperator operator_;
     /////////////////////////////////////////
     // BEGIN: Members that need to be reset()
 
-    private final QueryStmt queryStmt_;
+    private QueryStmt queryStmt_;
 
     // Analyzer used for this operand. Set in analyze().
     // We must preserve the conjuncts registered in the analyzer for partition pruning.
     private Analyzer analyzer_;
 
-    // Map from UnionStmt's result slots to our resultExprs. Used during plan generation.
+    // Map from SetOperationStmt's result slots to our resultExprs. Used during plan
+    // generation.
     private final ExprSubstitutionMap smap_;
 
     // END: Members that need to be reset()
     /////////////////////////////////////////
 
-    public UnionOperand(QueryStmt queryStmt, Qualifier qualifier) {
+    public SetOperand(QueryStmt queryStmt, SetOperator operator, Qualifier qualifier) {
       queryStmt_ = queryStmt;
+      operator_ = operator;
       qualifier_ = qualifier;
       smap_ = new ExprSubstitutionMap();
     }
 
     public void analyze(Analyzer parent) throws AnalysisException {
       if (isAnalyzed()) return;
+      // Used to trigger a rewrite in StmtRewriter
+      if (operator_ == SetOperator.INTERSECT || operator_ == SetOperator.EXCEPT) {
+        parent.setSetOpNeedsRewrite();
+      }
       analyzer_ = new Analyzer(parent);
       queryStmt_.analyze(analyzer_);
     }
@@ -91,8 +105,11 @@ public class UnionStmt extends QueryStmt {
     public boolean isAnalyzed() { return analyzer_ != null; }
     public QueryStmt getQueryStmt() { return queryStmt_; }
     public Qualifier getQualifier() { return qualifier_; }
+    public SetOperator getSetOperator() { return operator_; }
     // Used for propagating DISTINCT.
     public void setQualifier(Qualifier qualifier) { qualifier_ = qualifier; }
+    public void setSetOperator(SetOperator operator) { operator_ = operator; }
+    public void setQueryStmt(QueryStmt stmt) { queryStmt_ = stmt; }
     public Analyzer getAnalyzer() { return analyzer_; }
     public ExprSubstitutionMap getSmap() { return smap_; }
 
@@ -100,17 +117,18 @@ public class UnionStmt extends QueryStmt {
       if (queryStmt_ instanceof SelectStmt) {
         return ((SelectStmt) queryStmt_).hasAnalyticInfo();
       } else {
-        Preconditions.checkState(queryStmt_ instanceof UnionStmt);
-        return ((UnionStmt) queryStmt_).hasAnalyticExprs();
+        Preconditions.checkState(queryStmt_ instanceof SetOperationStmt);
+        return ((SetOperationStmt) queryStmt_).hasAnalyticExprs();
       }
     }
 
     /**
      * C'tor for cloning.
      */
-    private UnionOperand(UnionOperand other) {
+    private SetOperand(SetOperand other) {
       queryStmt_ = other.queryStmt_.clone();
       qualifier_ = other.qualifier_;
+      operator_ = other.operator_;
       analyzer_ = other.analyzer_;
       smap_ = other.smap_.clone();
     }
@@ -122,27 +140,34 @@ public class UnionStmt extends QueryStmt {
     }
 
     @Override
-    public UnionOperand clone() { return new UnionOperand(this); }
+    public SetOperand clone() {
+      return new SetOperand(this);
+    }
   }
 
   /////////////////////////////////////////
   // BEGIN: Members that need to be reset()
 
-  // before analysis, this contains the list of union operands derived verbatim
-  // from the query;
+  // before analysis, this contains the list of operands derived verbatim from the query;
   // after analysis, this contains all of distinctOperands followed by allOperands
-  protected final List<UnionOperand> operands_;
+  protected final List<SetOperand> operands_;
 
   // filled during analyze(); contains all operands that need to go through
   // distinct aggregation
-  protected final List<UnionOperand> distinctOperands_ = new ArrayList<>();
+  protected final List<SetOperand> unionDistinctOperands_ = new ArrayList<>();
 
   // filled during analyze(); contains all operands that can be aggregated with
   // a simple merge without duplicate elimination (also needs to merge the output
   // of the DISTINCT operands)
-  protected final List<UnionOperand> allOperands_ = new ArrayList<>();
+  protected final List<SetOperand> unionAllOperands_ = new ArrayList<>();
+
+  // filled during analyze(); contains all intersect distinct operands
+  protected final List<SetOperand> intersectDistinctOperands_ = new ArrayList<>();
 
-  protected MultiAggregateInfo distinctAggInfo_; // only set if we have DISTINCT ops
+  // filled during analyze(); contains all except distinct operands
+  protected final List<SetOperand> exceptDistinctOperands_ = new ArrayList<>();
+
+  protected MultiAggregateInfo distinctAggInfo_; // only set if we have UNION DISTINCT ops
 
   // Single tuple materialized by the union. Set in analyze().
   protected TupleId tupleId_;
@@ -155,19 +180,23 @@ public class UnionStmt extends QueryStmt {
 
   // List of output expressions produced by the union without the ORDER BY portion
   // (if any). Same as resultExprs_ if there is no ORDER BY.
-  private List<Expr> unionResultExprs_ = new ArrayList<>();
+  protected List<Expr> setOperationResultExprs_ = new ArrayList<>();
 
-  // List of expressions produced by analyzer.castToUnionCompatibleTypes().
+  // List of expressions produced by analyzer.castToSetOpCompatibleTypes().
   // Contains a list of exprs such that for every i-th expr in that list, it is the first
   // widest compatible expression encountered among all i-th exprs in every result expr
-  // list of the union operands.
+  // list of the operands.
   protected List<Expr> widestExprs_ = new ArrayList<>();
 
+  // Holds the SelectStmt as a result of rewriting EXCEPT and INTERSECT. For cases where
+  // all the operands are UNION this will remain null.
+  protected QueryStmt rewrittenStmt_ = null;
+
   // END: Members that need to be reset()
   /////////////////////////////////////////
 
-  public UnionStmt(List<UnionOperand> operands,
-      List<OrderByElement> orderByElements, LimitElement limitElement) {
+  public SetOperationStmt(List<SetOperand> operands, List<OrderByElement> orderByElements,
+      LimitElement limitElement) {
     super(orderByElements, limitElement);
     Preconditions.checkNotNull(operands);
     Preconditions.checkState(operands.size() > 0);
@@ -177,18 +206,25 @@ public class UnionStmt extends QueryStmt {
   /**
    * C'tor for cloning.
    */
-  protected UnionStmt(UnionStmt other) {
+  protected SetOperationStmt(SetOperationStmt other) {
     super(other.cloneOrderByElements(),
         (other.limitElement_ == null) ? null : other.limitElement_.clone());
     operands_ = new ArrayList<>();
     if (analyzer_ != null) {
-      for (UnionOperand o: other.distinctOperands_) distinctOperands_.add(o.clone());
-      for (UnionOperand o: other.allOperands_) allOperands_.add(o.clone());
-      operands_.addAll(distinctOperands_);
-      operands_.addAll(allOperands_);
-    } else {
-      for (UnionOperand operand: other.operands_) operands_.add(operand.clone());
+      for (SetOperand o : other.unionDistinctOperands_) {
+        unionDistinctOperands_.add(o.clone());
+      }
+      for (SetOperand o : other.unionAllOperands_) {
+        unionAllOperands_.add(o.clone());
+      }
+      for (SetOperand o : other.exceptDistinctOperands_) {
+        exceptDistinctOperands_.add(o.clone());
+      }
+      for (SetOperand o : other.intersectDistinctOperands_) {
+        intersectDistinctOperands_.add(o.clone());
+      }
     }
+    for (SetOperand operand : other.operands_) operands_.add(operand.clone());
     analyzer_ = other.analyzer_;
     distinctAggInfo_ =
         (other.distinctAggInfo_ != null) ? other.distinctAggInfo_.clone() : null;
@@ -196,27 +232,75 @@ public class UnionStmt extends QueryStmt {
     toSqlString_ = (other.toSqlString_ != null) ? new String(other.toSqlString_) : null;
     hasAnalyticExprs_ = other.hasAnalyticExprs_;
     withClause_ = (other.withClause_ != null) ? other.withClause_.clone() : null;
-    unionResultExprs_ = Expr.cloneList(other.unionResultExprs_);
+    setOperationResultExprs_ = Expr.cloneList(other.setOperationResultExprs_);
     widestExprs_ = other.widestExprs_;
+    rewrittenStmt_ = other.rewrittenStmt_;
+  }
+
+  public static QueryStmt createUnionOrSetOperation(List<SetOperand> operands,
+      List<OrderByElement> orderByElements, LimitElement limitElement) {
+      boolean unionOnly = true;
+      for (SetOperand op : operands) {
+        if (op.getSetOperator() != null && op.getSetOperator() != SetOperator.UNION) {
+          unionOnly = false;
+          break;
+        }
+      }
+      if (unionOnly) {
+        return new UnionStmt(operands, orderByElements, limitElement);
+      } else {
+        return new SetOperationStmt(operands, orderByElements, limitElement);
+      }
+  }
+
+  public List<SetOperand> getOperands() { return operands_; }
+  public List<SetOperand> getUnionDistinctOperands() { return unionDistinctOperands_; }
+  public boolean hasUnionDistinctOps() { return !unionDistinctOperands_.isEmpty(); }
+  public List<SetOperand> getUnionAllOperands() { return unionAllOperands_; }
+  public boolean hasUnionAllOps() { return !unionAllOperands_.isEmpty(); }
+  public List<SetOperand> getExceptDistinctOperands() { return exceptDistinctOperands_; }
+  public boolean hasExceptDistinctOps() { return !exceptDistinctOperands_.isEmpty(); }
+  public List<SetOperand> getIntersectDistinctOperands() {
+    return intersectDistinctOperands_;
+  }
+  public boolean hasIntersectDistinctOps() {
+    return !intersectDistinctOperands_.isEmpty();
+  }
+  public boolean hasOnlyUnionOps() {
+    return (hasUnionDistinctOps() || hasUnionAllOps()) && !hasIntersectDistinctOps()
+        && !hasExceptDistinctOps();
+  }
+  public boolean hasOnlyUnionDistinctOps() {
+    return hasUnionDistinctOps() && !hasUnionAllOps() && !hasIntersectDistinctOps()
+        && !hasExceptDistinctOps();
+  }
+  public boolean hasOnlyUnionAllOps() {
+    return hasUnionAllOps() && !hasUnionDistinctOps() && !hasIntersectDistinctOps()
+        && !hasExceptDistinctOps();
+  }
+  public boolean hasOnlyIntersectDistinctOps() {
+    return hasIntersectDistinctOps() && !hasUnionDistinctOps() && !hasUnionAllOps()
+        && !hasExceptDistinctOps();
+  }
+  public boolean hasOnlyExceptDistinctOps() {
+    return hasExceptDistinctOps() && !hasUnionDistinctOps() && !hasUnionAllOps()
+        && !hasIntersectDistinctOps();
   }
 
-  public List<UnionOperand> getOperands() { return operands_; }
-  public List<UnionOperand> getDistinctOperands() { return distinctOperands_; }
-  public boolean hasDistinctOps() { return !distinctOperands_.isEmpty(); }
-  public List<UnionOperand> getAllOperands() { return allOperands_; }
-  public boolean hasAllOps() { return !allOperands_.isEmpty(); }
   public MultiAggregateInfo getDistinctAggInfo() { return distinctAggInfo_; }
   public boolean hasAnalyticExprs() { return hasAnalyticExprs_; }
   public TupleId getTupleId() { return tupleId_; }
+  public boolean hasRewrittenStmt() { return rewrittenStmt_ != null; }
+  public QueryStmt getRewrittenStmt() { return rewrittenStmt_; }
 
-  public void removeAllOperands() {
-    operands_.removeAll(allOperands_);
-    allOperands_.clear();
+  public void removeUnionAllOperands() {
+    operands_.removeAll(unionAllOperands_);
+    unionAllOperands_.clear();
   }
 
   @Override
   public void setDoTableMasking(boolean doTableMasking) {
-    for (UnionOperand op : operands_) {
+    for (SetOperand op : operands_) {
       op.getQueryStmt().setDoTableMasking(doTableMasking);
     }
   }
@@ -229,7 +313,6 @@ public class UnionStmt extends QueryStmt {
     // Propagates DISTINCT from right to left.
     propagateDistinct();
 
-    // Analyze all operands and make sure they return an equal number of exprs.
     analyzeOperands(analyzer);
 
     // Remember the SQL string before unnesting operands.
@@ -243,7 +326,7 @@ public class UnionStmt extends QueryStmt {
 
     // Compute hasAnalyticExprs_
     hasAnalyticExprs_ = false;
-    for (UnionOperand op: operands_) {
+    for (SetOperand op : operands_) {
       if (op.hasAnalyticExprs()) {
         hasAnalyticExprs_ = true;
         break;
@@ -252,10 +335,10 @@ public class UnionStmt extends QueryStmt {
 
     // Collect all result expr lists and cast the exprs as necessary.
     List<List<Expr>> resultExprLists = new ArrayList<>();
-    for (UnionOperand op: operands_) {
+    for (SetOperand op : operands_) {
       resultExprLists.add(op.getQueryStmt().getResultExprs());
     }
-    widestExprs_ = analyzer.castToUnionCompatibleTypes(resultExprLists);
+    widestExprs_ = analyzer.castToSetOpCompatibleTypes(resultExprLists);
 
     // Create tuple descriptor materialized by this UnionStmt, its resultExprs, and
     // its sortInfo if necessary.
@@ -263,10 +346,10 @@ public class UnionStmt extends QueryStmt {
     createSortInfo(analyzer);
 
     // Create unnested operands' smaps.
-    for (UnionOperand operand: operands_) setOperandSmap(operand, analyzer);
+    for (SetOperand operand : operands_) setOperandSmap(operand, analyzer);
 
     // Create distinctAggInfo, if necessary.
-    if (!distinctOperands_.isEmpty()) {
+    if (!unionDistinctOperands_.isEmpty()) {
       // Aggregate produces exactly the same tuple as the original union stmt.
       List<Expr> groupingExprs = Expr.cloneList(resultExprs_);
       try {
@@ -275,11 +358,11 @@ public class UnionStmt extends QueryStmt {
       } catch (AnalysisException e) {
         // Should never happen.
         throw new IllegalStateException(
-            "Error creating agg info in UnionStmt.analyze()", e);
+            "Error creating agg info in SetOperationStmt.analyze()", e);
       }
     }
 
-    unionResultExprs_ = Expr.cloneList(resultExprs_);
+    setOperationResultExprs_ = Expr.cloneList(resultExprs_);
     if (evaluateOrderBy_) createSortTupleInfo(analyzer);
     baseTblResultExprs_ = resultExprs_;
   }
@@ -289,7 +372,7 @@ public class UnionStmt extends QueryStmt {
    * Throws an AnalysisException if that is not the case, or if analyzing
    * an operand fails.
    */
-  private void analyzeOperands(Analyzer analyzer) throws AnalysisException {
+  protected void analyzeOperands(Analyzer analyzer) throws AnalysisException {
     for (int i = 0; i < operands_.size(); ++i) {
       operands_.get(i).analyze(analyzer);
       QueryStmt firstQuery = operands_.get(0).getQueryStmt();
@@ -315,7 +398,7 @@ public class UnionStmt extends QueryStmt {
     TupleDescriptor tupleDesc = analyzer.getDescTbl().getTupleDesc(tupleId_);
     // to keep things simple we materialize all grouping exprs = output slots,
     // regardless of what's being referenced externally
-    if (!distinctOperands_.isEmpty()) tupleDesc.materializeSlots();
+    if (!unionDistinctOperands_.isEmpty()) tupleDesc.materializeSlots();
 
     if (evaluateOrderBy_) sortInfo_.materializeRequiredSlots(analyzer, null);
 
@@ -325,7 +408,7 @@ public class UnionStmt extends QueryStmt {
     for (int i = 0; i < outputSlots.size(); ++i) {
       SlotDescriptor slotDesc = outputSlots.get(i);
       if (!slotDesc.isMaterialized()) continue;
-      for (UnionOperand op: operands_) {
+      for (SetOperand op : operands_) {
         exprs.add(op.getQueryStmt().getBaseTblResultExprs().get(i));
       }
     }
@@ -334,130 +417,165 @@ public class UnionStmt extends QueryStmt {
     }
     materializeSlots(analyzer, exprs);
 
-    for (UnionOperand op: operands_) {
+    for (SetOperand op : operands_) {
       op.getQueryStmt().materializeRequiredSlots(analyzer);
     }
   }
 
   /**
-   * Fill distinct-/allOperands and performs possible unnesting of UnionStmt
-   * operands in the process.
+   * String representation of queryStmt used in reporting errors.
+   * Allow subclasses to override this.
    */
-  private void unnestOperands(Analyzer analyzer) throws AnalysisException {
-    if (operands_.size() == 1) {
-      // ValuesStmt for a single row.
-      allOperands_.add(operands_.get(0));
-      return;
-    }
+  protected String queryStmtToSql(QueryStmt queryStmt) {
+    return queryStmt.toSql();
+  }
 
-    // find index of first ALL operand
-    int firstUnionAllIdx = operands_.size();
-    for (int i = 1; i < operands_.size(); ++i) {
-      UnionOperand operand = operands_.get(i);
-      if (operand.getQualifier() == Qualifier.ALL) {
-        firstUnionAllIdx = (i == 1 ? 0 : i);
-        break;
+  /**
+   * Propagates DISTINCT (if present) from right to left.
+   * Implied associativity:
+   * A UNION ALL B UNION DISTINCT C = (A UNION ALL B) UNION DISTINCT C
+   * = A UNION DISTINCT B UNION DISTINCT C
+   */
+  protected void propagateDistinct() {
+    int lastDistinctPos = -1;
+    for (int i = operands_.size() - 1; i > 0; --i) {
+      SetOperand operand = operands_.get(i);
+      if (lastDistinctPos != -1) {
+        // There is a DISTINCT somewhere to the right.
+        operand.setQualifier(Qualifier.DISTINCT);
+      } else if (operand.getQualifier() == Qualifier.DISTINCT) {
+        lastDistinctPos = i;
       }
     }
-    // operands[0] is always implicitly ALL, so operands[1] can't be the
-    // first one
-    Preconditions.checkState(firstUnionAllIdx != 1);
+  }
 
-    // unnest DISTINCT operands
-    Preconditions.checkState(distinctOperands_.isEmpty());
-    for (int i = 0; i < firstUnionAllIdx; ++i) {
-      unnestOperand(distinctOperands_, Qualifier.DISTINCT, operands_.get(i));
+  @Override
+  public void rewriteExprs(ExprRewriter rewriter) throws AnalysisException {
+    // No rewrites are needed for distinctAggInfo_, resultExprs_, or baseTblResultExprs_
+    // as those Exprs are simply SlotRefs.
+    for (SetOperand op : operands_) op.getQueryStmt().rewriteExprs(rewriter);
+    if (orderByElements_ != null) {
+      for (OrderByElement orderByElem : orderByElements_) {
+        orderByElem.setExpr(rewriter.rewrite(orderByElem.getExpr(), analyzer_));
+      }
     }
+  }
 
-    // unnest ALL operands
-    Preconditions.checkState(allOperands_.isEmpty());
-    for (int i = firstUnionAllIdx; i < operands_.size(); ++i) {
-      unnestOperand(allOperands_, Qualifier.ALL, operands_.get(i));
+  @Override
+  public void getMaterializedTupleIds(List<TupleId> tupleIdList) {
+    // Return the sort tuple if there is an evaluated order by.
+    if (evaluateOrderBy_) {
+      tupleIdList.add(sortInfo_.getSortTupleDescriptor().getId());
+    } else {
+      tupleIdList.add(tupleId_);
     }
+  }
 
-    for (UnionOperand op: distinctOperands_) op.setQualifier(Qualifier.DISTINCT);
-    for (UnionOperand op: allOperands_) op.setQualifier(Qualifier.ALL);
-
-    operands_.clear();
-    operands_.addAll(distinctOperands_);
-    operands_.addAll(allOperands_);
+  @Override
+  public void collectTableRefs(List<TableRef> tblRefs, boolean fromClauseOnly) {
+    super.collectTableRefs(tblRefs, fromClauseOnly);
+    for (SetOperand op : operands_) {
+      op.getQueryStmt().collectTableRefs(tblRefs, fromClauseOnly);
+    }
   }
 
-  /**
-   * Sets the smap for the given operand. It maps from the output slots this union's
-   * tuple to the corresponding result exprs of the operand.
-   */
-  private void setOperandSmap(UnionOperand operand, Analyzer analyzer) {
-    TupleDescriptor tupleDesc = analyzer.getDescTbl().getTupleDesc(tupleId_);
-    // operands' smaps were already set in the operands' analyze()
-    operand.getSmap().clear();
-    List<Expr> resultExprs = operand.getQueryStmt().getResultExprs();
-    Preconditions.checkState(resultExprs.size() == tupleDesc.getSlots().size());
-    for (int i = 0; i < tupleDesc.getSlots().size(); ++i) {
-      SlotDescriptor outputSlot = tupleDesc.getSlots().get(i);
-      // Map to the original (uncast) result expr of the operand.
-      Expr origExpr = resultExprs.get(i).unwrapExpr(true).clone();
-      operand.getSmap().put(new SlotRef(outputSlot), origExpr);
+  @Override
+  public void collectInlineViews(Set<FeView> inlineViews) {
+    super.collectInlineViews(inlineViews);
+    for (SetOperand operand : operands_) {
+      operand.getQueryStmt().collectInlineViews(inlineViews);
     }
   }
 
-  /**
-   * Add a single operand to the target list; if the operand itself is a UnionStmt, apply
-   * unnesting to the extent possible (possibly modifying 'operand' in the process).
-   */
-  private void unnestOperand(
-      List<UnionOperand> target, Qualifier targetQualifier, UnionOperand operand) {
-    Preconditions.checkState(operand.isAnalyzed());
-    QueryStmt queryStmt = operand.getQueryStmt();
-    if (queryStmt instanceof SelectStmt) {
-      target.add(operand);
-      return;
+  @Override
+  public String toSql(ToSqlOptions options) {
+    if (!options.showRewritten() && toSqlString_ != null) return toSqlString_;
+
+    StringBuilder strBuilder = new StringBuilder();
+    Preconditions.checkState(operands_.size() > 0);
+
+    if (withClause_ != null) {
+      strBuilder.append(withClause_.toSql(options));
+      strBuilder.append(" ");
     }
 
-    Preconditions.checkState(queryStmt instanceof UnionStmt);
-    UnionStmt unionStmt = (UnionStmt) queryStmt;
-    if (unionStmt.hasLimit() || unionStmt.hasOffset()) {
-      // we must preserve the nested Union
-      target.add(operand);
-    } else if (targetQualifier == Qualifier.DISTINCT || !unionStmt.hasDistinctOps()) {
-      // there is no limit in the nested Union and we can absorb all of its
-      // operands as-is
-      target.addAll(unionStmt.getDistinctOperands());
-      target.addAll(unionStmt.getAllOperands());
+    strBuilder.append(operands_.get(0).getQueryStmt().toSql(options));
+    for (int i = 1; i < operands_.size() - 1; ++i) {
+      String opName = operands_.get(i).getSetOperator() != null ?
+          operands_.get(i).getSetOperator().name() :
+          "UNION";
+      strBuilder.append(" " + opName + " "
+          + ((operands_.get(i).getQualifier() == Qualifier.ALL) ? "ALL " : ""));
+      if (operands_.get(i).getQueryStmt() instanceof SetOperationStmt) {
+        strBuilder.append("(");
+      }
+      strBuilder.append(operands_.get(i).getQueryStmt().toSql(options));
+      if (operands_.get(i).getQueryStmt() instanceof SetOperationStmt) {
+        strBuilder.append(")");
+      }
+    }
+    // Determine whether we need parentheses around the last union operand.
+    SetOperand lastOperand = operands_.get(operands_.size() - 1);
+    QueryStmt lastQueryStmt = lastOperand.getQueryStmt();
+    strBuilder.append(" " + lastOperand.getSetOperator().name() + " "
+        + ((lastOperand.getQualifier() == Qualifier.ALL) ? "ALL " : ""));
+    if (lastQueryStmt instanceof SetOperationStmt
+        || ((hasOrderByClause() || hasLimit() || hasOffset()) && !lastQueryStmt.hasLimit()
+               && !lastQueryStmt.hasOffset() && !lastQueryStmt.hasOrderByClause())) {
+      strBuilder.append("(");
+      strBuilder.append(lastQueryStmt.toSql(options));
+      strBuilder.append(")");
     } else {
-      // the nested Union contains some Distinct ops and we're accumulating
-      // into our All ops; unnest only the All ops and leave the rest in place
-      target.addAll(unionStmt.getAllOperands());
-      unionStmt.removeAllOperands();
-      target.add(operand);
+      strBuilder.append(lastQueryStmt.toSql(options));
+    }
+    // Order By clause
+    if (hasOrderByClause()) {
+      strBuilder.append(" ORDER BY ");
+      for (int i = 0; i < orderByElements_.size(); ++i) {
+        strBuilder.append(orderByElements_.get(i).toSql(options));
+        strBuilder.append((i + 1 != orderByElements_.size()) ? ", " : "");
+      }
     }
+    // Limit clause.
+    strBuilder.append(limitElement_.toSql(options));
+    return strBuilder.toString();
   }
 
-  /**
-   * String representation of queryStmt used in reporting errors.
-   * Allow subclasses to override this.
-   */
-  protected String queryStmtToSql(QueryStmt queryStmt) {
-    return queryStmt.toSql();
+  @Override
+  public List<String> getColLabels() {
+    Preconditions.checkState(operands_.size() > 0);
+    return operands_.get(0).getQueryStmt().getColLabels();
   }
 
+  public List<Expr> getSetOperationResultExprs() { return setOperationResultExprs_; }
+
+  public List<Expr> getWidestExprs() { return widestExprs_; }
+
+  @Override
+  public SetOperationStmt clone() { return new SetOperationStmt(this); }
+
   /**
-   * Propagates DISTINCT (if present) from right to left.
-   * Implied associativity:
-   * A UNION ALL B UNION DISTINCT C = (A UNION ALL B) UNION DISTINCT C
-   * = A UNION DISTINCT B UNION DISTINCT C
+   * Undoes all changes made by analyze() except distinct propagation and unnesting. After
+   * analysis, operands_ contains the list of unnested operands with qualifiers adjusted
+   * to reflect distinct propagation. Every operand in that list is reset(). The
+   * unionDistinctOperands_ and unionAllOperands_ are cleared because they are redundant
+   * with operands_.
    */
-  private void propagateDistinct() {
-    int lastDistinctPos = -1;
-    for (int i = operands_.size() - 1; i > 0; --i) {
-      UnionOperand operand = operands_.get(i);
-      if (lastDistinctPos != -1) {
-        // There is a DISTINCT somewhere to the right.
-        operand.setQualifier(Qualifier.DISTINCT);
-      } else if (operand.getQualifier() == Qualifier.DISTINCT) {
-        lastDistinctPos = i;
-      }
-    }
+  @Override
+  public void reset() {
+    super.reset();
+    for (SetOperand op : operands_) op.reset();
+    unionDistinctOperands_.clear();
+    unionAllOperands_.clear();
+    intersectDistinctOperands_.clear();
+    exceptDistinctOperands_.clear();
+    distinctAggInfo_ = null;
+    tupleId_ = null;
+    toSqlString_ = null;
+    hasAnalyticExprs_ = false;
+    setOperationResultExprs_.clear();
+    widestExprs_ = null;
+    rewrittenStmt_ = null;
   }
 
   /**
@@ -472,7 +590,7 @@ public class UnionStmt extends QueryStmt {
     tupleDesc.setIsMaterialized(true);
     tupleId_ = tupleDesc.getId();
     if (LOG.isTraceEnabled()) {
-      LOG.trace("UnionStmt.createMetadata: tupleId=" + tupleId_.toString());
+      LOG.trace("SetOperationStmt.createMetadata: tupleId=" + tupleId_.toString());
     }
 
     // One slot per expr in the select blocks. Use first select block as representative.
@@ -517,7 +635,7 @@ public class UnionStmt extends QueryStmt {
       // to operands' result exprs (if those happen to be slotrefs);
       // don't do that if the operand computes analytic exprs
       // (see Planner.createInlineViewPlan() for the reasoning)
-      for (UnionOperand op: operands_) {
+      for (SetOperand op : operands_) {
         Expr resultExpr = op.getQueryStmt().getResultExprs().get(i);
         slotDesc.addSourceExpr(resultExpr);
         SlotRef slotRef = resultExpr.unwrapSlotRef(false);
@@ -534,125 +652,187 @@ public class UnionStmt extends QueryStmt {
     baseTblResultExprs_ = resultExprs_;
   }
 
-  @Override
-  public void rewriteExprs(ExprRewriter rewriter) throws AnalysisException {
-    for (UnionOperand op: operands_) op.getQueryStmt().rewriteExprs(rewriter);
-    if (orderByElements_ != null) {
-      for (OrderByElement orderByElem: orderByElements_) {
-        orderByElem.setExpr(rewriter.rewrite(orderByElem.getExpr(), analyzer_));
+  /**
+   * Fill distinct-/all Operands and performs possible unnesting of SetOperationStmt
+   * operands in the process.
+   */
+  private void unnestOperands(Analyzer analyzer) throws AnalysisException {
+    if (operands_.size() == 1) {
+      // ValuesStmt for a single row.
+      unionAllOperands_.add(operands_.get(0));
+      return;
+    }
+
+    // find index of first ALL operand
+    int firstUnionAllIdx = operands_.size();
+    for (int i = 1; i < operands_.size(); ++i) {
+      SetOperand operand = operands_.get(i);
+      if (operand.getQualifier() == Qualifier.ALL) {
+        firstUnionAllIdx = (i == 1 ? 0 : i);
+        break;
       }
     }
-  }
 
-  @Override
-  public void getMaterializedTupleIds(List<TupleId> tupleIdList) {
-    // Return the sort tuple if there is an evaluated order by.
-    if (evaluateOrderBy_) {
-      tupleIdList.add(sortInfo_.getSortTupleDescriptor().getId());
-    } else {
-      tupleIdList.add(tupleId_);
+    List<SetOperand> localOps = new ArrayList<>();
+    List<SetOperand> tempOps = new ArrayList<>();
+    // operands[0] is always implicitly ALL, so operands[1] can't be the
+    // first one
+    Preconditions.checkState(firstUnionAllIdx != 1);
+
+    // unnest DISTINCT operands
+    for (int i = 0; i < firstUnionAllIdx; ++i) {
+      tempOps.clear();
+      SetOperator op = operands_.get(i).getSetOperator();
+      if (op == null || op == SetOperator.UNION) {
+        // It is safe to handle the first operand here as we know the first ALL index
+        // isn't 1 therefore any union can be absorbed.
+        unnestOperand(tempOps, Qualifier.DISTINCT, operands_.get(i));
+        localOps.addAll(tempOps);
+        // If the first operand isn't unnested we treat it as distinct
+        unionDistinctOperands_.addAll(tempOps);
+      } else if (op == SetOperator.EXCEPT) {
+        unnestOperand(tempOps, Qualifier.DISTINCT, operands_.get(i));
+        localOps.addAll(tempOps);
+        exceptDistinctOperands_.addAll(tempOps);
+      } else if (op == SetOperator.INTERSECT) {
+        unnestOperand(tempOps, Qualifier.DISTINCT, operands_.get(i));
+        localOps.addAll(tempOps);
+        intersectDistinctOperands_.addAll(tempOps);
+      } else {
+          throw new AnalysisException("Invalid operand in SetOperationStmt: " +
+              queryStmtToSql(this)); }
     }
-  }
 
-  @Override
-  public void collectTableRefs(List<TableRef> tblRefs, boolean fromClauseOnly) {
-    super.collectTableRefs(tblRefs, fromClauseOnly);
-    for (UnionOperand op: operands_) {
-      op.getQueryStmt().collectTableRefs(tblRefs, fromClauseOnly);
+    // unnest ALL operands
+    for (int i = firstUnionAllIdx; i < operands_.size(); ++i) {
+      tempOps.clear();
+      unnestOperand(tempOps, Qualifier.ALL, operands_.get(i));
+      localOps.addAll(tempOps);
+      unionAllOperands_.addAll(tempOps);
     }
-  }
 
-  @Override
-  public void collectInlineViews(Set<FeView> inlineViews) {
-    super.collectInlineViews(inlineViews);
-    for (UnionOperand operand : operands_) {
-      operand.getQueryStmt().collectInlineViews(inlineViews);
+    for (SetOperand op : unionDistinctOperands_) {
+      op.setSetOperator(SetOperator.UNION);
+      op.setQualifier(Qualifier.DISTINCT);
+    }
+    for (SetOperand op : intersectDistinctOperands_) {
+      op.setSetOperator(SetOperator.INTERSECT);
+      op.setQualifier(Qualifier.DISTINCT);
+    }
+    for (SetOperand op : exceptDistinctOperands_) {
+      op.setSetOperator(SetOperator.EXCEPT);
+      op.setQualifier(Qualifier.DISTINCT);
+    }
+    for (SetOperand op : unionAllOperands_) {
+      op.setSetOperator(SetOperator.UNION);
+      op.setQualifier(Qualifier.ALL);
     }
-  }
 
-  @Override
-  public String toSql(ToSqlOptions options) {
-    if (!options.showRewritten() && toSqlString_ != null) return toSqlString_;
+    operands_.clear();
+    operands_.addAll(localOps);
+  }
 
-    StringBuilder strBuilder = new StringBuilder();
-    Preconditions.checkState(operands_.size() > 0);
+  /**
+   * Add a single operand to the target list; if the operand itself is a SetOperationStmt,
+   * apply unnesting to the extent possible (possibly modifying 'operand' in the process).
+   *
+   * Absorb means convert qualifier into target type, this applies to ALL -> DISTINCT,
+   * currently only done with UNIQUE propagateDistinct() ensures ALL operands are always
+   * the rightmost, therefore they can be pulled out and absorbed into DISTINCT or just
+   * added to the outer ALL.
+   *
+   * EXCEPT is never unnested.
+   * (E E) E -> (E E) E
+   * INTERSECT is unnested unless first
+   * (I) I I -> (I) I I
+   * I (I) I -> I I I
+   * I (I (I I)) -> I I I I
+   * UNION ALL plucks UNION ALL out
+   * (UD UD UA) UA UA -> (UD UD) UA UA UA
+   * UNION DISTINCT absorbs other unions.
+   * (UD UD UA) UD UA -> UD UD UD UD UA
+   * (UA UA UA) UA UA -> UA UA UA UA UA
+   * (UA UA UA) UD UD -> UD UD UD UD UD
+   *
+   */
+  private void unnestOperand(
+      List<SetOperand> target, Qualifier targetQualifier, SetOperand operand) {
+    Preconditions.checkState(operand.isAnalyzed());
+    QueryStmt queryStmt = operand.getQueryStmt();
+    if (queryStmt instanceof SelectStmt) {
+      target.add(operand);
+      return;
+    }
 
-    if (withClause_ != null) {
-      strBuilder.append(withClause_.toSql(options));
-      strBuilder.append(" ");
+    Preconditions.checkState(queryStmt instanceof SetOperationStmt);
+    SetOperationStmt stmt = (SetOperationStmt) queryStmt;
+    if (stmt.hasLimit() || stmt.hasOffset()) {
+      // we must preserve the nested Set Operation
+      target.add(operand);
+      return;
     }
 
-    strBuilder.append(operands_.get(0).getQueryStmt().toSql(options));
-    for (int i = 1; i < operands_.size() - 1; ++i) {
-      strBuilder.append(" UNION " +
-          ((operands_.get(i).getQualifier() == Qualifier.ALL) ? "ALL " : ""));
-      if (operands_.get(i).getQueryStmt() instanceof UnionStmt) {
-        strBuilder.append("(");
+    // 1. Unnest INTERSECT only if the nested statement contains only INTERSECTs.
+    // 2. Union distinct will absorb UNION DISTNCT and UNION ALL if no INTERSECT / EXCEPT
+    // are present
+    // 3. Union ALL will always be to the right of any DISTINCT, so we can unnest ALL if
+    // the target is ALL
+    // 4. For the first operand with a distinct target, we unnest and absorb only when
+    // UNIONs are the only operator. All INTERSECT in the first operand aren't unnested,
+    // this doesn't affect correctness, it's just a simplification.
+    SetOperator targetOperator = operand.getSetOperator();
+    if (targetOperator == SetOperator.INTERSECT && stmt.hasOnlyIntersectDistinctOps()) {
+      target.addAll(stmt.getIntersectDistinctOperands());
+    } else if (targetOperator == SetOperator.EXCEPT && stmt.hasOnlyExceptDistinctOps()) {
+      // EXCEPT should not be unnested
+      target.add(operand);
+    } else if (targetOperator == SetOperator.UNION && stmt.hasOnlyUnionOps()) {
+      if (targetQualifier == Qualifier.DISTINCT || !stmt.hasUnionDistinctOps()) {
+        target.addAll(stmt.getUnionDistinctOperands());
+        target.addAll(stmt.getUnionAllOperands());
+      } else {
+        target.addAll(stmt.getUnionAllOperands());
+        stmt.removeUnionAllOperands();
+        target.add(operand);
       }
-      strBuilder.append(operands_.get(i).getQueryStmt().toSql(options));
-      if (operands_.get(i).getQueryStmt() instanceof UnionStmt) {
-        strBuilder.append(")");
+    // Special cases for the first operand.
+    } else if (targetOperator == null && targetQualifier == Qualifier.ALL) {
+      // Case 3
+      target.addAll(stmt.getUnionAllOperands());
+      if (stmt.hasUnionDistinctOps() || stmt.hasExceptDistinctOps()
+          || stmt.hasIntersectDistinctOps()) {
+        stmt.removeUnionAllOperands();
+        target.add(operand);
       }
-    }
-    // Determine whether we need parenthesis around the last union operand.
-    UnionOperand lastOperand = operands_.get(operands_.size() - 1);
-    QueryStmt lastQueryStmt = lastOperand.getQueryStmt();
-    strBuilder.append(" UNION " +
-        ((lastOperand.getQualifier() == Qualifier.ALL) ? "ALL " : ""));
-    if (lastQueryStmt instanceof UnionStmt ||
-        ((hasOrderByClause() || hasLimit() || hasOffset()) &&
-            !lastQueryStmt.hasLimit() && !lastQueryStmt.hasOffset() &&
-            !lastQueryStmt.hasOrderByClause())) {
-      strBuilder.append("(");
-      strBuilder.append(lastQueryStmt.toSql(options));
-      strBuilder.append(")");
-    } else {
-      strBuilder.append(lastQueryStmt.toSql(options));
-    }
-    // Order By clause
-    if (hasOrderByClause()) {
-      strBuilder.append(" ORDER BY ");
-      for (int i = 0; i < orderByElements_.size(); ++i) {
-        strBuilder.append(orderByElements_.get(i).toSql(options));
-        strBuilder.append((i+1 != orderByElements_.size()) ? ", " : "");
+    } else if (targetOperator == null && targetQualifier == Qualifier.DISTINCT) {
+      // Case 4
+      if (stmt.hasOnlyUnionOps()) {
+        target.addAll(stmt.getUnionDistinctOperands());
+        target.addAll(stmt.getUnionAllOperands());
+      } else {
+        target.add(operand);
       }
+    } else {
+      // Mixed operators are not safe to unnest
+      target.add(operand);
     }
-    // Limit clause.
-    strBuilder.append(limitElement_.toSql(options));
-    return strBuilder.toString();
-  }
-
-  @Override
-  public List<String> getColLabels() {
-    Preconditions.checkState(operands_.size() > 0);
-    return operands_.get(0).getQueryStmt().getColLabels();
   }
 
-  public List<Expr> getUnionResultExprs() { return unionResultExprs_; }
-
-  public List<Expr> getWidestExprs() { return widestExprs_; }
-
-  @Override
-  public UnionStmt clone() { return new UnionStmt(this); }
-
   /**
-   * Undoes all changes made by analyze() except distinct propagation and unnesting.
-   * After analysis, operands_ contains the list of unnested operands with qualifiers
-   * adjusted to reflect distinct propagation. Every operand in that list is reset().
-   * The distinctOperands_ and allOperands_ are cleared because they are redundant
-   * with operands_.
+   * Sets the smap for the given operand. It maps from the output slots this union's
+   * tuple to the corresponding result exprs of the operand.
    */
-  @Override
-  public void reset() {
-    super.reset();
-    for (UnionOperand op: operands_) op.reset();
-    distinctOperands_.clear();
-    allOperands_.clear();
-    distinctAggInfo_ = null;
-    tupleId_ = null;
-    toSqlString_ = null;
-    hasAnalyticExprs_ = false;
-    unionResultExprs_.clear();
-    widestExprs_ = null;
+  private void setOperandSmap(SetOperand operand, Analyzer analyzer) {
+    TupleDescriptor tupleDesc = analyzer.getDescTbl().getTupleDesc(tupleId_);
+    // operands' smaps were already set in the operands' analyze()
+    operand.getSmap().clear();
+    List<Expr> resultExprs = operand.getQueryStmt().getResultExprs();
+    Preconditions.checkState(resultExprs.size() == tupleDesc.getSlots().size());
+    for (int i = 0; i < tupleDesc.getSlots().size(); ++i) {
+      SlotDescriptor outputSlot = tupleDesc.getSlots().get(i);
+      // Map to the original (uncast) result expr of the operand.
+      Expr origExpr = resultExprs.get(i).unwrapExpr(true).clone();
+      operand.getSmap().put(new SlotRef(outputSlot), origExpr);
+    }
   }
 }
diff --git a/fe/src/main/java/org/apache/impala/analysis/StmtRewriter.java b/fe/src/main/java/org/apache/impala/analysis/StmtRewriter.java
index e546244..2aa7f8f 100644
--- a/fe/src/main/java/org/apache/impala/analysis/StmtRewriter.java
+++ b/fe/src/main/java/org/apache/impala/analysis/StmtRewriter.java
@@ -22,11 +22,12 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.impala.analysis.AnalysisContext.AnalysisResult;
-import org.apache.impala.analysis.UnionStmt.UnionOperand;
+import org.apache.impala.analysis.SetOperationStmt.SetOperand;
+import org.apache.impala.analysis.SetOperationStmt.SetOperator;
 import org.apache.impala.common.AnalysisException;
+import org.apache.impala.common.TableAliasGenerator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicates;
 import com.google.common.collect.Iterables;
@@ -75,7 +76,7 @@ public class StmtRewriter {
 
   /**
    * Calls the appropriate rewrite method based on the specific type of query stmt. See
-   * rewriteSelectStatement() and rewriteUnionStatement() documentation.
+   * rewriteSelectStatement() and rewriteSetOperationStatement() documentation.
    */
   protected void rewriteQueryStatement(QueryStmt stmt, Analyzer analyzer)
       throws AnalysisException {
@@ -83,8 +84,8 @@ public class StmtRewriter {
     Preconditions.checkState(stmt.isAnalyzed());
     if (stmt instanceof SelectStmt) {
       rewriteSelectStatement((SelectStmt) stmt, analyzer);
-    } else if (stmt instanceof UnionStmt) {
-      rewriteUnionStatement((UnionStmt) stmt);
+    } else if (stmt instanceof SetOperationStmt) {
+      rewriteSetOperationStatement((SetOperationStmt) stmt, analyzer);
     } else {
       throw new AnalysisException(
           "Subqueries not supported for " + stmt.getClass().getSimpleName() +
@@ -98,7 +99,15 @@ public class StmtRewriter {
       if (!(tblRef instanceof InlineViewRef)) continue;
       InlineViewRef inlineViewRef = (InlineViewRef) tblRef;
       rewriteQueryStatement(inlineViewRef.getViewStmt(), inlineViewRef.getAnalyzer());
+      // SetOperationStmt can be rewritten to SelectStmt.
+      if (inlineViewRef.getViewStmt() instanceof SetOperationStmt
+          && !(inlineViewRef.getViewStmt() instanceof UnionStmt)) {
+        inlineViewRef.queryStmt_ =
+            ((SetOperationStmt) inlineViewRef.getViewStmt()).getRewrittenStmt();
+        Preconditions.checkState(inlineViewRef.queryStmt_ != null);
+      }
     }
+
     // Currently only SubqueryRewriter touches the where clause. Recurse into the where
     // clause when the need arises.
     rewriteSelectStmtHook(stmt, analyzer);
@@ -106,14 +115,254 @@ public class StmtRewriter {
   }
 
   /**
-   * Rewrite all operands in a UNION. The conditions that apply to SelectStmt rewriting
-   * also apply here.
+   * Generate the join predicate for EXCEPT / INTERSECT rewrites. The set semantics
+   * require null = null to evaluate to true.
    */
-  private void rewriteUnionStatement(UnionStmt stmt) throws AnalysisException {
-    for (UnionOperand operand : stmt.getOperands()) {
-      Preconditions.checkState(operand.getQueryStmt() instanceof SelectStmt);
-      rewriteSelectStatement((SelectStmt) operand.getQueryStmt(), operand.getAnalyzer());
+  private static Expr getSetOpJoinPredicates(
+      InlineViewRef left, InlineViewRef right, SetOperator operator) {
+    Preconditions.checkState(left.getColLabels().size() == right.getColLabels().size());
+    Preconditions.checkState(
+        operator == SetOperator.EXCEPT || operator == SetOperator.INTERSECT);
+
+    BinaryPredicate.Operator eqOp = BinaryPredicate.Operator.NOT_DISTINCT;
+    final List<Expr> conjuncts =
+        Lists.newArrayListWithCapacity(left.getColLabels().size());
+    for (int i = 0; i < left.getColLabels().size(); ++i) {
+      conjuncts.add(new BinaryPredicate(eqOp,
+          new SlotRef(
+              Path.createRawPath(left.getUniqueAlias(), left.getColLabels().get(i))),
+          new SlotRef(
+              Path.createRawPath(right.getUniqueAlias(), right.getColLabels().get(i)))));
+    }
+    return CompoundPredicate.createConjunctivePredicate(conjuncts);
+  }
+
+  /**
+   * Rewrite all operands in a SetOperation (EXCEPT/INTERSECT/UNION).
+   *
+   * IMPALA-9944: To support the [ALL] qualifier for EXCEPT & INTERSECT
+   *
+   * EXCEPT is rewritten using a Left Anti Join with an additional IS NOT DISTINCT
+   * predicate to ensure NULL equality returns true. INTERSECT is handled similarly
+   * however using LEFT SEMI or INNER Joins.
+   *
+   * We walk the list of set operands from left to right, combining consecutive EXCEPT and
+   * INTERSECT operands by combining their joins. When we encounter a UNION operand, we
+   * start a new SelectStmt to combine all subsequent UNIONS, with the leftmost operand
+   * being the joined statements from EXCEPT / INTERSECT
+   *
+   * Example: SELECT a FROM T1 INTERSECT SELECT a FROM T2 UNION SELECT b FROM T3
+   *
+   * We start by building a new SelectStmt to combine the operands as joined view.
+   *
+   * SELECT DISTINCT * FROM (SELECT a FROM T1) $a$1 LEFT SEMI JOIN (SELECT a FROM T2) $a$2
+   *   ON $a$1.a IS NOT DISTINCT FROM $a$2.a
+   *
+   * Subsequent UNION operands require a switch to a new SelectStmt. When creating a new
+   * SelectStmt if the current operand is a UNION then the first operand's from clause
+   * will contain the previous SelectStmt. When switching to EXCEPT/INTERSECT the first
+   * element in the from clause will contain the SelectStmt from the Union.
+   *
+   * Now we create a new SelectStmt for the UNION.
+   *
+   * SELECT * FROM ( op1 UNION op2 )
+   *
+   * Filling in op1 and op2
+   *
+   * SELECT * FROM(
+   *  SELECT DISTINCT * FROM (SELECT a FROM T1) $a$1 LEFT SEMI JOIN
+   *  (SELECT a FROM T2) $a$2 ON $a$1.a IS NOT DISTINCT FROM $a$2.a
+   * UNION
+   *   SELECT b FROM T3) $a$3
+   *
+   * We continue to create new SelectStmts whenever we switch from a
+   * series of EXCEPT/INTERSECT to UNION.
+   *
+   */
+  private void rewriteSetOperationStatement(SetOperationStmt stmt, Analyzer analyzer)
+      throws AnalysisException {
+    // Early out for UnionStmt as we don't rewrite the union operator
+    if (stmt instanceof UnionStmt) {
+      for (SetOperand operand : stmt.getOperands()) {
+        rewriteQueryStatement(operand.getQueryStmt(), operand.getAnalyzer());
+        if (operand.getQueryStmt() instanceof SetOperationStmt
+            && !(operand.getQueryStmt() instanceof UnionStmt)) {
+          SetOperationStmt setOpStmt = ((SetOperationStmt) operand.getQueryStmt());
+          if (setOpStmt.hasRewrittenStmt()) {
+            QueryStmt rewrittenStmt = setOpStmt.getRewrittenStmt();
+            operand.setQueryStmt(rewrittenStmt);
+          }
+        }
+      }
+      return;
+    }
+
+    // During each iteration of the loop below, exactly one of eiSelect or uSelect becomes
+    // non-null, they function as placeholders for the current sequence of rewrites for
+    // except/intersect or union operands respectively. If the last operand processed was
+    // a union, uSelect is the current select statement that has unionStmt nested inside,
+    // which in turn contains preceding union operands.  If the last operator processed
+    // was an except or intersect, eiSelect is the current select statement containing
+    // preceding except or intersect operands in the from clause.
+    TableAliasGenerator tableAliasGenerator = new TableAliasGenerator(analyzer, null);
+    SelectStmt uSelect = null, eiSelect = null;
+    SetOperationStmt unionStmt = null;
+
+    SetOperand firstOperand = stmt.getOperands().get(0);
+    rewriteQueryStatement(firstOperand.getQueryStmt(), firstOperand.getAnalyzer());
+    if (firstOperand.getQueryStmt() instanceof SetOperationStmt) {
+      SetOperationStmt setOpStmt = ((SetOperationStmt) firstOperand.getQueryStmt());
+      if (setOpStmt.hasRewrittenStmt()) {
+        firstOperand.setQueryStmt(setOpStmt.getRewrittenStmt());
+      }
     }
+
+    for (int i = 1; i < stmt.getOperands().size(); ++i) {
+      SetOperand operand = stmt.getOperands().get(i);
+      rewriteQueryStatement(operand.getQueryStmt(), operand.getAnalyzer());
+      if (operand.getQueryStmt() instanceof SetOperationStmt) {
+        SetOperationStmt setOpStmt = ((SetOperationStmt) operand.getQueryStmt());
+        if (setOpStmt.hasRewrittenStmt()) {
+          operand.setQueryStmt(setOpStmt.getRewrittenStmt());
+        }
+      }
+
+      switch (operand.getSetOperator()) {
+        case EXCEPT:
+        case INTERSECT:
+          if (eiSelect == null) {
+            // For a new SelectStmt the left most tableref will either by the first
+            // operand or a the SelectStmt from the union operands.
+            InlineViewRef leftMostView = null;
+            SelectList sl =
+                new SelectList(Lists.newArrayList(SelectListItem.createStarItem(null)));
+            // Intersect/Except have set semantics in SQL they must not return duplicates
+            // As an optimization if the leftmost operand is already distinct we remove
+            // the distinct here.
+            // This would be best done in a cost based manner during planning.
+            sl.setIsDistinct(true);
+            eiSelect = new SelectStmt(sl, null, null, null, null, null, null);
+
+            if (i == 1) {
+              if (firstOperand.getQueryStmt() instanceof SelectStmt) {
+                // optimize out the distinct aggregation in the outer query
+                if (((SelectStmt) firstOperand.getQueryStmt()).getSelectList()
+                    .isDistinct()) {
+                  sl.setIsDistinct(false);
+                }
+              }
+              leftMostView = new InlineViewRef(tableAliasGenerator.getNextAlias(),
+                  firstOperand.getQueryStmt(), (TableSampleClause) null);
+              leftMostView.analyze(analyzer);
+              eiSelect.getTableRefs().add(leftMostView);
+            }
+
+            // There was a union operator before this one.
+            if (uSelect != null) {
+              Preconditions.checkState(i != 1);
+              if (uSelect.getSelectList().isDistinct()
+                  && eiSelect.getTableRefs().size() == 0) {
+                // optimize out the distinct aggregation in the outer query
+                sl.setIsDistinct(false);
+              }
+              leftMostView = new InlineViewRef(
+                  tableAliasGenerator.getNextAlias(), uSelect, (TableSampleClause) null);
+              leftMostView.analyze(analyzer);
+              eiSelect.getTableRefs().add(leftMostView);
+              uSelect = null;
+            }
+          }
+
+          // INTERSECT => Left Semi Join and EXCEPT => Left Anti Join
+          JoinOperator joinOp = operand.getSetOperator() == SetOperator.EXCEPT ?
+              JoinOperator.LEFT_ANTI_JOIN :
+              JoinOperator.LEFT_SEMI_JOIN;
+          TableRef rightMostTbl =
+              eiSelect.getTableRefs().get(eiSelect.getTableRefs().size() - 1);
+
+          // As an optimization we can rewrite INTERSECT with an inner join if both
+          // operands return distinct rows.
+          if (operand.getQueryStmt() instanceof SelectStmt) {
+            SelectStmt inner = ((SelectStmt) operand.getQueryStmt());
+            if (inner.getSelectList().isDistinct()) {
+              if (rightMostTbl instanceof InlineViewRef) {
+                QueryStmt outer = ((InlineViewRef) rightMostTbl).getViewStmt();
+                if (outer instanceof SelectStmt) {
+                  if (((SelectStmt) outer).getSelectList().isDistinct()
+                      && operand.getSetOperator() == SetOperator.INTERSECT) {
+                    joinOp = JoinOperator.INNER_JOIN;
+                    TableRef firstTbl = eiSelect.getTableRefs().get(0);
+                    // Make sure only the leftmost view's tuples are visible
+                    eiSelect.getSelectList().getItems().set(0, SelectListItem
+                        .createStarItem(Lists.newArrayList(firstTbl.getUniqueAlias())));
+                  }
+                }
+              }
+            }
+          }
+          List<String> colLabels = new ArrayList<>();
+          for (int j = 0; j < operand.getQueryStmt().getColLabels().size(); ++j) {
+            colLabels.add(eiSelect.getColumnAliasGenerator().getNextAlias());
+          }
+          // Wraps the query statement for the current operand.
+          InlineViewRef opWrapperView = new InlineViewRef(
+              tableAliasGenerator.getNextAlias(), operand.getQueryStmt(), colLabels);
+          opWrapperView.setLeftTblRef(rightMostTbl);
+          opWrapperView.setJoinOp(joinOp);
+          opWrapperView.setOnClause(
+              getSetOpJoinPredicates((InlineViewRef) eiSelect.getTableRefs().get(0),
+                  opWrapperView, operand.getSetOperator()));
+          opWrapperView.analyze(analyzer);
+          eiSelect.getTableRefs().add(opWrapperView);
+          break;
+
+        case UNION:
+          // Create a new SelectStmt for unions.
+          if (uSelect == null) {
+            unionStmt = null;
+            SelectList sl =
+                new SelectList(Lists.newArrayList(SelectListItem.createStarItem(null)));
+            uSelect = new SelectStmt(sl, null, null, null, null, null, null);
+            SetOperationStmt.SetOperand eiOperand = null;
+            if (eiSelect != null) {
+              eiOperand = new SetOperationStmt.SetOperand(eiSelect, null, null);
+              eiSelect = null;
+            }
+            List<SetOperationStmt.SetOperand> initialOps = new ArrayList<>();
+            if (i == 1) {
+              initialOps.add(firstOperand);
+              firstOperand = null;
+            }
+            if (eiOperand != null) {
+              initialOps.add(eiOperand);
+            }
+            unionStmt = new UnionStmt(initialOps, null, null);
+            uSelect.getTableRefs().add(new InlineViewRef(
+                tableAliasGenerator.getNextAlias(), unionStmt, (TableSampleClause) null));
+          }
+          operand.reset();
+          unionStmt.getOperands().add(operand);
+          break;
+
+        default:
+          throw new AnalysisException("Unknown Set Operation Statement Operator Type");
+      }
+    }
+
+    final SelectStmt newStmt = uSelect != null ? uSelect : eiSelect;
+    Preconditions.checkNotNull(newStmt);
+
+    newStmt.limitElement_ = stmt.limitElement_;
+    newStmt.limitElement_.reset();
+    if (stmt.hasOrderByClause()) {
+      newStmt.orderByElements_ = stmt.cloneOrderByElements();
+      if (newStmt.orderByElements_ != null) {
+        for (OrderByElement o : newStmt.orderByElements_) o.getExpr().reset();
+      }
+    }
+
+    newStmt.analyze(analyzer);
+    stmt.rewrittenStmt_ = newStmt;
   }
 
   protected void rewriteSelectStmtHook(SelectStmt stmt, Analyzer analyzer)
diff --git a/fe/src/main/java/org/apache/impala/analysis/UnionStmt.java b/fe/src/main/java/org/apache/impala/analysis/UnionStmt.java
index bf4c0bf..12a1e99 100644
--- a/fe/src/main/java/org/apache/impala/analysis/UnionStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/UnionStmt.java
@@ -17,642 +17,60 @@
 
 package org.apache.impala.analysis;
 
-import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
-
-import org.apache.impala.catalog.ColumnStats;
-import org.apache.impala.catalog.FeView;
 import org.apache.impala.common.AnalysisException;
-import org.apache.impala.rewrite.ExprRewriter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Preconditions;
 
 /**
- * Representation of a union with its list of operands, and optional order by and limit.
- * A union materializes its results, and its resultExprs are SlotRefs into a new
- * materialized tuple.
- * During analysis, the operands are normalized (separated into a single sequence of
- * DISTINCT followed by a single sequence of ALL operands) and unnested to the extent
- * possible. This also creates the AggregationInfo for DISTINCT operands.
- *
- * Use of resultExprs vs. baseTblResultExprs:
- * We consistently use/cast the resultExprs of union operands because the final expr
- * substitution happens during planning. The only place where baseTblResultExprs are
- * used is in materializeRequiredSlots() because that is called before plan generation
- * and we need to mark the slots of resolved exprs as materialized.
+ * Representation of a union.
  */
-public class UnionStmt extends QueryStmt {
-  private final static Logger LOG = LoggerFactory.getLogger(UnionStmt.class);
-
-  public static enum Qualifier {
-    ALL,
-    DISTINCT
-  }
-
-  /**
-   * Represents an operand to a union. It consists of a query statement and its left
-   * all/distinct qualifier (null for the first operand).
-   */
-  public static class UnionOperand {
-    // Effective qualifier. Should not be reset() to preserve changes made during
-    // distinct propagation and unnesting that are needed after rewriting Subqueries.
-    private Qualifier qualifier_;
-
-    /////////////////////////////////////////
-    // BEGIN: Members that need to be reset()
-
-    private final QueryStmt queryStmt_;
-
-    // Analyzer used for this operand. Set in analyze().
-    // We must preserve the conjuncts registered in the analyzer for partition pruning.
-    private Analyzer analyzer_;
-
-    // Map from UnionStmt's result slots to our resultExprs. Used during plan generation.
-    private final ExprSubstitutionMap smap_;
-
-    // END: Members that need to be reset()
-    /////////////////////////////////////////
-
-    public UnionOperand(QueryStmt queryStmt, Qualifier qualifier) {
-      queryStmt_ = queryStmt;
-      qualifier_ = qualifier;
-      smap_ = new ExprSubstitutionMap();
-    }
-
-    public void analyze(Analyzer parent) throws AnalysisException {
-      if (isAnalyzed()) return;
-      analyzer_ = new Analyzer(parent);
-      queryStmt_.analyze(analyzer_);
-    }
-
-    public boolean isAnalyzed() { return analyzer_ != null; }
-    public QueryStmt getQueryStmt() { return queryStmt_; }
-    public Qualifier getQualifier() { return qualifier_; }
-    // Used for propagating DISTINCT.
-    public void setQualifier(Qualifier qualifier) { qualifier_ = qualifier; }
-    public Analyzer getAnalyzer() { return analyzer_; }
-    public ExprSubstitutionMap getSmap() { return smap_; }
-
-    public boolean hasAnalyticExprs() {
-      if (queryStmt_ instanceof SelectStmt) {
-        return ((SelectStmt) queryStmt_).hasAnalyticInfo();
-      } else {
-        Preconditions.checkState(queryStmt_ instanceof UnionStmt);
-        return ((UnionStmt) queryStmt_).hasAnalyticExprs();
-      }
-    }
-
-    /**
-     * C'tor for cloning.
-     */
-    private UnionOperand(UnionOperand other) {
-      queryStmt_ = other.queryStmt_.clone();
-      qualifier_ = other.qualifier_;
-      analyzer_ = other.analyzer_;
-      smap_ = other.smap_.clone();
-    }
-
-    public void reset() {
-      queryStmt_.reset();
-      analyzer_ = null;
-      smap_.clear();
-    }
-
-    @Override
-    public UnionOperand clone() { return new UnionOperand(this); }
-  }
-
+public class UnionStmt extends SetOperationStmt {
   /////////////////////////////////////////
   // BEGIN: Members that need to be reset()
 
-  // before analysis, this contains the list of union operands derived verbatim
-  // from the query;
-  // after analysis, this contains all of distinctOperands followed by allOperands
-  protected final List<UnionOperand> operands_;
-
-  // filled during analyze(); contains all operands that need to go through
-  // distinct aggregation
-  protected final List<UnionOperand> distinctOperands_ = new ArrayList<>();
-
-  // filled during analyze(); contains all operands that can be aggregated with
-  // a simple merge without duplicate elimination (also needs to merge the output
-  // of the DISTINCT operands)
-  protected final List<UnionOperand> allOperands_ = new ArrayList<>();
-
-  protected MultiAggregateInfo distinctAggInfo_; // only set if we have DISTINCT ops
-
-  // Single tuple materialized by the union. Set in analyze().
-  protected TupleId tupleId_;
-
-  // set prior to unnesting
-  protected String toSqlString_ = null;
-
-  // true if any of the operands_ references an AnalyticExpr
-  private boolean hasAnalyticExprs_ = false;
-
-  // List of output expressions produced by the union without the ORDER BY portion
-  // (if any). Same as resultExprs_ if there is no ORDER BY.
-  private List<Expr> unionResultExprs_ = new ArrayList<>();
-
-  // List of expressions produced by analyzer.castToUnionCompatibleTypes().
-  // Contains a list of exprs such that for every i-th expr in that list, it is the first
-  // widest compatible expression encountered among all i-th exprs in every result expr
-  // list of the union operands.
-  protected List<Expr> widestExprs_ = new ArrayList<>();
 
   // END: Members that need to be reset()
   /////////////////////////////////////////
 
-  public UnionStmt(List<UnionOperand> operands,
+  public UnionStmt(List<SetOperand> operands,
       List<OrderByElement> orderByElements, LimitElement limitElement) {
-    super(orderByElements, limitElement);
-    Preconditions.checkNotNull(operands);
-    Preconditions.checkState(operands.size() > 0);
-    operands_ = operands;
+    super(operands, orderByElements, limitElement);
   }
 
   /**
    * C'tor for cloning.
    */
-  protected UnionStmt(UnionStmt other) {
-    super(other.cloneOrderByElements(),
-        (other.limitElement_ == null) ? null : other.limitElement_.clone());
-    operands_ = new ArrayList<>();
-    if (analyzer_ != null) {
-      for (UnionOperand o: other.distinctOperands_) distinctOperands_.add(o.clone());
-      for (UnionOperand o: other.allOperands_) allOperands_.add(o.clone());
-      operands_.addAll(distinctOperands_);
-      operands_.addAll(allOperands_);
-    } else {
-      for (UnionOperand operand: other.operands_) operands_.add(operand.clone());
-    }
-    analyzer_ = other.analyzer_;
-    distinctAggInfo_ =
-        (other.distinctAggInfo_ != null) ? other.distinctAggInfo_.clone() : null;
-    tupleId_ = other.tupleId_;
-    toSqlString_ = (other.toSqlString_ != null) ? new String(other.toSqlString_) : null;
-    hasAnalyticExprs_ = other.hasAnalyticExprs_;
-    withClause_ = (other.withClause_ != null) ? other.withClause_.clone() : null;
-    unionResultExprs_ = Expr.cloneList(other.unionResultExprs_);
-    widestExprs_ = other.widestExprs_;
-  }
-
-  public List<UnionOperand> getOperands() { return operands_; }
-  public List<UnionOperand> getDistinctOperands() { return distinctOperands_; }
-  public boolean hasDistinctOps() { return !distinctOperands_.isEmpty(); }
-  public List<UnionOperand> getAllOperands() { return allOperands_; }
-  public boolean hasAllOps() { return !allOperands_.isEmpty(); }
-  public MultiAggregateInfo getDistinctAggInfo() { return distinctAggInfo_; }
-  public boolean hasAnalyticExprs() { return hasAnalyticExprs_; }
-  public TupleId getTupleId() { return tupleId_; }
-
-  public void removeAllOperands() {
-    operands_.removeAll(allOperands_);
-    allOperands_.clear();
-  }
-
-  @Override
-  public void setDoTableMasking(boolean doTableMasking) {
-    for (UnionOperand op : operands_) {
-      op.getQueryStmt().setDoTableMasking(doTableMasking);
-    }
+  protected UnionStmt(SetOperationStmt other) {
+    super(other);
   }
 
   @Override
   public void analyze(Analyzer analyzer) throws AnalysisException {
     if (isAnalyzed()) return;
     super.analyze(analyzer);
-
-    // Propagates DISTINCT from right to left.
-    propagateDistinct();
-
-    // Analyze all operands and make sure they return an equal number of exprs.
-    analyzeOperands(analyzer);
-
-    // Remember the SQL string before unnesting operands.
-    toSqlString_ = toSql();
-    if (origSqlString_ == null) origSqlString_ = toSqlString_;
-
-    // Unnest the operands before casting the result exprs. Unnesting may add
-    // additional entries to operands_ and the result exprs of those unnested
-    // operands must also be cast properly.
-    unnestOperands(analyzer);
-
-    // Compute hasAnalyticExprs_
-    hasAnalyticExprs_ = false;
-    for (UnionOperand op: operands_) {
-      if (op.hasAnalyticExprs()) {
-        hasAnalyticExprs_ = true;
-        break;
-      }
-    }
-
-    // Collect all result expr lists and cast the exprs as necessary.
-    List<List<Expr>> resultExprLists = new ArrayList<>();
-    for (UnionOperand op: operands_) {
-      resultExprLists.add(op.getQueryStmt().getResultExprs());
-    }
-    widestExprs_ = analyzer.castToUnionCompatibleTypes(resultExprLists);
-
-    // Create tuple descriptor materialized by this UnionStmt, its resultExprs, and
-    // its sortInfo if necessary.
-    createMetadata(analyzer);
-    createSortInfo(analyzer);
-
-    // Create unnested operands' smaps.
-    for (UnionOperand operand: operands_) setOperandSmap(operand, analyzer);
-
-    // Create distinctAggInfo, if necessary.
-    if (!distinctOperands_.isEmpty()) {
-      // Aggregate produces exactly the same tuple as the original union stmt.
-      List<Expr> groupingExprs = Expr.cloneList(resultExprs_);
-      try {
-        distinctAggInfo_ = MultiAggregateInfo.createDistinct(
-            groupingExprs, analyzer.getTupleDesc(tupleId_), analyzer);
-      } catch (AnalysisException e) {
-        // Should never happen.
-        throw new IllegalStateException(
-            "Error creating agg info in UnionStmt.analyze()", e);
-      }
-    }
-
-    unionResultExprs_ = Expr.cloneList(resultExprs_);
-    if (evaluateOrderBy_) createSortTupleInfo(analyzer);
-    baseTblResultExprs_ = resultExprs_;
-  }
-
-  /**
-   * Analyzes all operands and checks that they return an equal number of exprs.
-   * Throws an AnalysisException if that is not the case, or if analyzing
-   * an operand fails.
-   */
-  private void analyzeOperands(Analyzer analyzer) throws AnalysisException {
-    for (int i = 0; i < operands_.size(); ++i) {
-      operands_.get(i).analyze(analyzer);
-      QueryStmt firstQuery = operands_.get(0).getQueryStmt();
-      List<Expr> firstExprs = operands_.get(0).getQueryStmt().getResultExprs();
-      QueryStmt query = operands_.get(i).getQueryStmt();
-      List<Expr> exprs = query.getResultExprs();
-      if (firstExprs.size() != exprs.size()) {
-        throw new AnalysisException("Operands have unequal number of columns:\n" +
-            "'" + queryStmtToSql(firstQuery) + "' has " +
-            firstExprs.size() + " column(s)\n" +
-            "'" + queryStmtToSql(query) + "' has " + exprs.size() + " column(s)");
-      }
-    }
-  }
-
-  /**
-   * Marks the baseTblResultExprs of its operands as materialized, based on
-   * which of the output slots have been marked.
-   * Calls materializeRequiredSlots() on the operands themselves.
-   */
-  @Override
-  public void materializeRequiredSlots(Analyzer analyzer) {
-    TupleDescriptor tupleDesc = analyzer.getDescTbl().getTupleDesc(tupleId_);
-    // to keep things simple we materialize all grouping exprs = output slots,
-    // regardless of what's being referenced externally
-    if (!distinctOperands_.isEmpty()) tupleDesc.materializeSlots();
-
-    if (evaluateOrderBy_) sortInfo_.materializeRequiredSlots(analyzer, null);
-
-    // collect operands' result exprs
-    List<SlotDescriptor> outputSlots = tupleDesc.getSlots();
-    List<Expr> exprs = new ArrayList<>();
-    for (int i = 0; i < outputSlots.size(); ++i) {
-      SlotDescriptor slotDesc = outputSlots.get(i);
-      if (!slotDesc.isMaterialized()) continue;
-      for (UnionOperand op: operands_) {
-        exprs.add(op.getQueryStmt().getBaseTblResultExprs().get(i));
-      }
-    }
-    if (distinctAggInfo_ != null) {
-      distinctAggInfo_.materializeRequiredSlots(analyzer, null);
-    }
-    materializeSlots(analyzer, exprs);
-
-    for (UnionOperand op: operands_) {
-      op.getQueryStmt().materializeRequiredSlots(analyzer);
-    }
-  }
-
-  /**
-   * Fill distinct-/allOperands and performs possible unnesting of UnionStmt
-   * operands in the process.
-   */
-  private void unnestOperands(Analyzer analyzer) throws AnalysisException {
-    if (operands_.size() == 1) {
-      // ValuesStmt for a single row.
-      allOperands_.add(operands_.get(0));
-      return;
-    }
-
-    // find index of first ALL operand
-    int firstUnionAllIdx = operands_.size();
-    for (int i = 1; i < operands_.size(); ++i) {
-      UnionOperand operand = operands_.get(i);
-      if (operand.getQualifier() == Qualifier.ALL) {
-        firstUnionAllIdx = (i == 1 ? 0 : i);
-        break;
-      }
-    }
-    // operands[0] is always implicitly ALL, so operands[1] can't be the
-    // first one
-    Preconditions.checkState(firstUnionAllIdx != 1);
-
-    // unnest DISTINCT operands
-    Preconditions.checkState(distinctOperands_.isEmpty());
-    for (int i = 0; i < firstUnionAllIdx; ++i) {
-      unnestOperand(distinctOperands_, Qualifier.DISTINCT, operands_.get(i));
-    }
-
-    // unnest ALL operands
-    Preconditions.checkState(allOperands_.isEmpty());
-    for (int i = firstUnionAllIdx; i < operands_.size(); ++i) {
-      unnestOperand(allOperands_, Qualifier.ALL, operands_.get(i));
-    }
-
-    for (UnionOperand op: distinctOperands_) op.setQualifier(Qualifier.DISTINCT);
-    for (UnionOperand op: allOperands_) op.setQualifier(Qualifier.ALL);
-
-    operands_.clear();
-    operands_.addAll(distinctOperands_);
-    operands_.addAll(allOperands_);
-  }
-
-  /**
-   * Sets the smap for the given operand. It maps from the output slots this union's
-   * tuple to the corresponding result exprs of the operand.
-   */
-  private void setOperandSmap(UnionOperand operand, Analyzer analyzer) {
-    TupleDescriptor tupleDesc = analyzer.getDescTbl().getTupleDesc(tupleId_);
-    // operands' smaps were already set in the operands' analyze()
-    operand.getSmap().clear();
-    List<Expr> resultExprs = operand.getQueryStmt().getResultExprs();
-    Preconditions.checkState(resultExprs.size() == tupleDesc.getSlots().size());
-    for (int i = 0; i < tupleDesc.getSlots().size(); ++i) {
-      SlotDescriptor outputSlot = tupleDesc.getSlots().get(i);
-      // Map to the original (uncast) result expr of the operand.
-      Expr origExpr = resultExprs.get(i).unwrapExpr(true).clone();
-      operand.getSmap().put(new SlotRef(outputSlot), origExpr);
-    }
-  }
-
-  /**
-   * Add a single operand to the target list; if the operand itself is a UnionStmt, apply
-   * unnesting to the extent possible (possibly modifying 'operand' in the process).
-   */
-  private void unnestOperand(
-      List<UnionOperand> target, Qualifier targetQualifier, UnionOperand operand) {
-    Preconditions.checkState(operand.isAnalyzed());
-    QueryStmt queryStmt = operand.getQueryStmt();
-    if (queryStmt instanceof SelectStmt) {
-      target.add(operand);
-      return;
-    }
-
-    Preconditions.checkState(queryStmt instanceof UnionStmt);
-    UnionStmt unionStmt = (UnionStmt) queryStmt;
-    if (unionStmt.hasLimit() || unionStmt.hasOffset()) {
-      // we must preserve the nested Union
-      target.add(operand);
-    } else if (targetQualifier == Qualifier.DISTINCT || !unionStmt.hasDistinctOps()) {
-      // there is no limit in the nested Union and we can absorb all of its
-      // operands as-is
-      target.addAll(unionStmt.getDistinctOperands());
-      target.addAll(unionStmt.getAllOperands());
-    } else {
-      // the nested Union contains some Distinct ops and we're accumulating
-      // into our All ops; unnest only the All ops and leave the rest in place
-      target.addAll(unionStmt.getAllOperands());
-      unionStmt.removeAllOperands();
-      target.add(operand);
-    }
-  }
-
-  /**
-   * String representation of queryStmt used in reporting errors.
-   * Allow subclasses to override this.
-   */
-  protected String queryStmtToSql(QueryStmt queryStmt) {
-    return queryStmt.toSql();
-  }
-
-  /**
-   * Propagates DISTINCT (if present) from right to left.
-   * Implied associativity:
-   * A UNION ALL B UNION DISTINCT C = (A UNION ALL B) UNION DISTINCT C
-   * = A UNION DISTINCT B UNION DISTINCT C
-   */
-  private void propagateDistinct() {
-    int lastDistinctPos = -1;
-    for (int i = operands_.size() - 1; i > 0; --i) {
-      UnionOperand operand = operands_.get(i);
-      if (lastDistinctPos != -1) {
-        // There is a DISTINCT somewhere to the right.
-        operand.setQualifier(Qualifier.DISTINCT);
-      } else if (operand.getQualifier() == Qualifier.DISTINCT) {
-        lastDistinctPos = i;
-      }
-    }
-  }
-
-  /**
-   * Create a descriptor for the tuple materialized by the union.
-   * Set resultExprs to be slot refs into that tuple.
-   * Also fills the substitution map, such that "order by" can properly resolve
-   * column references from the result of the union.
-   */
-  private void createMetadata(Analyzer analyzer) throws AnalysisException {
-    // Create tuple descriptor for materialized tuple created by the union.
-    TupleDescriptor tupleDesc = analyzer.getDescTbl().createTupleDescriptor("union");
-    tupleDesc.setIsMaterialized(true);
-    tupleId_ = tupleDesc.getId();
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("UnionStmt.createMetadata: tupleId=" + tupleId_.toString());
-    }
-
-    // One slot per expr in the select blocks. Use first select block as representative.
-    List<Expr> firstSelectExprs = operands_.get(0).getQueryStmt().getResultExprs();
-
-    // Compute column stats for the materialized slots from the source exprs.
-    List<ColumnStats> columnStats = new ArrayList<>();
-    for (int i = 0; i < operands_.size(); ++i) {
-      List<Expr> selectExprs = operands_.get(i).getQueryStmt().getResultExprs();
-      for (int j = 0; j < selectExprs.size(); ++j) {
-        ColumnStats statsToAdd = ColumnStats.fromExpr(selectExprs.get(j));
-        if (i == 0) {
-          columnStats.add(statsToAdd);
-        } else {
-          columnStats.get(j).add(statsToAdd);
-        }
-      }
-    }
-
-    // Create tuple descriptor and slots.
-    for (int i = 0; i < firstSelectExprs.size(); ++i) {
-      Expr expr = firstSelectExprs.get(i);
-      SlotDescriptor slotDesc = analyzer.addSlotDescriptor(tupleDesc);
-      slotDesc.setLabel(getColLabels().get(i));
-      slotDesc.setType(expr.getType());
-      slotDesc.setStats(columnStats.get(i));
-      SlotRef outputSlotRef = new SlotRef(slotDesc);
-      resultExprs_.add(outputSlotRef);
-
-      // Add to aliasSMap so that column refs in "order by" can be resolved.
-      if (orderByElements_ != null) {
-        SlotRef aliasRef = new SlotRef(getColLabels().get(i));
-        if (aliasSmap_.containsMappingFor(aliasRef)) {
-          ambiguousAliasList_.add(aliasRef);
-        } else {
-          aliasSmap_.put(aliasRef, outputSlotRef);
-        }
-      }
-
-      boolean isNullable = false;
-      // register single-directional value transfers from output slot
-      // to operands' result exprs (if those happen to be slotrefs);
-      // don't do that if the operand computes analytic exprs
-      // (see Planner.createInlineViewPlan() for the reasoning)
-      for (UnionOperand op: operands_) {
-        Expr resultExpr = op.getQueryStmt().getResultExprs().get(i);
-        slotDesc.addSourceExpr(resultExpr);
-        SlotRef slotRef = resultExpr.unwrapSlotRef(false);
-        if (slotRef == null || slotRef.getDesc().getIsNullable()) isNullable = true;
-        if (op.hasAnalyticExprs()) continue;
-        slotRef = resultExpr.unwrapSlotRef(true);
-        if (slotRef == null) continue;
-        analyzer.registerValueTransfer(outputSlotRef.getSlotId(), slotRef.getSlotId());
-      }
-      // If all the child slots are not nullable, then the union output slot should not
-      // be nullable as well.
-      slotDesc.setIsNullable(isNullable);
-    }
-    baseTblResultExprs_ = resultExprs_;
-  }
-
-  @Override
-  public void rewriteExprs(ExprRewriter rewriter) throws AnalysisException {
-    for (UnionOperand op: operands_) op.getQueryStmt().rewriteExprs(rewriter);
-    if (orderByElements_ != null) {
-      for (OrderByElement orderByElem: orderByElements_) {
-        orderByElem.setExpr(rewriter.rewrite(orderByElem.getExpr(), analyzer_));
-      }
-    }
-  }
-
-  @Override
-  public void getMaterializedTupleIds(List<TupleId> tupleIdList) {
-    // Return the sort tuple if there is an evaluated order by.
-    if (evaluateOrderBy_) {
-      tupleIdList.add(sortInfo_.getSortTupleDescriptor().getId());
-    } else {
-      tupleIdList.add(tupleId_);
-    }
-  }
-
-  @Override
-  public void collectTableRefs(List<TableRef> tblRefs, boolean fromClauseOnly) {
-    super.collectTableRefs(tblRefs, fromClauseOnly);
-    for (UnionOperand op: operands_) {
-      op.getQueryStmt().collectTableRefs(tblRefs, fromClauseOnly);
-    }
-  }
-
-  @Override
-  public void collectInlineViews(Set<FeView> inlineViews) {
-    super.collectInlineViews(inlineViews);
-    for (UnionOperand operand : operands_) {
-      operand.getQueryStmt().collectInlineViews(inlineViews);
-    }
-  }
-
-  @Override
-  public String toSql(ToSqlOptions options) {
-    if (!options.showRewritten() && toSqlString_ != null) return toSqlString_;
-
-    StringBuilder strBuilder = new StringBuilder();
-    Preconditions.checkState(operands_.size() > 0);
-
-    if (withClause_ != null) {
-      strBuilder.append(withClause_.toSql(options));
-      strBuilder.append(" ");
-    }
-
-    strBuilder.append(operands_.get(0).getQueryStmt().toSql(options));
-    for (int i = 1; i < operands_.size() - 1; ++i) {
-      strBuilder.append(" UNION " +
-          ((operands_.get(i).getQualifier() == Qualifier.ALL) ? "ALL " : ""));
-      if (operands_.get(i).getQueryStmt() instanceof UnionStmt) {
-        strBuilder.append("(");
-      }
-      strBuilder.append(operands_.get(i).getQueryStmt().toSql(options));
-      if (operands_.get(i).getQueryStmt() instanceof UnionStmt) {
-        strBuilder.append(")");
-      }
-    }
-    // Determine whether we need parenthesis around the last union operand.
-    UnionOperand lastOperand = operands_.get(operands_.size() - 1);
-    QueryStmt lastQueryStmt = lastOperand.getQueryStmt();
-    strBuilder.append(" UNION " +
-        ((lastOperand.getQualifier() == Qualifier.ALL) ? "ALL " : ""));
-    if (lastQueryStmt instanceof UnionStmt ||
-        ((hasOrderByClause() || hasLimit() || hasOffset()) &&
-            !lastQueryStmt.hasLimit() && !lastQueryStmt.hasOffset() &&
-            !lastQueryStmt.hasOrderByClause())) {
-      strBuilder.append("(");
-      strBuilder.append(lastQueryStmt.toSql(options));
-      strBuilder.append(")");
-    } else {
-      strBuilder.append(lastQueryStmt.toSql(options));
-    }
-    // Order By clause
-    if (hasOrderByClause()) {
-      strBuilder.append(" ORDER BY ");
-      for (int i = 0; i < orderByElements_.size(); ++i) {
-        strBuilder.append(orderByElements_.get(i).toSql(options));
-        strBuilder.append((i+1 != orderByElements_.size()) ? ", " : "");
-      }
-    }
-    // Limit clause.
-    strBuilder.append(limitElement_.toSql(options));
-    return strBuilder.toString();
   }
 
   @Override
-  public List<String> getColLabels() {
-    Preconditions.checkState(operands_.size() > 0);
-    return operands_.get(0).getQueryStmt().getColLabels();
+  public SetOperationStmt clone() {
+    return new UnionStmt(this);
   }
 
-  public List<Expr> getUnionResultExprs() { return unionResultExprs_; }
-
-  public List<Expr> getWidestExprs() { return widestExprs_; }
-
-  @Override
-  public UnionStmt clone() { return new UnionStmt(this); }
-
   /**
    * Undoes all changes made by analyze() except distinct propagation and unnesting.
    * After analysis, operands_ contains the list of unnested operands with qualifiers
    * adjusted to reflect distinct propagation. Every operand in that list is reset().
-   * The distinctOperands_ and allOperands_ are cleared because they are redundant
+   * The unionDistinctOperands_ and unionAllOperands_ are cleared because they are
+   * redundant
    * with operands_.
    */
   @Override
   public void reset() {
     super.reset();
-    for (UnionOperand op: operands_) op.reset();
-    distinctOperands_.clear();
-    allOperands_.clear();
+    for (SetOperand op : operands_) op.reset();
+    unionDistinctOperands_.clear();
+    unionAllOperands_.clear();
     distinctAggInfo_ = null;
     tupleId_ = null;
     toSqlString_ = null;
-    hasAnalyticExprs_ = false;
-    unionResultExprs_.clear();
     widestExprs_ = null;
   }
 }
diff --git a/fe/src/main/java/org/apache/impala/analysis/ValuesStmt.java b/fe/src/main/java/org/apache/impala/analysis/ValuesStmt.java
index 06a1c67..642cac7 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ValuesStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ValuesStmt.java
@@ -31,9 +31,8 @@ import static org.apache.impala.analysis.ToSqlOptions.DEFAULT;
  * - No nesting of ValuesStmts
  */
 public class ValuesStmt extends UnionStmt {
-
-  public ValuesStmt(List<UnionOperand> operands,
-      List<OrderByElement> orderByElements, LimitElement limitElement) {
+  public ValuesStmt(List<SetOperand> operands, List<OrderByElement> orderByElements,
+      LimitElement limitElement) {
     super(operands, orderByElements, limitElement);
   }
 
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
index e816edd..b46518c 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanFragment.java
@@ -234,7 +234,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
 
       // Cast partition exprs of all hash-partitioning senders to their compatible types.
       try {
-        analyzer.castToUnionCompatibleTypes(senderPartitionExprs);
+        analyzer.castToSetOpCompatibleTypes(senderPartitionExprs);
       } catch (AnalysisException e) {
         // Should never happen. Analysis should have ensured type compatibility already.
         throw new IllegalStateException(e);
diff --git a/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java b/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
index 09d86f9..b1401cf 100644
--- a/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
@@ -47,6 +47,7 @@ import org.apache.impala.analysis.Path;
 import org.apache.impala.analysis.Path.PathType;
 import org.apache.impala.analysis.QueryStmt;
 import org.apache.impala.analysis.SelectStmt;
+import org.apache.impala.analysis.SetOperationStmt.SetOperand;
 import org.apache.impala.analysis.SingularRowSrcTableRef;
 import org.apache.impala.analysis.SlotDescriptor;
 import org.apache.impala.analysis.SlotId;
@@ -57,7 +58,6 @@ import org.apache.impala.analysis.TupleDescriptor;
 import org.apache.impala.analysis.TupleId;
 import org.apache.impala.analysis.TupleIsNullPredicate;
 import org.apache.impala.analysis.UnionStmt;
-import org.apache.impala.analysis.UnionStmt.UnionOperand;
 import org.apache.impala.catalog.Column;
 import org.apache.impala.catalog.ColumnStats;
 import org.apache.impala.catalog.FeDataSourceTable;
@@ -120,7 +120,7 @@ public class SingleNodePlanner {
    * performs the following actions.
    * In the top-down phase over query statements:
    * - Materialize the slots required for evaluating expressions of that statement.
-   * - Migrate conjuncts from parent blocks into inline views and union operands.
+   * - Migrate conjuncts from parent blocks into inline views and set operands.
    * In the bottom-up phase generate the plan tree for every query statement:
    * - Generate the plan for the FROM-clause of a select statement: The plan trees of
    *   absolute and uncorrelated table refs are connected via JoinNodes. The relative
@@ -237,7 +237,7 @@ public class SingleNodePlanner {
     EmptySetNode node = new EmptySetNode(ctx_.getNextNodeId(), tupleIds);
     node.init(analyzer);
     // Set the output smap to resolve exprs referencing inline views within stmt.
-    // Not needed for a UnionStmt because it materializes its input operands.
+    // Not needed for a SetOperationStmt because it materializes its input operands.
     if (stmt instanceof SelectStmt) {
       node.setOutputSmap(((SelectStmt) stmt).getBaseTblSmap());
     }
@@ -1988,13 +1988,11 @@ public class SingleNodePlanner {
    * distinct portion of the given unionStmt. The unionDistinctPlan is then added
    * as a child of the returned UnionNode.
    */
-  private UnionNode createUnionPlan(
-      Analyzer analyzer, UnionStmt unionStmt, List<UnionOperand> unionOperands,
-      PlanNode unionDistinctPlan)
-      throws ImpalaException {
+  private UnionNode createUnionPlan(Analyzer analyzer, UnionStmt unionStmt,
+      List<SetOperand> unionOperands, PlanNode unionDistinctPlan) throws ImpalaException {
     UnionNode unionNode = new UnionNode(ctx_.getNextNodeId(), unionStmt.getTupleId(),
-        unionStmt.getUnionResultExprs(), ctx_.hasSubplan());
-    for (UnionOperand op: unionOperands) {
+        unionStmt.getSetOperationResultExprs(), ctx_.hasSubplan());
+    for (SetOperand op : unionOperands) {
       if (op.getAnalyzer().hasEmptyResultSet()) {
         unmarkCollectionSlots(op.getQueryStmt());
         continue;
@@ -2016,7 +2014,7 @@ public class SingleNodePlanner {
     }
 
     if (unionDistinctPlan != null) {
-      Preconditions.checkState(unionStmt.hasDistinctOps());
+      Preconditions.checkState(unionStmt.hasUnionDistinctOps());
       Preconditions.checkState(unionDistinctPlan instanceof AggregationNode);
       unionNode.addChild(unionDistinctPlan,
           unionStmt.getDistinctAggInfo().getGroupingExprs());
@@ -2050,7 +2048,7 @@ public class SingleNodePlanner {
       // the individual operands.
       // Do this prior to creating the operands' plan trees so they get a chance to
       // pick up propagated predicates.
-      for (UnionOperand op: unionStmt.getOperands()) {
+      for (SetOperand op : unionStmt.getOperands()) {
         List<Expr> opConjuncts =
             Expr.substituteList(conjuncts, op.getSmap(), analyzer, false);
         op.getAnalyzer().registerConjuncts(opConjuncts);
@@ -2065,16 +2063,17 @@ public class SingleNodePlanner {
 
     PlanNode result = null;
     // create DISTINCT tree
-    if (unionStmt.hasDistinctOps()) {
+    if (unionStmt.hasUnionDistinctOps()) {
       result = createUnionPlan(
-          analyzer, unionStmt, unionStmt.getDistinctOperands(), null);
+          analyzer, unionStmt, unionStmt.getUnionDistinctOperands(), null);
       result = new AggregationNode(
           ctx_.getNextNodeId(), result, unionStmt.getDistinctAggInfo(), AggPhase.FIRST);
       result.init(analyzer);
     }
     // create ALL tree
-    if (unionStmt.hasAllOps()) {
-      result = createUnionPlan(analyzer, unionStmt, unionStmt.getAllOperands(), result);
+    if (unionStmt.hasUnionAllOps()) {
+      result =
+          createUnionPlan(analyzer, unionStmt, unionStmt.getUnionAllOperands(), result);
     }
 
     if (unionStmt.hasAnalyticExprs()) {
diff --git a/fe/src/main/java/org/apache/impala/planner/UnionNode.java b/fe/src/main/java/org/apache/impala/planner/UnionNode.java
index 0ec2457..72da0e6 100644
--- a/fe/src/main/java/org/apache/impala/planner/UnionNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/UnionNode.java
@@ -51,7 +51,7 @@ import com.google.common.base.Preconditions;
 public class UnionNode extends PlanNode {
   private final static Logger LOG = LoggerFactory.getLogger(UnionNode.class);
 
-  // List of union result exprs of the originating UnionStmt. Used for
+  // List of union result exprs of the originating SetOperationStmt. Used for
   // determining passthrough-compatibility of children.
   protected List<Expr> unionResultExprs_;
 
diff --git a/fe/src/main/jflex/sql-scanner.flex b/fe/src/main/jflex/sql-scanner.flex
index 7a7e3a5..16bb2a7 100644
--- a/fe/src/main/jflex/sql-scanner.flex
+++ b/fe/src/main/jflex/sql-scanner.flex
@@ -123,6 +123,7 @@ import org.apache.impala.thrift.TReservedWordsVersion;
     keywordMap.put("encoding", SqlParserSymbols.KW_ENCODING);
     keywordMap.put("end", SqlParserSymbols.KW_END);
     keywordMap.put("escaped", SqlParserSymbols.KW_ESCAPED);
+    keywordMap.put("except", SqlParserSymbols.KW_EXCEPT);
     keywordMap.put("exists", SqlParserSymbols.KW_EXISTS);
     keywordMap.put("explain", SqlParserSymbols.KW_EXPLAIN);
     keywordMap.put("extended", SqlParserSymbols.KW_EXTENDED);
@@ -162,6 +163,7 @@ import org.apache.impala.thrift.TReservedWordsVersion;
     keywordMap.put("int", SqlParserSymbols.KW_INT);
     keywordMap.put("integer", SqlParserSymbols.KW_INT);
     keywordMap.put("intermediate", SqlParserSymbols.KW_INTERMEDIATE);
+    keywordMap.put("intersect", SqlParserSymbols.KW_INTERSECT);
     keywordMap.put("interval", SqlParserSymbols.KW_INTERVAL);
     keywordMap.put("into", SqlParserSymbols.KW_INTO);
     keywordMap.put("invalidate", SqlParserSymbols.KW_INVALIDATE);
@@ -180,6 +182,7 @@ import org.apache.impala.thrift.TReservedWordsVersion;
     keywordMap.put("map", SqlParserSymbols.KW_MAP);
     keywordMap.put("merge_fn", SqlParserSymbols.KW_MERGE_FN);
     keywordMap.put("metadata", SqlParserSymbols.KW_METADATA);
+    keywordMap.put("minus", SqlParserSymbols.KW_MINUS);
     keywordMap.put("norely", SqlParserSymbols.KW_NORELY);
     keywordMap.put("not", SqlParserSymbols.KW_NOT);
     keywordMap.put("novalidate", SqlParserSymbols.KW_NOVALIDATE);
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
index 85d2e0c..73dae60 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
@@ -3009,6 +3009,195 @@ public class AnalyzeStmtsTest extends AnalyzerTest {
   }
 
   @Test
+  public void TestSetOperations() {
+    // Tests for SetOperationStmt analysis which includes intersect, except and union
+    AnalyzesOk("select rank() over (order by int_col) from functional.alltypes " +
+        "intersect select int_col from functional.alltypessmall");
+    // Selects on same table without aliases.
+    AnalyzesOk("select int_col from functional.alltypes intersect " +
+        "select int_col from functional.alltypes");
+    // Longer union chain.
+    AnalyzesOk("select int_col from functional.alltypes union " +
+        "select int_col from functional.alltypes " +
+        "intersect select int_col from functional.alltypes except " +
+        "select int_col from functional.alltypes");
+
+    // Nesting
+    AnalyzesOk("(select int_col from functional.alltypes " +
+        "intersect (select tinyint_col from functional.alltypessmall union " +
+                  " select tinyint_col from functional.alltypessmall) except " +
+                  " select tinyint_col from functional.alltypestiny where id = 1) " +
+        "union (select tinyint_col from functional.alltypessmall) ");
+
+    // All columns, perfectly compatible.
+    AnalyzesOk("select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, " +
+        "float_col, double_col, date_string_col, string_col, timestamp_col, year," +
+        "month from functional.alltypes union " +
+        "select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, " +
+        "float_col, double_col, date_string_col, string_col, timestamp_col, year," +
+        "month from functional.alltypes");
+    // Make sure table aliases aren't visible across union operands.
+    AnalyzesOk("select a.smallint_col from functional.alltypes a " +
+        "union select a.int_col from functional.alltypessmall a");
+    // All columns compatible with NULL.
+    AnalyzesOk("select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, " +
+        "float_col, double_col, date_string_col, string_col, timestamp_col, year," +
+        "month from functional.alltypes union " +
+        "select NULL, NULL, NULL, NULL, NULL, NULL, " +
+        "NULL, NULL, NULL, NULL, NULL, NULL," +
+        "NULL from functional.alltypes");
+
+    // No from clause. Has literals and NULLs. Requires implicit casts.
+    AnalyzesOk("select 1, 2, 3 " +
+        "union select NULL, NULL, NULL " +
+        "union select 1.0, NULL, 3 " +
+        "union select NULL, 10, NULL");
+    // Implicit casts on integer types.
+    AnalyzesOk("select tinyint_col from functional.alltypes " +
+        "union select smallint_col from functional.alltypes " +
+        "union select int_col from functional.alltypes " +
+        "union select bigint_col from functional.alltypes");
+    // Implicit casts on float types.
+    AnalyzesOk("select float_col from functional.alltypes union " +
+        "select double_col from functional.alltypes");
+    // Implicit casts on all numeric types with two columns from each select.
+    AnalyzesOk("select tinyint_col, double_col from functional.alltypes " +
+        "union select smallint_col, float_col from functional.alltypes " +
+        "union select int_col, bigint_col from functional.alltypes " +
+        "union select bigint_col, int_col from functional.alltypes " +
+        "union select float_col, smallint_col from functional.alltypes " +
+        "union select double_col, tinyint_col from functional.alltypes");
+
+    // With order by, offset and limit.
+    AnalyzesOk("(select int_col from functional.alltypes) " +
+        "union (select tinyint_col from functional.alltypessmall) " +
+        "order by int_col limit 1");
+    AnalyzesOk("(select int_col from functional.alltypes) " +
+        "union (select tinyint_col from functional.alltypessmall) " +
+        "order by int_col");
+    AnalyzesOk("(select int_col from functional.alltypes) " +
+        "union (select tinyint_col from functional.alltypessmall) " +
+        "order by int_col offset 5");
+    // Order by w/o limit is ignored in the union operand below.
+    AnalyzesOk("select int_col from functional.alltypes order by int_col " +
+        "union (select tinyint_col from functional.alltypessmall) ");
+    AnalysisError("select int_col from functional.alltypes order by int_col offset 5 " +
+        "union (select tinyint_col from functional.alltypessmall) ",
+        "Order-by with offset without limit not supported in nested queries");
+    AnalysisError("select int_col from functional.alltypes offset 5 " +
+        "union (select tinyint_col from functional.alltypessmall) ",
+        "OFFSET requires an ORDER BY clause: OFFSET 5");
+    // Order by w/o limit is ignored in the union operand below.
+    AnalyzesOk("select int_col from functional.alltypes " +
+        "union (select tinyint_col from functional.alltypessmall " +
+        "order by tinyint_col) ");
+    AnalysisError("select int_col from functional.alltypes " +
+        "union (select tinyint_col from functional.alltypessmall " +
+        "order by tinyint_col offset 5) ",
+        "Order-by with offset without limit not supported in nested queries");
+    AnalysisError("select int_col from functional.alltypes " +
+        "union (select tinyint_col from functional.alltypessmall offset 5) ",
+        "OFFSET requires an ORDER BY clause: OFFSET 5");
+    // Bigger order by.
+    AnalyzesOk("(select tinyint_col, double_col from functional.alltypes) " +
+        "union (select smallint_col, float_col from functional.alltypes) " +
+        "union (select int_col, bigint_col from functional.alltypes) " +
+        "union (select bigint_col, int_col from functional.alltypes) " +
+        "order by double_col, tinyint_col");
+    // Multiple union operands with valid order by clauses.
+    AnalyzesOk("select int_col from functional.alltypes order by int_col " +
+        "union select int_col from functional.alltypes order by int_col limit 10 " +
+        "union (select int_col from functional.alltypes " +
+        "order by int_col limit 10 offset 5) order by int_col offset 5");
+    // Bigger order by with ordinals.
+    AnalyzesOk("(select tinyint_col, double_col from functional.alltypes) " +
+        "union (select smallint_col, float_col from functional.alltypes) " +
+        "union (select int_col, bigint_col from functional.alltypes) " +
+        "union (select bigint_col, int_col from functional.alltypes) " +
+        "order by 2, 1");
+
+    // Unequal number of columns.
+    AnalysisError("select int_col from functional.alltypes " +
+        "union select int_col, float_col from functional.alltypes",
+        "Operands have unequal number of columns:\n" +
+        "'SELECT int_col FROM functional.alltypes' has 1 column(s)\n" +
+        "'SELECT int_col, float_col FROM functional.alltypes' has 2 column(s)");
+    // Unequal number of columns, longer union chain.
+    AnalysisError("select int_col from functional.alltypes " +
+        "union select tinyint_col from functional.alltypes " +
+        "union select smallint_col from functional.alltypes " +
+        "union select smallint_col, bigint_col from functional.alltypes",
+        "Operands have unequal number of columns:\n" +
+        "'SELECT int_col FROM functional.alltypes' has 1 column(s)\n" +
+        "'SELECT smallint_col, bigint_col FROM functional.alltypes' has 2 column(s)");
+    // Incompatible types.
+    AnalysisError("select bool_col from functional.alltypes " +
+        "union select lag(string_col) over(order by int_col) from functional.alltypes",
+        "Incompatible return types 'BOOLEAN' and 'STRING' of exprs " +
+        "'bool_col' and 'lag(string_col, 1, NULL)'.");
+    // Incompatible types, longer union chain.
+    AnalysisError("select int_col, string_col from functional.alltypes " +
+        "union select tinyint_col, bool_col from functional.alltypes " +
+        "union select smallint_col, int_col from functional.alltypes " +
+        "union select smallint_col, bool_col from functional.alltypes",
+        "Incompatible return types 'STRING' and 'BOOLEAN' of " +
+            "exprs 'string_col' and 'bool_col'.");
+    // Invalid ordinal in order by.
+    AnalysisError("(select int_col from functional.alltypes) " +
+        "union (select int_col from functional.alltypessmall) order by 2",
+        "ORDER BY: ordinal exceeds the number of items in the SELECT list: 2");
+    // Ambiguous order by.
+    AnalysisError("(select int_col a, string_col a from functional.alltypes) " +
+        "union (select int_col a, string_col a " +
+        "from functional.alltypessmall) order by a",
+        "ORDER BY: ambiguous alias: 'a'");
+    // Ambiguous alias in the second union operand should work.
+    AnalyzesOk("(select int_col a, string_col b from functional.alltypes) " +
+        "union (select int_col a, string_col a " +
+        "from functional.alltypessmall) order by a");
+    // Ambiguous alias even though the exprs of the first operand are identical
+    // (the corresponding in exprs in the other operand are different)
+    AnalysisError("select int_col a, int_col a from functional.alltypes " +
+        "union all (select 1, bigint_col from functional.alltypessmall) order by a",
+        "ORDER BY: ambiguous alias: 'a'");
+
+    // Column labels are inherited from first select block.
+    // Order by references an invalid column
+    AnalysisError("(select smallint_col from functional.alltypes) " +
+        "union (select int_col from functional.alltypessmall) order by int_col",
+        "Could not resolve column/field reference: 'int_col'");
+    // Make sure table aliases aren't visible across union operands.
+    AnalysisError("select a.smallint_col from functional.alltypes a " +
+        "union select a.int_col from functional.alltypessmall",
+        "Could not resolve column/field reference: 'a.int_col'");
+
+    // Regression test for IMPALA-1128, union of decimal and an int type that converts
+    // to the identical decimal.
+    AnalyzesOk("select cast(1 as bigint) union select cast(1 as decimal(19, 0))");
+
+    AnalysisContext decimalV1Ctx = createAnalysisCtx();
+    decimalV1Ctx.getQueryOptions().setDecimal_v2(false);
+    AnalysisContext decimalV2Ctx = createAnalysisCtx();
+    decimalV2Ctx.getQueryOptions().setDecimal_v2(true);
+
+    // IMPALA-6518: union of two incompatible decimal columns. There is no implicit cast
+    // if decimal_v2 is enabled.
+    String query = "select cast(123 as decimal(38, 0)) " +
+        "union all select cast(0.789 as decimal(38, 38))";
+    AnalyzesOk(query, decimalV1Ctx);
+    AnalysisError(query, decimalV2Ctx, "Incompatible return types 'DECIMAL(38,0)' and " +
+        "'DECIMAL(38,38)' of exprs 'CAST(123 AS DECIMAL(38,0))' and " +
+        "'CAST(0.789 AS DECIMAL(38,38))'.");
+
+    query = "select cast(123 as double) " +
+        "union all select cast(0.456 as float)" +
+        "union all select cast(0.789 as decimal(38, 38))";
+    AnalyzesOk(query, decimalV1Ctx);
+    AnalyzesOk(query, decimalV2Ctx);
+  }
+
+
+  @Test
   public void TestUnion() {
     // Selects on different tables.
     AnalyzesOk("select rank() over (order by int_col) from functional.alltypes union " +
@@ -3348,7 +3537,7 @@ public class AnalyzeStmtsTest extends AnalyzerTest {
         "select * from t1 union all select * from T2 union all select * from t3 " +
         "union all select * from T4 union all select * from t5");
 
-    // Multiple WITH clauses. One for the UnionStmt and one for each union operand.
+    // Multiple WITH clauses. One for the SetOperationStmt and one for each union operand.
     AnalyzesOk("with t1 as (values('a', 'b')) " +
         "(with t2 as (values('c', 'd')) select * from t2) union all" +
         "(with t3 as (values('e', 'f')) select * from t3) order by 1 limit 1");
@@ -4335,7 +4524,7 @@ public class AnalyzeStmtsTest extends AnalyzerTest {
   @Test
   public void TestClone() {
     testNumberOfMembers(QueryStmt.class, 11);
-    testNumberOfMembers(UnionStmt.class, 10);
+    testNumberOfMembers(SetOperationStmt.class, 13);
     testNumberOfMembers(ValuesStmt.class, 0);
 
     // Also check TableRefs.
diff --git a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
index 4539f03..582125b 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -818,110 +818,162 @@ public class ParserTest extends FrontendTestBase {
   }
 
   @Test
-  public void TestUnion() {
-    // Single union test.
-    ParsesOk("select a from test union select a from test");
-    ParsesOk("select a from test union all select a from test");
-    ParsesOk("select a from test union distinct select a from test");
-    // Chained union test.
-    ParsesOk("select a from test union select a from test " +
-        "union select a from test union select a from test");
-    ParsesOk("select a from test union all select a from test " +
-        "union all select a from test union all select a from test");
-    ParsesOk("select a from test union distinct select a from test " +
-        "union distinct select a from test union distinct select a from test ");
-    // Mixed union with all and distinct.
-    ParsesOk("select a from test union select a from test " +
-        "union all select a from test union distinct select a from test");
-    // No from clause.
-    ParsesOk("select sin() union select cos()");
-    ParsesOk("select sin() union all select cos()");
-    ParsesOk("select sin() union distinct select cos()");
-
-    // All select blocks in parenthesis.
-    ParsesOk("(select a from test) union (select a from test) " +
-        "union (select a from test) union (select a from test)");
-    // Union with order by,
-    ParsesOk("(select a from test) union (select a from test) " +
-        "union (select a from test) union (select a from test) order by a");
-    ParsesOk("(select a from test) union (select a from test) " +
-        "union (select a from test) union (select a from test) order by a nulls first");
-    // Union with limit.
-    ParsesOk("(select a from test) union (select a from test) " +
-        "union (select a from test) union (select a from test) limit 10");
-    // Union with order by, offset and limit.
-    ParsesOk("(select a from test) union (select a from test) " +
-        "union (select a from test) union (select a from test) order by a limit 10");
-    ParsesOk("(select a from test) union (select a from test) " +
-        "union (select a from test) union (select a from test) order by a " +
-        "nulls first limit 10");
-    ParsesOk("(select a from test) union (select a from test) " +
-        "union (select a from test) union (select a from test) order by a " +
-        "nulls first offset 10");
-    ParserError("select a from test union (select a from test) " +
-        "union (select a from test) union (select a from test) offset 10");
-    // Union with some select blocks in parenthesis, and others not.
-    ParsesOk("(select a from test) union select a from test " +
-        "union (select a from test) union select a from test");
-    ParsesOk("select a from test union (select a from test) " +
-        "union select a from test union (select a from test)");
-    // Union with order by, offset and limit binding to last select.
-    ParsesOk("(select a from test) union (select a from test) " +
-        "union select a from test union select a from test order by a limit 10");
-    ParsesOk("(select a from test) union (select a from test) " +
-        "union select a from test union select a from test order by a offset 10");
-    ParsesOk("(select a from test) union (select a from test) " +
-        "union select a from test union select a from test order by a");
-    // Union with order by and limit.
-    // Last select with order by and limit is in parenthesis.
-    ParsesOk("select a from test union (select a from test) " +
-        "union select a from test union (select a from test order by a limit 10) " +
-        "order by a limit 1");
-    ParsesOk("select a from test union (select a from test) " +
-        "union select a from test union (select a from test order by a offset 10) " +
-        "order by a limit 1");
-    ParsesOk("select a from test union (select a from test) " +
-        "union select a from test union (select a from test order by a) " +
-        "order by a limit 1");
-    // Union with order by, offset in first operand.
-    ParsesOk("select a from test order by a union select a from test");
-    ParsesOk("select a from test order by a offset 5 union select a from test");
-    ParsesOk("select a from test offset 5 union select a from test");
-    // Union with order by and limit.
-    // Last select with order by and limit is not in parenthesis.
-    ParsesOk("select a from test union select a from test " +
-        "union select a from test union select a from test order by a limit 10 " +
-        "order by a limit 1");
-
-    // Nested unions with order by and limit.
-    ParsesOk("select a union " +
-        "((select b) union (select c) order by 1 limit 1)");
-    ParsesOk("select a union " +
-        "((select b) union " +
-        "  ((select c) union (select d) " +
-        "   order by 1 limit 1) " +
-        " order by 1 limit 1)");
-
-    // Union in insert query.
-    ParsesOk("insert into table t select a from test union select a from test");
-    ParsesOk("insert into table t select a from test union select a from test " +
-        "union select a from test union select a from test");
-    ParsesOk("insert overwrite table t select a from test union select a from test");
-    ParsesOk("insert overwrite table t select a from test union select a from test " +
-        "union select a from test union select a from test");
-
-    // Union in upsert query.
-    ParsesOk("upsert into table t select a from test union select a from test");
-    ParsesOk("upsert into table t select a from test union select a from test " +
-        "union select a from test union select a from test");
-
-    // No complete select statement on lhs.
-    ParserError("a from test union select a from test");
-    // No complete select statement on rhs.
-    ParserError("select a from test union a from test");
-    // Union cannot be a column or table since it's a keyword.
-    ParserError("select union from test");
-    ParserError("select a from union");
+  public void TestSetOperations() {
+    // the ALL modifier isn't currently support with except / intersect
+    String allOp = "union";
+    ParsesOk(String.format("select a from test %s all select a from test", allOp));
+    ParsesOk(String.format("select sin() %s all select cos()", allOp));
+    // Mixed %s with all and distinct.
+    ParsesOk(String.format("select a from test %s select a from test "
+            + "%s all select a from test %s distinct select a from test",
+        allOp, allOp, allOp));
+    ParsesOk(String.format("select a from test %s all select a from test "
+            + "%s all select a from test %s all select a from test",
+        allOp, allOp, allOp));
+
+    final String[] noAllOps = new String[] {"except", "intersect", "minus"};
+    for (String noAllOp : noAllOps) {
+      ParserError(String.format("select a from test %s all select a from test", noAllOp));
+      ParserError(String.format("select sin() %s all select cos()", noAllOp));
+      // Mixed %s with all and distinct.
+      ParserError(String.format("select a from test %s select a from test "
+              + "%s all select a from test %s distinct select a from test",
+          noAllOp, noAllOp, noAllOp));
+      ParserError(String.format("select a from test %s all select a from test "
+              + "%s all select a from test %s all select a from test",
+          noAllOp, noAllOp, noAllOp));
+    }
+
+    final String[] setOperators = new String[] {"union", "except", "intersect", "minus"};
+    for (String op : setOperators) {
+      // Single union test.
+      ParsesOk(String.format("select a from test %s select a from test", op));
+      ParsesOk(String.format("select a from test %s distinct select a from test", op));
+      // Chained union test.
+      ParsesOk(String.format("select a from test %s select a from test "
+              + "%s select a from test %s select a from test",
+          op, op, op));
+      ParsesOk(String.format("select a from test %s distinct select a from test "
+              + "%s distinct select a from test %s distinct select a from test ",
+          op, op, op));
+      // No from clause.
+      ParsesOk(String.format("select sin() %s select cos()", op));
+      ParsesOk(String.format("select sin() %s distinct select cos()", op));
+
+      // All select blocks in parenthesis.
+      ParsesOk(String.format("(select a from test) %s (select a from test) "
+              + "%s (select a from test) %s (select a from test)",
+          op, op, op));
+      // op, op, operator with order by,
+      ParsesOk(String.format("(select a from test) %s (select a from test) "
+              + "%s (select a from test) %s (select a from test) order by a",
+          op, op, op));
+      ParsesOk(String.format("(select a from test) %s (select a from test) "
+              + "%s (select a from test) %s (select a from test) order by a nulls first",
+          op, op, op));
+      // op, op, operator with limit.
+      ParsesOk(String.format("(select a from test) %s (select a from test) "
+              + "%s (select a from test) %s (select a from test) limit 10",
+          op, op, op));
+      // op, op, operator with order by, offset and limit.
+      ParsesOk(String.format("(select a from test) %s (select a from test) "
+              + "%s (select a from test) %s (select a from test) order by a limit 10",
+          op, op, op));
+      ParsesOk(String.format("(select a from test) %s (select a from test) "
+              + "%s (select a from test) %s (select a from test) order by a "
+              + "nulls first limit 10",
+          op, op, op));
+      ParsesOk(String.format("(select a from test) %s (select a from test) "
+              + "%s (select a from test) %s (select a from test) order by a "
+              + "nulls first offset 10",
+          op, op, op));
+      ParserError(String.format("select a from test %s (select a from test) "
+              + "%s (select a from test) %s (select a from test) offset 10",
+          op, op, op));
+      // op, op, operator with some select blocks in parenthesis, and others not.
+      ParsesOk(String.format("(select a from test) %s select a from test "
+              + "%s (select a from test) %s select a from test",
+          op, op, op));
+      ParsesOk(String.format("select a from test %s (select a from test) "
+              + "%s select a from test %s (select a from test)",
+          op, op, op));
+      // op, op, operator with order by, offset and limit binding to last select.
+      ParsesOk(String.format("(select a from test) %s (select a from test) "
+              + "%s select a from test %s select a from test order by a limit 10",
+          op, op, op));
+      ParsesOk(String.format("(select a from test) %s (select a from test) "
+              + "%s select a from test %s select a from test order by a offset 10",
+          op, op, op));
+      ParsesOk(String.format("(select a from test) %s (select a from test) "
+              + "%s select a from test %s select a from test order by a",
+          op, op, op));
+      // op, op, operator with order by and limit.
+      // Last select with order by and limit is in parenthesis.
+      ParsesOk(String.format("select a from test %s (select a from test) "
+              + "%s select a from test %s (select a from test order by a limit 10) "
+              + "order by a limit 1",
+          op, op, op));
+      ParsesOk(String.format("select a from test %s (select a from test) "
+              + "%s select a from test %s (select a from test order by a offset 10) "
+              + "order by a limit 1",
+          op, op, op));
+      ParsesOk(String.format("select a from test %s (select a from test) "
+              + "%s select a from test %s (select a from test order by a) "
+              + "order by a limit 1",
+          op, op, op));
+      // operator with order by, offset in first operand.
+      ParsesOk(String.format("select a from test order by a %s select a from test", op));
+      ParsesOk(String.format(
+          "select a from test order by a offset 5 %s select a from test", op, op));
+      ParsesOk(String.format("select a from test offset 5 %s select a from test", op));
+      // operator with order by and limit.
+      // Last select with order by and limit is not in parenthesis.
+      ParsesOk(String.format("select a from test %s select a from test "
+              + "%s select a from test %s select a from test order by a limit 10 "
+              + "order by a limit 1",
+          op, op, op));
+
+      // Nested %ss with order by and limit.
+      ParsesOk(String.format("select a %s "
+              + "((select b) %s (select c) order by 1 limit 1)",
+          op, op));
+      ParsesOk(String.format("select a %s "
+              + "((select b) %s "
+              + "  ((select c) %s (select d) "
+              + "   order by 1 limit 1) "
+              + " order by 1 limit 1)",
+          op, op, op));
+
+      // operator in insert query.
+      ParsesOk(String.format(
+          "insert into table t select a from test %s select a from test", op));
+      ParsesOk(
+          String.format("insert into table t select a from test %s select a from test "
+                  + "%s select a from test %s select a from test",
+              op, op, op));
+      ParsesOk(String.format(
+          "insert overwrite table t select a from test %s select a from test", op));
+      ParsesOk(String.format(
+          "insert overwrite table t select a from test %s select a from test "
+              + "%s select a from test %s select a from test",
+          op, op, op));
+
+      // operator in upsert query.
+      ParsesOk(String.format(
+          "upsert into table t select a from test %s select a from test", op));
+      ParsesOk(
+          String.format("upsert into table t select a from test %s select a from test "
+                  + "%s select a from test %s select a from test",
+              op, op, op));
+
+      // No complete select statement on lhs.
+      ParserError(String.format("a from test %s select a from test", op));
+      // No complete select statement on rhs.
+      ParserError(String.format("select a from test %s a from test", op));
+      // operator cannot be a column or table since it's a keyword.
+      ParserError(String.format("select %s from test", op));
+      ParserError(String.format("select a from %s", op));
+    }
   }
 
   @Test
@@ -3403,8 +3455,9 @@ public class ParserTest extends FrontendTestBase {
         "select c, b, c where a = 5\n" +
         "               ^\n" +
         "Encountered: WHERE\n" +
-        "Expected: AND, AS, BETWEEN, DEFAULT, DIV, FROM, ILIKE, IN, IREGEXP, IS, LIKE, " +
-        "LIMIT, ||, NOT, OR, ORDER, REGEXP, RLIKE, UNION, COMMA, IDENTIFIER\n");
+        "Expected: AND, AS, BETWEEN, DEFAULT, DIV, EXCEPT, FROM, ILIKE, IN, INTERSECT, " +
+        "IREGEXP, IS, LIKE, LIMIT, ||, MINUS, NOT, OR, ORDER, REGEXP, RLIKE, UNION, " +
+        "COMMA, IDENTIFIER\n");
 
     // missing table list
     ParserError("select c, b, c from where a = 5",
@@ -3472,8 +3525,9 @@ public class ParserTest extends FrontendTestBase {
         "... b, c,c,c,c,c,c,c,c,c,a a a,c,c,c,c,c,c,c,cd,c,d,d,,c,...\n" +
         "                             ^\n" +
         "Encountered: IDENTIFIER\n" +
-        "Expected: CROSS, FROM, FULL, GROUP, HAVING, INNER, JOIN, LEFT, LIMIT, OFFSET, " +
-        "ON, ORDER, RIGHT, STRAIGHT_JOIN, TABLESAMPLE, UNION, USING, WHERE, COMMA\n");
+        "Expected: CROSS, EXCEPT, FROM, FULL, GROUP, HAVING, INNER, INTERSECT, JOIN, " +
+        "LEFT, LIMIT, MINUS, OFFSET, ON, ORDER, RIGHT, STRAIGHT_JOIN, TABLESAMPLE, " +
+        "UNION, USING, WHERE, COMMA\n");
 
     // Long line: error close to the start
     ParserError("select a a a, b, c,c,c,c,c,c,c,c,c,c,c,c,c,c,c,c,cd,c,d,d,,c, from t",
@@ -3481,8 +3535,9 @@ public class ParserTest extends FrontendTestBase {
         "select a a a, b, c,c,c,c,c,c,c,c,c,c,c,...\n" +
         "           ^\n" +
         "Encountered: IDENTIFIER\n" +
-        "Expected: CROSS, FROM, FULL, GROUP, HAVING, INNER, JOIN, LEFT, LIMIT, OFFSET, " +
-        "ON, ORDER, RIGHT, STRAIGHT_JOIN, TABLESAMPLE, UNION, USING, WHERE, COMMA\n");
+        "Expected: CROSS, EXCEPT, FROM, FULL, GROUP, HAVING, INNER, INTERSECT, JOIN, " +
+        "LEFT, LIMIT, MINUS, OFFSET, ON, ORDER, RIGHT, STRAIGHT_JOIN, TABLESAMPLE, " +
+        "UNION, USING, WHERE, COMMA\n");
 
     // Long line: error close to the end
     ParserError("select a, b, c,c,c,c,c,c,c,c,c,c,c,c,c,c,c,c,cd,c,d,d, ,c, from t",
diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
index df67702..875a740 100644
--- a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
@@ -57,6 +57,7 @@ public class PlannerTest extends PlannerTestBase {
     PlannerTestBase.setUp();
     // Rebalance the HBase tables. This is necessary because some tests rely on HBase
     // tables being arranged in a deterministic way. See IMPALA-7061 for details.
+
     HBaseTestDataRegionAssignment assignment = new HBaseTestDataRegionAssignment();
     assignment.performAssignment("functional_hbase.alltypessmall");
     assignment.performAssignment("functional_hbase.alltypesagg");
@@ -447,6 +448,11 @@ public class PlannerTest extends PlannerTestBase {
   }
 
   @Test
+  public void testSetOperationRewrite() {
+    runPlannerTestFile("setoperation-rewrite");
+  }
+
+  @Test
   public void testValues() {
     runPlannerTestFile("values");
   }
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/empty.test b/testdata/workloads/functional-planner/queries/PlannerTest/empty.test
index d8fb57c..e6cd733 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/empty.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/empty.test
@@ -635,3 +635,73 @@ PLAN-ROOT SINK
    HDFS partitions=24/24 files=24 size=478.45KB
    row-size=0B cardinality=7.30K
 ====
+# Constant false conjunct with intersect
+# This plan can be improved to eliminate the entire branch
+select int_col from functional.alltypessmall
+intersect
+select int_col from functional.alltypes where "abc" = "cde"
+---- PLAN
+PLAN-ROOT SINK
+|
+03:AGGREGATE [FINALIZE]
+|  group by: int_col
+|  row-size=4B cardinality=0
+|
+02:HASH JOIN [LEFT SEMI JOIN]
+|  hash predicates: int_col IS NOT DISTINCT FROM int_col
+|  runtime filters: RF000 <- int_col
+|  row-size=4B cardinality=0
+|
+|--01:EMPTYSET
+|
+00:SCAN HDFS [functional.alltypessmall]
+   HDFS partitions=4/4 files=4 size=6.32KB
+   runtime filters: RF000 -> int_col
+   row-size=4B cardinality=100
+====
+# Improve this so limit 0 removes the entire branch
+select id from functional.alltypessmall
+intersect
+select id from functional.alltypes limit 0
+except
+select id from functional.alltypestiny where int_col > 0
+---- PLAN
+PLAN-ROOT SINK
+|
+05:AGGREGATE [FINALIZE]
+|  group by: id
+|  row-size=4B cardinality=0
+|
+04:HASH JOIN [RIGHT ANTI JOIN]
+|  hash predicates: id IS NOT DISTINCT FROM id
+|  row-size=4B cardinality=0
+|
+|--03:HASH JOIN [LEFT SEMI JOIN]
+|  |  hash predicates: id IS NOT DISTINCT FROM id
+|  |  runtime filters: RF000 <- id
+|  |  row-size=4B cardinality=0
+|  |
+|  |--01:EMPTYSET
+|  |
+|  00:SCAN HDFS [functional.alltypessmall]
+|     HDFS partitions=4/4 files=4 size=6.32KB
+|     runtime filters: RF000 -> id
+|     row-size=4B cardinality=100
+|
+02:SCAN HDFS [functional.alltypestiny]
+   HDFS partitions=4/4 files=4 size=460B
+   predicates: int_col > 0
+   row-size=8B cardinality=1
+====
+# Limit 0 clears out all operands
+select id from functional.alltypessmall
+except
+select id from functional.alltypes where "abc" = "cde"
+except
+(select id from functional.alltypestiny)
+limit 0
+---- PLAN
+PLAN-ROOT SINK
+|
+00:EMPTYSET
+====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/setoperation-rewrite.test b/testdata/workloads/functional-planner/queries/PlannerTest/setoperation-rewrite.test
new file mode 100644
index 0000000..8bfd42e
--- /dev/null
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/setoperation-rewrite.test
@@ -0,0 +1,758 @@
+# intersect
+select year, month from functional.alltypes
+  intersect
+select year, month from functional.alltypes where year=2009
+---- PLAN
+PLAN-ROOT SINK
+|
+04:AGGREGATE [FINALIZE]
+|  group by: `year`, `month`
+|  row-size=8B cardinality=24
+|
+03:HASH JOIN [LEFT SEMI JOIN]
+|  hash predicates: `month` IS NOT DISTINCT FROM functional.alltypes.month, `year` IS NOT DISTINCT FROM functional.alltypes.year
+|  runtime filters: RF000 <- functional.alltypes.month, RF001 <- functional.alltypes.year
+|  row-size=8B cardinality=7.30K
+|
+|--02:AGGREGATE [FINALIZE]
+|  |  group by: functional.alltypes.year, functional.alltypes.month
+|  |  row-size=8B cardinality=24
+|  |
+|  01:SCAN HDFS [functional.alltypes]
+|     partition predicates: `year` = 2009
+|     HDFS partitions=12/24 files=12 size=238.68KB
+|     row-size=8B cardinality=3.65K
+|
+00:SCAN HDFS [functional.alltypes]
+   HDFS partitions=24/24 files=24 size=478.45KB
+   runtime filters: RF000 -> `month`, RF001 -> `year`
+   row-size=8B cardinality=7.30K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+10:EXCHANGE [UNPARTITIONED]
+|
+09:AGGREGATE [FINALIZE]
+|  group by: $a$1.year, $a$1.month
+|  row-size=8B cardinality=24
+|
+08:EXCHANGE [HASH($a$1.year,$a$1.month)]
+|
+04:AGGREGATE [STREAMING]
+|  group by: `year`, `month`
+|  row-size=8B cardinality=24
+|
+03:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  hash predicates: `month` IS NOT DISTINCT FROM functional.alltypes.month, `year` IS NOT DISTINCT FROM functional.alltypes.year
+|  runtime filters: RF000 <- functional.alltypes.month, RF001 <- functional.alltypes.year
+|  row-size=8B cardinality=7.30K
+|
+|--07:EXCHANGE [BROADCAST]
+|  |
+|  06:AGGREGATE [FINALIZE]
+|  |  group by: functional.alltypes.year, functional.alltypes.month
+|  |  row-size=8B cardinality=24
+|  |
+|  05:EXCHANGE [HASH(functional.alltypes.year,functional.alltypes.month)]
+|  |
+|  02:AGGREGATE [STREAMING]
+|  |  group by: functional.alltypes.year, functional.alltypes.month
+|  |  row-size=8B cardinality=24
+|  |
+|  01:SCAN HDFS [functional.alltypes]
+|     partition predicates: `year` = 2009
+|     HDFS partitions=12/24 files=12 size=238.68KB
+|     row-size=8B cardinality=3.65K
+|
+00:SCAN HDFS [functional.alltypes]
+   HDFS partitions=24/24 files=24 size=478.45KB
+   runtime filters: RF000 -> `month`, RF001 -> `year`
+   row-size=8B cardinality=7.30K
+====
+# intersect unnesting
+select id, year, month from functional.alltypestiny where year=2009 and month=1
+intersect
+(select id, year, month from functional.alltypestiny where year=2009 and month=1
+intersect
+ (select id, year, month from functional.alltypestiny where year=2009 and month=2
+  intersect
+ select id, year, month from functional.alltypestiny where year=2009 and month=2))
+---- PLAN
+PLAN-ROOT SINK
+|
+09:AGGREGATE [FINALIZE]
+|  group by: id, `year`, `month`
+|  row-size=12B cardinality=2
+|
+08:HASH JOIN [LEFT SEMI JOIN]
+|  hash predicates: `month` IS NOT DISTINCT FROM $a$1.month, `year` IS NOT DISTINCT FROM $a$1.year, id IS NOT DISTINCT FROM $a$1.id
+|  runtime filters: RF000 <- $a$1.month, RF001 <- $a$1.year, RF002 <- $a$1.id
+|  row-size=12B cardinality=2
+|
+|--07:AGGREGATE [FINALIZE]
+|  |  group by: id, `year`, `month`
+|  |  row-size=12B cardinality=2
+|  |
+|  06:HASH JOIN [LEFT SEMI JOIN]
+|  |  hash predicates: `month` IS NOT DISTINCT FROM $a$1.month, `year` IS NOT DISTINCT FROM $a$1.year, id IS NOT DISTINCT FROM $a$1.id
+|  |  runtime filters: RF006 <- $a$1.month, RF007 <- $a$1.year, RF008 <- $a$1.id
+|  |  row-size=12B cardinality=2
+|  |
+|  |--05:AGGREGATE [FINALIZE]
+|  |  |  group by: id, `year`, `month`
+|  |  |  row-size=12B cardinality=2
+|  |  |
+|  |  04:HASH JOIN [LEFT SEMI JOIN]
+|  |  |  hash predicates: `month` IS NOT DISTINCT FROM `month`, `year` IS NOT DISTINCT FROM `year`, id IS NOT DISTINCT FROM id
+|  |  |  runtime filters: RF012 <- `month`, RF013 <- `year`, RF014 <- id
+|  |  |  row-size=12B cardinality=2
+|  |  |
+|  |  |--03:SCAN HDFS [functional.alltypestiny]
+|  |  |     partition predicates: `year` = 2009, `month` = 2
+|  |  |     HDFS partitions=1/4 files=1 size=115B
+|  |  |     row-size=12B cardinality=2
+|  |  |
+|  |  02:SCAN HDFS [functional.alltypestiny]
+|  |     partition predicates: `year` = 2009, `month` = 2
+|  |     HDFS partitions=1/4 files=1 size=115B
+|  |     runtime filters: RF012 -> `month`, RF013 -> `year`, RF014 -> id
+|  |     row-size=12B cardinality=2
+|  |
+|  01:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     runtime filters: RF006 -> `month`, RF007 -> `year`, RF008 -> id
+|     row-size=12B cardinality=2
+|
+00:SCAN HDFS [functional.alltypestiny]
+   partition predicates: `year` = 2009, `month` = 1
+   HDFS partitions=1/4 files=1 size=115B
+   runtime filters: RF000 -> `month`, RF001 -> `year`, RF002 -> id
+   row-size=12B cardinality=2
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+19:EXCHANGE [UNPARTITIONED]
+|
+18:AGGREGATE [FINALIZE]
+|  group by: $a$1.id, $a$1.year, $a$1.month
+|  row-size=12B cardinality=2
+|
+17:EXCHANGE [HASH($a$1.id,$a$1.year,$a$1.month)]
+|
+09:AGGREGATE [STREAMING]
+|  group by: id, `year`, `month`
+|  row-size=12B cardinality=2
+|
+08:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  hash predicates: `month` IS NOT DISTINCT FROM $a$1.month, `year` IS NOT DISTINCT FROM $a$1.year, id IS NOT DISTINCT FROM $a$1.id
+|  runtime filters: RF000 <- $a$1.month, RF001 <- $a$1.year, RF002 <- $a$1.id
+|  row-size=12B cardinality=2
+|
+|--16:EXCHANGE [BROADCAST]
+|  |
+|  15:AGGREGATE [FINALIZE]
+|  |  group by: $a$1.id, $a$1.year, $a$1.month
+|  |  row-size=12B cardinality=2
+|  |
+|  14:EXCHANGE [HASH($a$1.id,$a$1.year,$a$1.month)]
+|  |
+|  07:AGGREGATE [STREAMING]
+|  |  group by: id, `year`, `month`
+|  |  row-size=12B cardinality=2
+|  |
+|  06:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  hash predicates: `month` IS NOT DISTINCT FROM $a$1.month, `year` IS NOT DISTINCT FROM $a$1.year, id IS NOT DISTINCT FROM $a$1.id
+|  |  runtime filters: RF006 <- $a$1.month, RF007 <- $a$1.year, RF008 <- $a$1.id
+|  |  row-size=12B cardinality=2
+|  |
+|  |--13:EXCHANGE [BROADCAST]
+|  |  |
+|  |  12:AGGREGATE [FINALIZE]
+|  |  |  group by: $a$1.id, $a$1.year, $a$1.month
+|  |  |  row-size=12B cardinality=2
+|  |  |
+|  |  11:EXCHANGE [HASH($a$1.id,$a$1.year,$a$1.month)]
+|  |  |
+|  |  05:AGGREGATE [STREAMING]
+|  |  |  group by: id, `year`, `month`
+|  |  |  row-size=12B cardinality=2
+|  |  |
+|  |  04:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  |  |  hash predicates: `month` IS NOT DISTINCT FROM `month`, `year` IS NOT DISTINCT FROM `year`, id IS NOT DISTINCT FROM id
+|  |  |  runtime filters: RF012 <- `month`, RF013 <- `year`, RF014 <- id
+|  |  |  row-size=12B cardinality=2
+|  |  |
+|  |  |--10:EXCHANGE [BROADCAST]
+|  |  |  |
+|  |  |  03:SCAN HDFS [functional.alltypestiny]
+|  |  |     partition predicates: `year` = 2009, `month` = 2
+|  |  |     HDFS partitions=1/4 files=1 size=115B
+|  |  |     row-size=12B cardinality=2
+|  |  |
+|  |  02:SCAN HDFS [functional.alltypestiny]
+|  |     partition predicates: `year` = 2009, `month` = 2
+|  |     HDFS partitions=1/4 files=1 size=115B
+|  |     runtime filters: RF012 -> `month`, RF013 -> `year`, RF014 -> id
+|  |     row-size=12B cardinality=2
+|  |
+|  01:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     runtime filters: RF006 -> `month`, RF007 -> `year`, RF008 -> id
+|     row-size=12B cardinality=2
+|
+00:SCAN HDFS [functional.alltypestiny]
+   partition predicates: `year` = 2009, `month` = 1
+   HDFS partitions=1/4 files=1 size=115B
+   runtime filters: RF000 -> `month`, RF001 -> `year`, RF002 -> id
+   row-size=12B cardinality=2
+====
+# intersect uses inner join if distinct on both sides
+select distinct id, year, month from functional.alltypestiny where year=2009 and month=1
+intersect
+select distinct id, year, month from functional.alltypestiny where year=2009 and month=1
+---- PLAN
+PLAN-ROOT SINK
+|
+04:HASH JOIN [INNER JOIN]
+|  hash predicates: `month` IS NOT DISTINCT FROM `month`, `year` IS NOT DISTINCT FROM `year`, id IS NOT DISTINCT FROM id
+|  runtime filters: RF000 <- `month`, RF001 <- `year`, RF002 <- id
+|  row-size=24B cardinality=2
+|
+|--03:AGGREGATE [FINALIZE]
+|  |  group by: id, `year`, `month`
+|  |  row-size=12B cardinality=2
+|  |
+|  02:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=12B cardinality=2
+|
+01:AGGREGATE [FINALIZE]
+|  group by: id, `year`, `month`
+|  row-size=12B cardinality=2
+|
+00:SCAN HDFS [functional.alltypestiny]
+   partition predicates: `year` = 2009, `month` = 1
+   HDFS partitions=1/4 files=1 size=115B
+   runtime filters: RF000 -> functional.alltypestiny.month, RF001 -> functional.alltypestiny.year, RF002 -> functional.alltypestiny.id
+   row-size=12B cardinality=2
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+10:EXCHANGE [UNPARTITIONED]
+|
+04:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash predicates: `month` IS NOT DISTINCT FROM `month`, `year` IS NOT DISTINCT FROM `year`, id IS NOT DISTINCT FROM id
+|  runtime filters: RF000 <- `month`, RF001 <- `year`, RF002 <- id
+|  row-size=24B cardinality=2
+|
+|--09:EXCHANGE [HASH(id,`year`,`month`)]
+|  |
+|  08:AGGREGATE [FINALIZE]
+|  |  group by: id, `year`, `month`
+|  |  row-size=12B cardinality=2
+|  |
+|  07:EXCHANGE [HASH(id,`year`,`month`)]
+|  |
+|  03:AGGREGATE [STREAMING]
+|  |  group by: id, `year`, `month`
+|  |  row-size=12B cardinality=2
+|  |
+|  02:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=12B cardinality=2
+|
+06:AGGREGATE [FINALIZE]
+|  group by: id, `year`, `month`
+|  row-size=12B cardinality=2
+|
+05:EXCHANGE [HASH(id,`year`,`month`)]
+|
+01:AGGREGATE [STREAMING]
+|  group by: id, `year`, `month`
+|  row-size=12B cardinality=2
+|
+00:SCAN HDFS [functional.alltypestiny]
+   partition predicates: `year` = 2009, `month` = 1
+   HDFS partitions=1/4 files=1 size=115B
+   runtime filters: RF000 -> functional.alltypestiny.month, RF001 -> functional.alltypestiny.year, RF002 -> functional.alltypestiny.id
+   row-size=12B cardinality=2
+====
+# except
+select id, year, month from functional.alltypestiny where year=2009 and month=1
+except
+select id, year, month from functional.alltypestiny where year=2009 and month=1
+except
+select id, year, month from functional.alltypestiny where year=2009 and month=2
+---- PLAN
+PLAN-ROOT SINK
+|
+05:AGGREGATE [FINALIZE]
+|  group by: id, `year`, `month`
+|  row-size=12B cardinality=2
+|
+04:HASH JOIN [LEFT ANTI JOIN]
+|  hash predicates: `month` IS NOT DISTINCT FROM `month`, `year` IS NOT DISTINCT FROM `year`, id IS NOT DISTINCT FROM id
+|  row-size=12B cardinality=2
+|
+|--02:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 2
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=12B cardinality=2
+|
+03:HASH JOIN [LEFT ANTI JOIN]
+|  hash predicates: `month` IS NOT DISTINCT FROM `month`, `year` IS NOT DISTINCT FROM `year`, id IS NOT DISTINCT FROM id
+|  row-size=12B cardinality=2
+|
+|--01:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=12B cardinality=2
+|
+00:SCAN HDFS [functional.alltypestiny]
+   partition predicates: `year` = 2009, `month` = 1
+   HDFS partitions=1/4 files=1 size=115B
+   row-size=12B cardinality=2
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+10:EXCHANGE [UNPARTITIONED]
+|
+09:AGGREGATE [FINALIZE]
+|  group by: $a$1.id, $a$1.year, $a$1.month
+|  row-size=12B cardinality=2
+|
+08:EXCHANGE [HASH($a$1.id,$a$1.year,$a$1.month)]
+|
+05:AGGREGATE [STREAMING]
+|  group by: id, `year`, `month`
+|  row-size=12B cardinality=2
+|
+04:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
+|  hash predicates: `month` IS NOT DISTINCT FROM `month`, `year` IS NOT DISTINCT FROM `year`, id IS NOT DISTINCT FROM id
+|  row-size=12B cardinality=2
+|
+|--07:EXCHANGE [BROADCAST]
+|  |
+|  02:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 2
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=12B cardinality=2
+|
+03:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
+|  hash predicates: `month` IS NOT DISTINCT FROM `month`, `year` IS NOT DISTINCT FROM `year`, id IS NOT DISTINCT FROM id
+|  row-size=12B cardinality=2
+|
+|--06:EXCHANGE [BROADCAST]
+|  |
+|  01:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=12B cardinality=2
+|
+00:SCAN HDFS [functional.alltypestiny]
+   partition predicates: `year` = 2009, `month` = 1
+   HDFS partitions=1/4 files=1 size=115B
+   row-size=12B cardinality=2
+====
+# nested except, shouldn't be unnested, if it had been the results would be incorrect
+# note the difference from the plan above with has all the operands at the same level
+select id, year, month from functional.alltypestiny where year=2009 and month=1
+except
+(select id, year, month from functional.alltypestiny where year=2009 and month=1
+except
+select id, year, month from functional.alltypestiny where year=2009 and month=2)
+---- PLAN
+PLAN-ROOT SINK
+|
+06:AGGREGATE [FINALIZE]
+|  group by: id, `year`, `month`
+|  row-size=12B cardinality=2
+|
+05:HASH JOIN [LEFT ANTI JOIN]
+|  hash predicates: `month` IS NOT DISTINCT FROM $a$1.month, `year` IS NOT DISTINCT FROM $a$1.year, id IS NOT DISTINCT FROM $a$1.id
+|  row-size=12B cardinality=2
+|
+|--04:AGGREGATE [FINALIZE]
+|  |  group by: id, `year`, `month`
+|  |  row-size=12B cardinality=2
+|  |
+|  03:HASH JOIN [LEFT ANTI JOIN]
+|  |  hash predicates: `month` IS NOT DISTINCT FROM `month`, `year` IS NOT DISTINCT FROM `year`, id IS NOT DISTINCT FROM id
+|  |  row-size=12B cardinality=2
+|  |
+|  |--02:SCAN HDFS [functional.alltypestiny]
+|  |     partition predicates: `year` = 2009, `month` = 2
+|  |     HDFS partitions=1/4 files=1 size=115B
+|  |     row-size=12B cardinality=2
+|  |
+|  01:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=12B cardinality=2
+|
+00:SCAN HDFS [functional.alltypestiny]
+   partition predicates: `year` = 2009, `month` = 1
+   HDFS partitions=1/4 files=1 size=115B
+   row-size=12B cardinality=2
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+13:EXCHANGE [UNPARTITIONED]
+|
+12:AGGREGATE [FINALIZE]
+|  group by: $a$1.id, $a$1.year, $a$1.month
+|  row-size=12B cardinality=2
+|
+11:EXCHANGE [HASH($a$1.id,$a$1.year,$a$1.month)]
+|
+06:AGGREGATE [STREAMING]
+|  group by: id, `year`, `month`
+|  row-size=12B cardinality=2
+|
+05:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
+|  hash predicates: `month` IS NOT DISTINCT FROM $a$1.month, `year` IS NOT DISTINCT FROM $a$1.year, id IS NOT DISTINCT FROM $a$1.id
+|  row-size=12B cardinality=2
+|
+|--10:EXCHANGE [BROADCAST]
+|  |
+|  09:AGGREGATE [FINALIZE]
+|  |  group by: $a$1.id, $a$1.year, $a$1.month
+|  |  row-size=12B cardinality=2
+|  |
+|  08:EXCHANGE [HASH($a$1.id,$a$1.year,$a$1.month)]
+|  |
+|  04:AGGREGATE [STREAMING]
+|  |  group by: id, `year`, `month`
+|  |  row-size=12B cardinality=2
+|  |
+|  03:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
+|  |  hash predicates: `month` IS NOT DISTINCT FROM `month`, `year` IS NOT DISTINCT FROM `year`, id IS NOT DISTINCT FROM id
+|  |  row-size=12B cardinality=2
+|  |
+|  |--07:EXCHANGE [BROADCAST]
+|  |  |
+|  |  02:SCAN HDFS [functional.alltypestiny]
+|  |     partition predicates: `year` = 2009, `month` = 2
+|  |     HDFS partitions=1/4 files=1 size=115B
+|  |     row-size=12B cardinality=2
+|  |
+|  01:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=12B cardinality=2
+|
+00:SCAN HDFS [functional.alltypestiny]
+   partition predicates: `year` = 2009, `month` = 1
+   HDFS partitions=1/4 files=1 size=115B
+   row-size=12B cardinality=2
+====
+# mixed intersect / except with unions first
+select * from functional.alltypestiny where year=2009 and month=1 limit 1
+union all
+select * from functional.alltypestiny where year=2009 and month=1 order by int_col limit 1
+union distinct
+select * from functional.alltypestiny where year=2009 and month=2 limit 1
+except
+select * from functional.alltypestiny where year=2009 and month=1 order by int_col limit 1
+intersect
+select * from functional.alltypestiny where year=2009 and month=2 limit 1
+---- PLAN
+PLAN-ROOT SINK
+|
+11:AGGREGATE [FINALIZE]
+|  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
+|  row-size=89B cardinality=1
+|
+10:HASH JOIN [LEFT SEMI JOIN]
+|  hash predicates: bigint_col IS NOT DISTINCT FROM functional.alltypestiny.bigint_col, bool_col IS NOT DISTINCT FROM functional.alltypestiny.bool_col, double_col IS NOT DISTINCT FROM functional.alltypestiny.double_col, float_col IS NOT DISTINCT FROM functional.alltypestiny.float_col, id IS NOT DISTINCT FROM functional.alltypestiny.id, int_col IS NOT DISTINCT FROM functional.alltypestiny.int_col, month IS NOT DISTINCT FROM functional.alltypestiny.month, smallint_col IS NOT DISTINCT FROM  [...]
+|  runtime filters: RF000 <- functional.alltypestiny.bigint_col, RF001 <- functional.alltypestiny.bool_col, RF002 <- functional.alltypestiny.double_col, RF003 <- functional.alltypestiny.float_col, RF004 <- functional.alltypestiny.id, RF005 <- functional.alltypestiny.int_col, RF006 <- functional.alltypestiny.month, RF007 <- functional.alltypestiny.smallint_col, RF008 <- functional.alltypestiny.timestamp_col, RF010 <- functional.alltypestiny.tinyint_col
+|  row-size=89B cardinality=1
+|
+|--08:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 2
+|     HDFS partitions=1/4 files=1 size=115B
+|     limit: 1
+|     row-size=89B cardinality=1
+|
+09:HASH JOIN [LEFT ANTI JOIN]
+|  hash predicates: bigint_col IS NOT DISTINCT FROM bigint_col, bool_col IS NOT DISTINCT FROM bool_col, double_col IS NOT DISTINCT FROM double_col, float_col IS NOT DISTINCT FROM float_col, id IS NOT DISTINCT FROM id, int_col IS NOT DISTINCT FROM int_col, month IS NOT DISTINCT FROM month, smallint_col IS NOT DISTINCT FROM smallint_col, timestamp_col IS NOT DISTINCT FROM timestamp_col, tinyint_col IS NOT DISTINCT FROM tinyint_col, year IS NOT DISTINCT FROM year, string_col IS NOT DISTINCT [...]
+|  row-size=89B cardinality=3
+|
+|--07:TOP-N [LIMIT=1]
+|  |  order by: int_col ASC
+|  |  row-size=89B cardinality=1
+|  |
+|  06:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=89B cardinality=2
+|
+05:AGGREGATE [FINALIZE]
+|  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
+|  row-size=89B cardinality=3
+|
+00:UNION
+|  pass-through-operands: all
+|  row-size=89B cardinality=3
+|
+|--04:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 2
+|     HDFS partitions=1/4 files=1 size=115B
+|     runtime filters: RF000 -> functional.alltypestiny.bigint_col, RF001 -> functional.alltypestiny.bool_col, RF002 -> functional.alltypestiny.double_col, RF003 -> functional.alltypestiny.float_col, RF004 -> functional.alltypestiny.id, RF005 -> functional.alltypestiny.int_col, RF006 -> functional.alltypestiny.month, RF007 -> functional.alltypestiny.smallint_col, RF008 -> functional.alltypestiny.timestamp_col, RF010 -> functional.alltypestiny.tinyint_col
+|     limit: 1
+|     row-size=89B cardinality=1
+|
+|--03:TOP-N [LIMIT=1]
+|  |  order by: int_col ASC
+|  |  row-size=89B cardinality=1
+|  |
+|  02:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=89B cardinality=2
+|
+01:SCAN HDFS [functional.alltypestiny]
+   partition predicates: `year` = 2009, `month` = 1
+   HDFS partitions=1/4 files=1 size=115B
+   runtime filters: RF000 -> functional.alltypestiny.bigint_col, RF001 -> functional.alltypestiny.bool_col, RF002 -> functional.alltypestiny.double_col, RF003 -> functional.alltypestiny.float_col, RF004 -> functional.alltypestiny.id, RF005 -> functional.alltypestiny.int_col, RF006 -> functional.alltypestiny.month, RF007 -> functional.alltypestiny.smallint_col, RF008 -> functional.alltypestiny.timestamp_col, RF010 -> functional.alltypestiny.tinyint_col
+   limit: 1
+   row-size=89B cardinality=1
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+11:AGGREGATE [FINALIZE]
+|  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
+|  row-size=89B cardinality=1
+|
+10:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  hash predicates: bigint_col IS NOT DISTINCT FROM functional.alltypestiny.bigint_col, bool_col IS NOT DISTINCT FROM functional.alltypestiny.bool_col, double_col IS NOT DISTINCT FROM functional.alltypestiny.double_col, float_col IS NOT DISTINCT FROM functional.alltypestiny.float_col, id IS NOT DISTINCT FROM functional.alltypestiny.id, int_col IS NOT DISTINCT FROM functional.alltypestiny.int_col, month IS NOT DISTINCT FROM functional.alltypestiny.month, smallint_col IS NOT DISTINCT FROM  [...]
+|  runtime filters: RF000 <- functional.alltypestiny.bigint_col, RF001 <- functional.alltypestiny.bool_col, RF002 <- functional.alltypestiny.double_col, RF003 <- functional.alltypestiny.float_col, RF004 <- functional.alltypestiny.id, RF005 <- functional.alltypestiny.int_col, RF006 <- functional.alltypestiny.month, RF007 <- functional.alltypestiny.smallint_col, RF008 <- functional.alltypestiny.timestamp_col, RF010 <- functional.alltypestiny.tinyint_col
+|  row-size=89B cardinality=1
+|
+|--18:EXCHANGE [UNPARTITIONED]
+|  |
+|  17:EXCHANGE [UNPARTITIONED]
+|  |  limit: 1
+|  |
+|  08:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 2
+|     HDFS partitions=1/4 files=1 size=115B
+|     limit: 1
+|     row-size=89B cardinality=1
+|
+09:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
+|  hash predicates: bigint_col IS NOT DISTINCT FROM bigint_col, bool_col IS NOT DISTINCT FROM bool_col, double_col IS NOT DISTINCT FROM double_col, float_col IS NOT DISTINCT FROM float_col, id IS NOT DISTINCT FROM id, int_col IS NOT DISTINCT FROM int_col, month IS NOT DISTINCT FROM month, smallint_col IS NOT DISTINCT FROM smallint_col, timestamp_col IS NOT DISTINCT FROM timestamp_col, tinyint_col IS NOT DISTINCT FROM tinyint_col, year IS NOT DISTINCT FROM year, string_col IS NOT DISTINCT [...]
+|  row-size=89B cardinality=3
+|
+|--16:EXCHANGE [UNPARTITIONED]
+|  |
+|  15:MERGING-EXCHANGE [UNPARTITIONED]
+|  |  order by: int_col ASC
+|  |  limit: 1
+|  |
+|  07:TOP-N [LIMIT=1]
+|  |  order by: int_col ASC
+|  |  row-size=89B cardinality=1
+|  |
+|  06:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=89B cardinality=2
+|
+05:AGGREGATE [FINALIZE]
+|  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
+|  row-size=89B cardinality=3
+|
+00:UNION
+|  pass-through-operands: all
+|  row-size=89B cardinality=3
+|
+|--14:EXCHANGE [UNPARTITIONED]
+|  |  limit: 1
+|  |
+|  04:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 2
+|     HDFS partitions=1/4 files=1 size=115B
+|     runtime filters: RF000 -> functional.alltypestiny.bigint_col, RF001 -> functional.alltypestiny.bool_col, RF002 -> functional.alltypestiny.double_col, RF003 -> functional.alltypestiny.float_col, RF004 -> functional.alltypestiny.id, RF005 -> functional.alltypestiny.int_col, RF006 -> functional.alltypestiny.month, RF007 -> functional.alltypestiny.smallint_col, RF008 -> functional.alltypestiny.timestamp_col, RF010 -> functional.alltypestiny.tinyint_col
+|     limit: 1
+|     row-size=89B cardinality=1
+|
+|--13:MERGING-EXCHANGE [UNPARTITIONED]
+|  |  order by: int_col ASC
+|  |  limit: 1
+|  |
+|  03:TOP-N [LIMIT=1]
+|  |  order by: int_col ASC
+|  |  row-size=89B cardinality=1
+|  |
+|  02:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=89B cardinality=2
+|
+12:EXCHANGE [UNPARTITIONED]
+|  limit: 1
+|
+01:SCAN HDFS [functional.alltypestiny]
+   partition predicates: `year` = 2009, `month` = 1
+   HDFS partitions=1/4 files=1 size=115B
+   runtime filters: RF000 -> functional.alltypestiny.bigint_col, RF001 -> functional.alltypestiny.bool_col, RF002 -> functional.alltypestiny.double_col, RF003 -> functional.alltypestiny.float_col, RF004 -> functional.alltypestiny.id, RF005 -> functional.alltypestiny.int_col, RF006 -> functional.alltypestiny.month, RF007 -> functional.alltypestiny.smallint_col, RF008 -> functional.alltypestiny.timestamp_col, RF010 -> functional.alltypestiny.tinyint_col
+   limit: 1
+   row-size=89B cardinality=1
+====
+# mixed intersect / except with unions second
+select * from functional.alltypestiny where year=2009 and month=2 limit 1
+except
+select * from functional.alltypestiny where year=2009 and month=1 order by int_col limit 1
+intersect
+select * from functional.alltypestiny where year=2009 and month=2 limit 1
+union all
+select * from functional.alltypestiny where year=2009 and month=1 limit 1
+union distinct
+select * from functional.alltypestiny where year=2009 and month=1 order by int_col limit 1
+---- PLAN
+PLAN-ROOT SINK
+|
+11:AGGREGATE [FINALIZE]
+|  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
+|  row-size=89B cardinality=3
+|
+00:UNION
+|  pass-through-operands: all
+|  row-size=89B cardinality=3
+|
+|--10:TOP-N [LIMIT=1]
+|  |  order by: int_col ASC
+|  |  row-size=89B cardinality=1
+|  |
+|  09:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=89B cardinality=2
+|
+|--08:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     limit: 1
+|     row-size=89B cardinality=1
+|
+07:AGGREGATE [FINALIZE]
+|  group by: functional.alltypestiny.id, functional.alltypestiny.bool_col, functional.alltypestiny.tinyint_col, functional.alltypestiny.smallint_col, functional.alltypestiny.int_col, functional.alltypestiny.bigint_col, functional.alltypestiny.float_col, functional.alltypestiny.double_col, functional.alltypestiny.date_string_col, functional.alltypestiny.string_col, functional.alltypestiny.timestamp_col, functional.alltypestiny.year, functional.alltypestiny.month
+|  row-size=89B cardinality=1
+|
+06:HASH JOIN [LEFT SEMI JOIN]
+|  hash predicates: functional.alltypestiny.bigint_col IS NOT DISTINCT FROM functional.alltypestiny.bigint_col, functional.alltypestiny.bool_col IS NOT DISTINCT FROM functional.alltypestiny.bool_col, functional.alltypestiny.double_col IS NOT DISTINCT FROM functional.alltypestiny.double_col, functional.alltypestiny.float_col IS NOT DISTINCT FROM functional.alltypestiny.float_col, functional.alltypestiny.id IS NOT DISTINCT FROM functional.alltypestiny.id, functional.alltypestiny.int_col IS [...]
+|  runtime filters: RF000 <- functional.alltypestiny.bigint_col, RF001 <- functional.alltypestiny.bool_col, RF002 <- functional.alltypestiny.double_col, RF003 <- functional.alltypestiny.float_col, RF004 <- functional.alltypestiny.id, RF005 <- functional.alltypestiny.int_col, RF006 <- functional.alltypestiny.month, RF007 <- functional.alltypestiny.smallint_col, RF008 <- functional.alltypestiny.timestamp_col, RF010 <- functional.alltypestiny.tinyint_col
+|  row-size=89B cardinality=1
+|
+|--04:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 2
+|     HDFS partitions=1/4 files=1 size=115B
+|     limit: 1
+|     row-size=89B cardinality=1
+|
+05:HASH JOIN [LEFT ANTI JOIN]
+|  hash predicates: functional.alltypestiny.bigint_col IS NOT DISTINCT FROM bigint_col, functional.alltypestiny.bool_col IS NOT DISTINCT FROM bool_col, functional.alltypestiny.double_col IS NOT DISTINCT FROM double_col, functional.alltypestiny.float_col IS NOT DISTINCT FROM float_col, functional.alltypestiny.id IS NOT DISTINCT FROM id, functional.alltypestiny.int_col IS NOT DISTINCT FROM int_col, functional.alltypestiny.month IS NOT DISTINCT FROM month, functional.alltypestiny.smallint_c [...]
+|  row-size=89B cardinality=1
+|
+|--03:TOP-N [LIMIT=1]
+|  |  order by: int_col ASC
+|  |  row-size=89B cardinality=1
+|  |
+|  02:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=89B cardinality=2
+|
+01:SCAN HDFS [functional.alltypestiny]
+   partition predicates: `year` = 2009, `month` = 2
+   HDFS partitions=1/4 files=1 size=115B
+   runtime filters: RF000 -> functional.alltypestiny.bigint_col, RF001 -> functional.alltypestiny.bool_col, RF002 -> functional.alltypestiny.double_col, RF003 -> functional.alltypestiny.float_col, RF004 -> functional.alltypestiny.id, RF005 -> functional.alltypestiny.int_col, RF006 -> functional.alltypestiny.month, RF007 -> functional.alltypestiny.smallint_col, RF008 -> functional.alltypestiny.timestamp_col, RF010 -> functional.alltypestiny.tinyint_col
+   limit: 1
+   row-size=89B cardinality=1
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+11:AGGREGATE [FINALIZE]
+|  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
+|  row-size=89B cardinality=3
+|
+00:UNION
+|  pass-through-operands: all
+|  row-size=89B cardinality=3
+|
+|--18:MERGING-EXCHANGE [UNPARTITIONED]
+|  |  order by: int_col ASC
+|  |  limit: 1
+|  |
+|  10:TOP-N [LIMIT=1]
+|  |  order by: int_col ASC
+|  |  row-size=89B cardinality=1
+|  |
+|  09:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=89B cardinality=2
+|
+|--17:EXCHANGE [UNPARTITIONED]
+|  |  limit: 1
+|  |
+|  08:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     limit: 1
+|     row-size=89B cardinality=1
+|
+07:AGGREGATE [FINALIZE]
+|  group by: functional.alltypestiny.id, functional.alltypestiny.bool_col, functional.alltypestiny.tinyint_col, functional.alltypestiny.smallint_col, functional.alltypestiny.int_col, functional.alltypestiny.bigint_col, functional.alltypestiny.float_col, functional.alltypestiny.double_col, functional.alltypestiny.date_string_col, functional.alltypestiny.string_col, functional.alltypestiny.timestamp_col, functional.alltypestiny.year, functional.alltypestiny.month
+|  row-size=89B cardinality=1
+|
+06:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
+|  hash predicates: functional.alltypestiny.bigint_col IS NOT DISTINCT FROM functional.alltypestiny.bigint_col, functional.alltypestiny.bool_col IS NOT DISTINCT FROM functional.alltypestiny.bool_col, functional.alltypestiny.double_col IS NOT DISTINCT FROM functional.alltypestiny.double_col, functional.alltypestiny.float_col IS NOT DISTINCT FROM functional.alltypestiny.float_col, functional.alltypestiny.id IS NOT DISTINCT FROM functional.alltypestiny.id, functional.alltypestiny.int_col IS [...]
+|  runtime filters: RF000 <- functional.alltypestiny.bigint_col, RF001 <- functional.alltypestiny.bool_col, RF002 <- functional.alltypestiny.double_col, RF003 <- functional.alltypestiny.float_col, RF004 <- functional.alltypestiny.id, RF005 <- functional.alltypestiny.int_col, RF006 <- functional.alltypestiny.month, RF007 <- functional.alltypestiny.smallint_col, RF008 <- functional.alltypestiny.timestamp_col, RF010 <- functional.alltypestiny.tinyint_col
+|  row-size=89B cardinality=1
+|
+|--16:EXCHANGE [UNPARTITIONED]
+|  |
+|  15:EXCHANGE [UNPARTITIONED]
+|  |  limit: 1
+|  |
+|  04:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 2
+|     HDFS partitions=1/4 files=1 size=115B
+|     limit: 1
+|     row-size=89B cardinality=1
+|
+05:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
+|  hash predicates: functional.alltypestiny.bigint_col IS NOT DISTINCT FROM bigint_col, functional.alltypestiny.bool_col IS NOT DISTINCT FROM bool_col, functional.alltypestiny.double_col IS NOT DISTINCT FROM double_col, functional.alltypestiny.float_col IS NOT DISTINCT FROM float_col, functional.alltypestiny.id IS NOT DISTINCT FROM id, functional.alltypestiny.int_col IS NOT DISTINCT FROM int_col, functional.alltypestiny.month IS NOT DISTINCT FROM month, functional.alltypestiny.smallint_c [...]
+|  row-size=89B cardinality=1
+|
+|--14:EXCHANGE [UNPARTITIONED]
+|  |
+|  13:MERGING-EXCHANGE [UNPARTITIONED]
+|  |  order by: int_col ASC
+|  |  limit: 1
+|  |
+|  03:TOP-N [LIMIT=1]
+|  |  order by: int_col ASC
+|  |  row-size=89B cardinality=1
+|  |
+|  02:SCAN HDFS [functional.alltypestiny]
+|     partition predicates: `year` = 2009, `month` = 1
+|     HDFS partitions=1/4 files=1 size=115B
+|     row-size=89B cardinality=2
+|
+12:EXCHANGE [UNPARTITIONED]
+|  limit: 1
+|
+01:SCAN HDFS [functional.alltypestiny]
+   partition predicates: `year` = 2009, `month` = 2
+   HDFS partitions=1/4 files=1 size=115B
+   runtime filters: RF000 -> functional.alltypestiny.bigint_col, RF001 -> functional.alltypestiny.bool_col, RF002 -> functional.alltypestiny.double_col, RF003 -> functional.alltypestiny.float_col, RF004 -> functional.alltypestiny.id, RF005 -> functional.alltypestiny.int_col, RF006 -> functional.alltypestiny.month, RF007 -> functional.alltypestiny.smallint_col, RF008 -> functional.alltypestiny.timestamp_col, RF010 -> functional.alltypestiny.tinyint_col
+   limit: 1
+   row-size=89B cardinality=1
+====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
index c08726c..63f0b8f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
@@ -13624,6 +13624,552 @@ PLAN-ROOT SINK
    HDFS partitions=1824/1824 files=1824 size=346.60MB
    runtime filters: RF000 -> ss_store_sk, RF002 -> ss_item_sk, RF004 -> ss_sold_date_sk
    row-size=24B cardinality=2.88M
+=======
+---- QUERY: TPCDS-Q38
+SELECT count(*)
+FROM
+  (SELECT DISTINCT c_last_name,
+                   c_first_name,
+                   d_date
+   FROM store_sales,
+        date_dim,
+        customer
+   WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk
+     AND store_sales.ss_customer_sk = customer.c_customer_sk
+     AND d_month_seq BETWEEN 1200 AND 1200 + 11 INTERSECT
+     SELECT DISTINCT c_last_name,
+                     c_first_name,
+                     d_date
+     FROM catalog_sales,
+          date_dim,
+          customer WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
+     AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk
+     AND d_month_seq BETWEEN 1200 AND 1200 + 11 INTERSECT
+     SELECT DISTINCT c_last_name,
+                     c_first_name,
+                     d_date
+     FROM web_sales,
+          date_dim,
+          customer WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk
+     AND web_sales.ws_bill_customer_sk = customer.c_customer_sk
+     AND d_month_seq BETWEEN 1200 AND 1200 + 11 ) hot_cust
+LIMIT 100
+---- PLAN
+Max Per-Host Resource Reservation: Memory=194.69MB Threads=10
+Per-Host Resource Estimates: Memory=861MB
+PLAN-ROOT SINK
+|
+20:AGGREGATE [FINALIZE]
+|  output: count(*)
+|  limit: 100
+|  row-size=8B cardinality=1
+|
+19:HASH JOIN [INNER JOIN]
+|  hash predicates: c_first_name IS NOT DISTINCT FROM c_first_name, c_last_name IS NOT DISTINCT FROM c_last_name, d_date IS NOT DISTINCT FROM d_date
+|  runtime filters: RF000 <- c_first_name, RF001 <- c_last_name, RF002 <- d_date
+|  row-size=174B cardinality=2.88M
+|
+|--17:AGGREGATE [FINALIZE]
+|  |  group by: c_last_name, c_first_name, d_date
+|  |  row-size=58B cardinality=719.38K
+|  |
+|  16:HASH JOIN [INNER JOIN]
+|  |  hash predicates: web_sales.ws_bill_customer_sk = customer.c_customer_sk
+|  |  row-size=78B cardinality=719.38K
+|  |
+|  |--14:SCAN HDFS [tpcds.customer]
+|  |     HDFS partitions=1/1 files=1 size=12.60MB
+|  |     row-size=40B cardinality=100.00K
+|  |
+|  15:HASH JOIN [INNER JOIN]
+|  |  hash predicates: web_sales.ws_sold_date_sk = date_dim.d_date_sk
+|  |  row-size=38B cardinality=719.38K
+|  |
+|  |--13:SCAN HDFS [tpcds.date_dim]
+|  |     HDFS partitions=1/1 files=1 size=9.84MB
+|  |     predicates: d_month_seq <= 1211, d_month_seq >= 1200
+|  |     row-size=30B cardinality=7.30K
+|  |
+|  12:SCAN HDFS [tpcds.web_sales]
+|     HDFS partitions=1/1 files=1 size=140.07MB
+|     row-size=8B cardinality=719.38K
+|
+18:HASH JOIN [INNER JOIN]
+|  hash predicates: c_first_name IS NOT DISTINCT FROM c_first_name, c_last_name IS NOT DISTINCT FROM c_last_name, d_date IS NOT DISTINCT FROM d_date
+|  runtime filters: RF006 <- c_first_name, RF007 <- c_last_name, RF008 <- d_date
+|  row-size=116B cardinality=2.88M
+|
+|--11:AGGREGATE [FINALIZE]
+|  |  group by: c_last_name, c_first_name, d_date
+|  |  row-size=58B cardinality=1.44M
+|  |
+|  10:HASH JOIN [INNER JOIN]
+|  |  hash predicates: catalog_sales.cs_bill_customer_sk = customer.c_customer_sk
+|  |  runtime filters: RF016 <- customer.c_customer_sk
+|  |  row-size=78B cardinality=1.44M
+|  |
+|  |--08:SCAN HDFS [tpcds.customer]
+|  |     HDFS partitions=1/1 files=1 size=12.60MB
+|  |     row-size=40B cardinality=100.00K
+|  |
+|  09:HASH JOIN [INNER JOIN]
+|  |  hash predicates: catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
+|  |  runtime filters: RF018 <- date_dim.d_date_sk
+|  |  row-size=38B cardinality=1.44M
+|  |
+|  |--07:SCAN HDFS [tpcds.date_dim]
+|  |     HDFS partitions=1/1 files=1 size=9.84MB
+|  |     predicates: d_month_seq <= 1211, d_month_seq >= 1200
+|  |     row-size=30B cardinality=7.30K
+|  |
+|  06:SCAN HDFS [tpcds.catalog_sales]
+|     HDFS partitions=1/1 files=1 size=282.20MB
+|     runtime filters: RF016 -> catalog_sales.cs_bill_customer_sk, RF018 -> catalog_sales.cs_sold_date_sk
+|     row-size=8B cardinality=1.44M
+|
+05:AGGREGATE [FINALIZE]
+|  group by: c_last_name, c_first_name, d_date
+|  row-size=58B cardinality=2.88M
+|
+04:HASH JOIN [INNER JOIN]
+|  hash predicates: store_sales.ss_customer_sk = customer.c_customer_sk
+|  runtime filters: RF012 <- customer.c_customer_sk
+|  row-size=78B cardinality=2.88M
+|
+|--02:SCAN HDFS [tpcds.customer]
+|     HDFS partitions=1/1 files=1 size=12.60MB
+|     runtime filters: RF000 -> tpcds.customer.c_first_name, RF001 -> tpcds.customer.c_last_name, RF006 -> tpcds.customer.c_first_name, RF007 -> tpcds.customer.c_last_name
+|     row-size=40B cardinality=100.00K
+|
+03:HASH JOIN [INNER JOIN]
+|  hash predicates: store_sales.ss_sold_date_sk = date_dim.d_date_sk
+|  runtime filters: RF014 <- date_dim.d_date_sk
+|  row-size=38B cardinality=2.88M
+|
+|--01:SCAN HDFS [tpcds.date_dim]
+|     HDFS partitions=1/1 files=1 size=9.84MB
+|     predicates: d_month_seq <= 1211, d_month_seq >= 1200
+|     runtime filters: RF002 -> tpcds.date_dim.d_date, RF008 -> tpcds.date_dim.d_date
+|     row-size=30B cardinality=7.30K
+|
+00:SCAN HDFS [tpcds.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
+   runtime filters: RF012 -> store_sales.ss_customer_sk, RF014 -> store_sales.ss_sold_date_sk
+   row-size=8B cardinality=2.88M
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=363.06MB Threads=22
+Per-Host Resource Estimates: Memory=1.21GB
+PLAN-ROOT SINK
+|
+36:AGGREGATE [FINALIZE]
+|  output: count:merge(*)
+|  limit: 100
+|  row-size=8B cardinality=1
+|
+35:EXCHANGE [UNPARTITIONED]
+|
+20:AGGREGATE
+|  output: count(*)
+|  row-size=8B cardinality=1
+|
+19:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash predicates: c_first_name IS NOT DISTINCT FROM c_first_name, c_last_name IS NOT DISTINCT FROM c_last_name, d_date IS NOT DISTINCT FROM d_date
+|  runtime filters: RF000 <- c_first_name, RF001 <- c_last_name, RF002 <- d_date
+|  row-size=174B cardinality=2.88M
+|
+|--34:EXCHANGE [HASH(c_last_name,c_first_name,d_date)]
+|  |
+|  33:AGGREGATE [FINALIZE]
+|  |  group by: c_last_name, c_first_name, d_date
+|  |  row-size=58B cardinality=719.38K
+|  |
+|  32:EXCHANGE [HASH(c_last_name,c_first_name,d_date)]
+|  |
+|  17:AGGREGATE [STREAMING]
+|  |  group by: c_last_name, c_first_name, d_date
+|  |  row-size=58B cardinality=719.38K
+|  |
+|  16:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: web_sales.ws_bill_customer_sk = customer.c_customer_sk
+|  |  row-size=78B cardinality=719.38K
+|  |
+|  |--31:EXCHANGE [BROADCAST]
+|  |  |
+|  |  14:SCAN HDFS [tpcds.customer]
+|  |     HDFS partitions=1/1 files=1 size=12.60MB
+|  |     row-size=40B cardinality=100.00K
+|  |
+|  15:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: web_sales.ws_sold_date_sk = date_dim.d_date_sk
+|  |  row-size=38B cardinality=719.38K
+|  |
+|  |--30:EXCHANGE [BROADCAST]
+|  |  |
+|  |  13:SCAN HDFS [tpcds.date_dim]
+|  |     HDFS partitions=1/1 files=1 size=9.84MB
+|  |     predicates: d_month_seq <= 1211, d_month_seq >= 1200
+|  |     row-size=30B cardinality=7.30K
+|  |
+|  12:SCAN HDFS [tpcds.web_sales]
+|     HDFS partitions=1/1 files=1 size=140.07MB
+|     row-size=8B cardinality=719.38K
+|
+18:HASH JOIN [INNER JOIN, PARTITIONED]
+|  hash predicates: c_first_name IS NOT DISTINCT FROM c_first_name, c_last_name IS NOT DISTINCT FROM c_last_name, d_date IS NOT DISTINCT FROM d_date
+|  runtime filters: RF006 <- c_first_name, RF007 <- c_last_name, RF008 <- d_date
+|  row-size=116B cardinality=2.88M
+|
+|--29:EXCHANGE [HASH(c_last_name,c_first_name,d_date)]
+|  |
+|  28:AGGREGATE [FINALIZE]
+|  |  group by: c_last_name, c_first_name, d_date
+|  |  row-size=58B cardinality=1.44M
+|  |
+|  27:EXCHANGE [HASH(c_last_name,c_first_name,d_date)]
+|  |
+|  11:AGGREGATE [STREAMING]
+|  |  group by: c_last_name, c_first_name, d_date
+|  |  row-size=58B cardinality=1.44M
+|  |
+|  10:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: catalog_sales.cs_bill_customer_sk = customer.c_customer_sk
+|  |  runtime filters: RF016 <- customer.c_customer_sk
+|  |  row-size=78B cardinality=1.44M
+|  |
+|  |--26:EXCHANGE [BROADCAST]
+|  |  |
+|  |  08:SCAN HDFS [tpcds.customer]
+|  |     HDFS partitions=1/1 files=1 size=12.60MB
+|  |     row-size=40B cardinality=100.00K
+|  |
+|  09:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
+|  |  runtime filters: RF018 <- date_dim.d_date_sk
+|  |  row-size=38B cardinality=1.44M
+|  |
+|  |--25:EXCHANGE [BROADCAST]
+|  |  |
+|  |  07:SCAN HDFS [tpcds.date_dim]
+|  |     HDFS partitions=1/1 files=1 size=9.84MB
+|  |     predicates: d_month_seq <= 1211, d_month_seq >= 1200
+|  |     row-size=30B cardinality=7.30K
+|  |
+|  06:SCAN HDFS [tpcds.catalog_sales]
+|     HDFS partitions=1/1 files=1 size=282.20MB
+|     runtime filters: RF016 -> catalog_sales.cs_bill_customer_sk, RF018 -> catalog_sales.cs_sold_date_sk
+|     row-size=8B cardinality=1.44M
+|
+24:AGGREGATE [FINALIZE]
+|  group by: c_last_name, c_first_name, d_date
+|  row-size=58B cardinality=2.88M
+|
+23:EXCHANGE [HASH(c_last_name,c_first_name,d_date)]
+|
+05:AGGREGATE [STREAMING]
+|  group by: c_last_name, c_first_name, d_date
+|  row-size=58B cardinality=2.88M
+|
+04:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash predicates: store_sales.ss_customer_sk = customer.c_customer_sk
+|  runtime filters: RF012 <- customer.c_customer_sk
+|  row-size=78B cardinality=2.88M
+|
+|--22:EXCHANGE [BROADCAST]
+|  |
+|  02:SCAN HDFS [tpcds.customer]
+|     HDFS partitions=1/1 files=1 size=12.60MB
+|     runtime filters: RF000 -> tpcds.customer.c_first_name, RF001 -> tpcds.customer.c_last_name, RF006 -> tpcds.customer.c_first_name, RF007 -> tpcds.customer.c_last_name
+|     row-size=40B cardinality=100.00K
+|
+03:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash predicates: store_sales.ss_sold_date_sk = date_dim.d_date_sk
+|  runtime filters: RF014 <- date_dim.d_date_sk
+|  row-size=38B cardinality=2.88M
+|
+|--21:EXCHANGE [BROADCAST]
+|  |
+|  01:SCAN HDFS [tpcds.date_dim]
+|     HDFS partitions=1/1 files=1 size=9.84MB
+|     predicates: d_month_seq <= 1211, d_month_seq >= 1200
+|     runtime filters: RF002 -> tpcds.date_dim.d_date, RF008 -> tpcds.date_dim.d_date
+|     row-size=30B cardinality=7.30K
+|
+00:SCAN HDFS [tpcds.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
+   runtime filters: RF012 -> store_sales.ss_customer_sk, RF014 -> store_sales.ss_sold_date_sk
+   row-size=8B cardinality=2.88M
+====
+---- QUERY: TPCDS-Q87
+SELECT count(*)
+FROM ((SELECT DISTINCT c_last_name,
+                         c_first_name,
+                         d_date
+         FROM store_sales,
+              date_dim,
+              customer
+         WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk
+           AND store_sales.ss_customer_sk = customer.c_customer_sk
+           AND d_month_seq BETWEEN 1200 AND 1200+11)
+      EXCEPT
+        (SELECT DISTINCT c_last_name,
+                         c_first_name,
+                         d_date
+         FROM catalog_sales,
+              date_dim,
+              customer
+         WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
+           AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk
+           AND d_month_seq BETWEEN 1200 AND 1200+11)
+      EXCEPT
+        (SELECT DISTINCT c_last_name,
+                         c_first_name,
+                         d_date
+         FROM web_sales,
+              date_dim,
+              customer
+         WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk
+           AND web_sales.ws_bill_customer_sk = customer.c_customer_sk
+           AND d_month_seq BETWEEN 1200 AND 1200+11)) cool_cust;
+---- PLAN
+Max Per-Host Resource Reservation: Memory=190.69MB Threads=10
+Per-Host Resource Estimates: Memory=857MB
+PLAN-ROOT SINK
+|
+20:AGGREGATE [FINALIZE]
+|  output: count(*)
+|  row-size=8B cardinality=1
+|
+19:HASH JOIN [LEFT ANTI JOIN]
+|  hash predicates: c_first_name IS NOT DISTINCT FROM c_first_name, c_last_name IS NOT DISTINCT FROM c_last_name, d_date IS NOT DISTINCT FROM d_date
+|  row-size=58B cardinality=2.88M
+|
+|--17:AGGREGATE [FINALIZE]
+|  |  group by: c_last_name, c_first_name, d_date
+|  |  row-size=58B cardinality=719.38K
+|  |
+|  16:HASH JOIN [INNER JOIN]
+|  |  hash predicates: web_sales.ws_bill_customer_sk = customer.c_customer_sk
+|  |  runtime filters: RF008 <- customer.c_customer_sk
+|  |  row-size=78B cardinality=719.38K
+|  |
+|  |--14:SCAN HDFS [tpcds.customer]
+|  |     HDFS partitions=1/1 files=1 size=12.60MB
+|  |     row-size=40B cardinality=100.00K
+|  |
+|  15:HASH JOIN [INNER JOIN]
+|  |  hash predicates: web_sales.ws_sold_date_sk = date_dim.d_date_sk
+|  |  runtime filters: RF010 <- date_dim.d_date_sk
+|  |  row-size=38B cardinality=719.38K
+|  |
+|  |--13:SCAN HDFS [tpcds.date_dim]
+|  |     HDFS partitions=1/1 files=1 size=9.84MB
+|  |     predicates: d_month_seq <= 1211, d_month_seq >= 1200
+|  |     row-size=30B cardinality=7.30K
+|  |
+|  12:SCAN HDFS [tpcds.web_sales]
+|     HDFS partitions=1/1 files=1 size=140.07MB
+|     runtime filters: RF008 -> web_sales.ws_bill_customer_sk, RF010 -> web_sales.ws_sold_date_sk
+|     row-size=8B cardinality=719.38K
+|
+18:HASH JOIN [LEFT ANTI JOIN]
+|  hash predicates: c_first_name IS NOT DISTINCT FROM c_first_name, c_last_name IS NOT DISTINCT FROM c_last_name, d_date IS NOT DISTINCT FROM d_date
+|  row-size=58B cardinality=2.88M
+|
+|--11:AGGREGATE [FINALIZE]
+|  |  group by: c_last_name, c_first_name, d_date
+|  |  row-size=58B cardinality=1.44M
+|  |
+|  10:HASH JOIN [INNER JOIN]
+|  |  hash predicates: catalog_sales.cs_bill_customer_sk = customer.c_customer_sk
+|  |  runtime filters: RF004 <- customer.c_customer_sk
+|  |  row-size=78B cardinality=1.44M
+|  |
+|  |--08:SCAN HDFS [tpcds.customer]
+|  |     HDFS partitions=1/1 files=1 size=12.60MB
+|  |     row-size=40B cardinality=100.00K
+|  |
+|  09:HASH JOIN [INNER JOIN]
+|  |  hash predicates: catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
+|  |  runtime filters: RF006 <- date_dim.d_date_sk
+|  |  row-size=38B cardinality=1.44M
+|  |
+|  |--07:SCAN HDFS [tpcds.date_dim]
+|  |     HDFS partitions=1/1 files=1 size=9.84MB
+|  |     predicates: d_month_seq <= 1211, d_month_seq >= 1200
+|  |     row-size=30B cardinality=7.30K
+|  |
+|  06:SCAN HDFS [tpcds.catalog_sales]
+|     HDFS partitions=1/1 files=1 size=282.20MB
+|     runtime filters: RF004 -> catalog_sales.cs_bill_customer_sk, RF006 -> catalog_sales.cs_sold_date_sk
+|     row-size=8B cardinality=1.44M
+|
+05:AGGREGATE [FINALIZE]
+|  group by: c_last_name, c_first_name, d_date
+|  row-size=58B cardinality=2.88M
+|
+04:HASH JOIN [INNER JOIN]
+|  hash predicates: store_sales.ss_customer_sk = customer.c_customer_sk
+|  runtime filters: RF000 <- customer.c_customer_sk
+|  row-size=78B cardinality=2.88M
+|
+|--02:SCAN HDFS [tpcds.customer]
+|     HDFS partitions=1/1 files=1 size=12.60MB
+|     row-size=40B cardinality=100.00K
+|
+03:HASH JOIN [INNER JOIN]
+|  hash predicates: store_sales.ss_sold_date_sk = date_dim.d_date_sk
+|  runtime filters: RF002 <- date_dim.d_date_sk
+|  row-size=38B cardinality=2.88M
+|
+|--01:SCAN HDFS [tpcds.date_dim]
+|     HDFS partitions=1/1 files=1 size=9.84MB
+|     predicates: d_month_seq <= 1211, d_month_seq >= 1200
+|     row-size=30B cardinality=7.30K
+|
+00:SCAN HDFS [tpcds.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
+   runtime filters: RF000 -> store_sales.ss_customer_sk, RF002 -> store_sales.ss_sold_date_sk
+   row-size=8B cardinality=2.88M
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=353.06MB Threads=22
+Per-Host Resource Estimates: Memory=1.20GB
+PLAN-ROOT SINK
+|
+36:AGGREGATE [FINALIZE]
+|  output: count:merge(*)
+|  row-size=8B cardinality=1
+|
+35:EXCHANGE [UNPARTITIONED]
+|
+20:AGGREGATE
+|  output: count(*)
+|  row-size=8B cardinality=1
+|
+19:HASH JOIN [LEFT ANTI JOIN, PARTITIONED]
+|  hash predicates: c_first_name IS NOT DISTINCT FROM c_first_name, c_last_name IS NOT DISTINCT FROM c_last_name, d_date IS NOT DISTINCT FROM d_date
+|  row-size=58B cardinality=2.88M
+|
+|--34:EXCHANGE [HASH(c_last_name,c_first_name,d_date)]
+|  |
+|  33:AGGREGATE [FINALIZE]
+|  |  group by: c_last_name, c_first_name, d_date
+|  |  row-size=58B cardinality=719.38K
+|  |
+|  32:EXCHANGE [HASH(c_last_name,c_first_name,d_date)]
+|  |
+|  17:AGGREGATE [STREAMING]
+|  |  group by: c_last_name, c_first_name, d_date
+|  |  row-size=58B cardinality=719.38K
+|  |
+|  16:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: web_sales.ws_bill_customer_sk = customer.c_customer_sk
+|  |  runtime filters: RF008 <- customer.c_customer_sk
+|  |  row-size=78B cardinality=719.38K
+|  |
+|  |--31:EXCHANGE [BROADCAST]
+|  |  |
+|  |  14:SCAN HDFS [tpcds.customer]
+|  |     HDFS partitions=1/1 files=1 size=12.60MB
+|  |     row-size=40B cardinality=100.00K
+|  |
+|  15:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: web_sales.ws_sold_date_sk = date_dim.d_date_sk
+|  |  runtime filters: RF010 <- date_dim.d_date_sk
+|  |  row-size=38B cardinality=719.38K
+|  |
+|  |--30:EXCHANGE [BROADCAST]
+|  |  |
+|  |  13:SCAN HDFS [tpcds.date_dim]
+|  |     HDFS partitions=1/1 files=1 size=9.84MB
+|  |     predicates: d_month_seq <= 1211, d_month_seq >= 1200
+|  |     row-size=30B cardinality=7.30K
+|  |
+|  12:SCAN HDFS [tpcds.web_sales]
+|     HDFS partitions=1/1 files=1 size=140.07MB
+|     runtime filters: RF008 -> web_sales.ws_bill_customer_sk, RF010 -> web_sales.ws_sold_date_sk
+|     row-size=8B cardinality=719.38K
+|
+18:HASH JOIN [LEFT ANTI JOIN, PARTITIONED]
+|  hash predicates: c_first_name IS NOT DISTINCT FROM c_first_name, c_last_name IS NOT DISTINCT FROM c_last_name, d_date IS NOT DISTINCT FROM d_date
+|  row-size=58B cardinality=2.88M
+|
+|--29:EXCHANGE [HASH(c_last_name,c_first_name,d_date)]
+|  |
+|  28:AGGREGATE [FINALIZE]
+|  |  group by: c_last_name, c_first_name, d_date
+|  |  row-size=58B cardinality=1.44M
+|  |
+|  27:EXCHANGE [HASH(c_last_name,c_first_name,d_date)]
+|  |
+|  11:AGGREGATE [STREAMING]
+|  |  group by: c_last_name, c_first_name, d_date
+|  |  row-size=58B cardinality=1.44M
+|  |
+|  10:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: catalog_sales.cs_bill_customer_sk = customer.c_customer_sk
+|  |  runtime filters: RF004 <- customer.c_customer_sk
+|  |  row-size=78B cardinality=1.44M
+|  |
+|  |--26:EXCHANGE [BROADCAST]
+|  |  |
+|  |  08:SCAN HDFS [tpcds.customer]
+|  |     HDFS partitions=1/1 files=1 size=12.60MB
+|  |     row-size=40B cardinality=100.00K
+|  |
+|  09:HASH JOIN [INNER JOIN, BROADCAST]
+|  |  hash predicates: catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
+|  |  runtime filters: RF006 <- date_dim.d_date_sk
+|  |  row-size=38B cardinality=1.44M
+|  |
+|  |--25:EXCHANGE [BROADCAST]
+|  |  |
+|  |  07:SCAN HDFS [tpcds.date_dim]
+|  |     HDFS partitions=1/1 files=1 size=9.84MB
+|  |     predicates: d_month_seq <= 1211, d_month_seq >= 1200
+|  |     row-size=30B cardinality=7.30K
+|  |
+|  06:SCAN HDFS [tpcds.catalog_sales]
+|     HDFS partitions=1/1 files=1 size=282.20MB
+|     runtime filters: RF004 -> catalog_sales.cs_bill_customer_sk, RF006 -> catalog_sales.cs_sold_date_sk
+|     row-size=8B cardinality=1.44M
+|
+24:AGGREGATE [FINALIZE]
+|  group by: c_last_name, c_first_name, d_date
+|  row-size=58B cardinality=2.88M
+|
+23:EXCHANGE [HASH(c_last_name,c_first_name,d_date)]
+|
+05:AGGREGATE [STREAMING]
+|  group by: c_last_name, c_first_name, d_date
+|  row-size=58B cardinality=2.88M
+|
+04:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash predicates: store_sales.ss_customer_sk = customer.c_customer_sk
+|  runtime filters: RF000 <- customer.c_customer_sk
+|  row-size=78B cardinality=2.88M
+|
+|--22:EXCHANGE [BROADCAST]
+|  |
+|  02:SCAN HDFS [tpcds.customer]
+|     HDFS partitions=1/1 files=1 size=12.60MB
+|     row-size=40B cardinality=100.00K
+|
+03:HASH JOIN [INNER JOIN, BROADCAST]
+|  hash predicates: store_sales.ss_sold_date_sk = date_dim.d_date_sk
+|  runtime filters: RF002 <- date_dim.d_date_sk
+|  row-size=38B cardinality=2.88M
+|
+|--21:EXCHANGE [BROADCAST]
+|  |
+|  01:SCAN HDFS [tpcds.date_dim]
+|     HDFS partitions=1/1 files=1 size=9.84MB
+|     predicates: d_month_seq <= 1211, d_month_seq >= 1200
+|     row-size=30B cardinality=7.30K
+|
+00:SCAN HDFS [tpcds.store_sales]
+   HDFS partitions=1824/1824 files=1824 size=346.60MB
+   runtime filters: RF000 -> store_sales.ss_customer_sk, RF002 -> store_sales.ss_sold_date_sk
+   row-size=8B cardinality=2.88M
 ====
 # Q23-1
 with frequent_ss_items as
diff --git a/testdata/workloads/functional-query/queries/QueryTest/except.test b/testdata/workloads/functional-query/queries/QueryTest/except.test
new file mode 100644
index 0000000..51dcd0f
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/except.test
@@ -0,0 +1,628 @@
+====
+---- QUERY
+# Showing contents of alltypestiny for convenience
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+4,true,0,0,0,0,0,0,'03/01/09','0',2009-03-01 00:00:00,2009,3
+5,false,1,1,1,10,1.100000023841858,10.1,'03/01/09','1',2009-03-01 00:01:00,2009,3
+6,true,0,0,0,0,0,0,'04/01/09','0',2009-04-01 00:00:00,2009,4
+7,false,1,1,1,10,1.100000023841858,10.1,'04/01/09','1',2009-04-01 00:01:00,2009,4
+====
+---- QUERY
+# Only except
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+except
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+except
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=3
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# Only EXCEPT with limit inside operands. One of the operands also has an order by.
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1 and id < 2 limit 3
+except
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1 order by int_col limit 1
+except
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2 and id = 2 limit 1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# limits containing arithmetic expressions
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1 and id < 3 order by id limit 3-2+3
+except
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1 order by int_col limit 1*1*1
+except
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2 order by id limit 1&1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# mixed with union distinct
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+union distinct
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+except
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# mixed selects with and without from clauses, no nested unions
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month < 3
+except
+select 0,true,0,0,0,0,0,0,'01/01/09','0',cast('2009-01-01 00:00:00' as timestamp), 2009,1
+except
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+except
+select 1,false,1,1,1,10,1.1,10.1,'01/01/09','1',cast('2009-01-01 00:01:00' as timestamp), 2009,1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# except with one nested with order by and limit
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month < 4
+except
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=3
+except distinct
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+order by 1 limit 1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+====
+---- QUERY
+# Mixed except and union, no nested unions, with order by and limit
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+union all
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+except distinct
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+except distinct
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+order by 1,2 limit 3
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# simple query unnesting tests
+select 10 except select 11 except select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+====
+---- QUERY
+# don't unnest
+select 10 except (select 11 except select 10)
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+10
+====
+---- QUERY
+# left operand don't unnest
+(select 10 except select 11) except select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+====
+---- QUERY
+# middle
+select 11 except (select 10 except select 11) except select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+11
+====
+---- QUERY
+# if middle were unnested it would return no rows
+select 11 except select 10 except select 11 except select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+====
+---- QUERY
+# nested with union distinct
+(select 10 except select 11) union select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+10
+====
+---- QUERY
+# nested with union all
+(select 10 except select 11) union all select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+10
+10
+====
+---- QUERY
+# nested with union all
+(select 10 except select 11) union all select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+10
+10
+====
+---- QUERY
+# union all followed by except
+select 10 union all select 11 except select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+11
+====
+---- QUERY
+# multiple union alls followed by except
+select 10 union all select 11 union all select 11 except select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+11
+====
+---- QUERY
+# nested union outer except
+(select 10 union distinct select 11) except select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+11
+====
+---- QUERY
+# union distinct followed by except
+select 10 union distinct select 11 except select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+11
+====
+---- QUERY
+# except outer nested union distinct
+select 10 except (select 11 union select 10)
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+====
+---- QUERY
+# except then union distinct
+select 10 except select 11 union select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+10
+====
+---- QUERY
+# except then union distinct
+select 10 except (select 11 union all select 10)
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+====
+---- QUERY
+# except then union distinct
+select 10 except select 11 union all select 10
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+10
+10
+====
+---- QUERY
+# minus distinct as an alias for except distinct
+select a+1 from (select id as a from alltypessmall minus distinct select id from alltypestiny) T where T.a > 90
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+92
+93
+94
+95
+96
+97
+98
+99
+100
+====
+---- QUERY
+# minus as an alias for except
+select alltypestiny.id+2 as a from alltypestiny left outer join dimtbl on alltypestiny.id = dimtbl.id  minus select id from alltypestiny
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+8
+9
+====
+---- QUERY
+#  unnesting first operand is nested
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+ except
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+except 
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+====
+---- QUERY
+#  unnesting second operand is nested
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+ except
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+except 
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1)
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# no unnesting: first operand is nested union all
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+ union all
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+ except
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# no unnesting: second operand is nested unionall
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+except
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+   union all
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+====
+---- QUERY
+# no unnesting: UNION DISTINCT in first operand except in outer
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+ union distinct
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+except
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# no unnesting: UNION DISTINCT in second operand except in first
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+except
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+   union distinct
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+====
+---- QUERY
+# UNION ALL absorbs the children but not directly the operands
+# of a nested EXCEPT in the first operand
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)
+except
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+ union all
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+union all
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# UNION ALL absorbs the children but not directly the operands
+# of a nested EXCEPT in the second operand
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+union all
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+   union all
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# results if the except was unnested
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+union all
+  select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+   union all
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# no except unnesting: UNION ALL doesn't absorb the children of a nested union
+# with mixed except and limit, second operand is nested
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+union all
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+   union all
+   (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+   limit 10)
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# no except unnesting: UNION DISTINCT doesn't absorb nested EXCEPT
+# second operand is nested
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month = 1
+union distinct
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1)
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# unnested results
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month = 1
+union distinct
+  select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# Complex except no unnesting
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+except
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+   except
+     (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+      except
+        (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+         except
+         select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=3)))
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+4,true,0,0,0,0,0,0,'03/01/09','0',2009-03-01 00:00:00,2009,3
+5,false,1,1,1,10,1.100000023841858,10.1,'03/01/09','1',2009-03-01 00:01:00,2009,3
+====
+---- QUERY
+# EXCEPT in a subquery
+select x.id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2) x
+except
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+order by 1 limit 5
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# EXCEPT in subquery with a WHERE condition in the outer select.
+select x.id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2) x
+where x.int_col < 5 and x.bool_col = false
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# mixed with intersect
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3,4)
+intersect
+  select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+4,true,0,0,0,0,0,0,'03/01/09','0',2009-03-01 00:00:00,2009,3
+5,false,1,1,1,10,1.100000023841858,10.1,'03/01/09','1',2009-03-01 00:01:00,2009,3
+====
+---- QUERY
+# mixed with intersect with final order by bound to last operand, non-standard behavior see IMPALA-4741
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3,4)
+intersect
+  select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1 order by id desc limit 1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+4,true,0,0,0,0,0,0,'03/01/09','0',2009-03-01 00:00:00,2009,3
+5,false,1,1,1,10,1.100000023841858,10.1,'03/01/09','1',2009-03-01 00:01:00,2009,3
+====
+---- QUERY
+# mixed with intersect with final order by for the whole statement
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3,4)
+intersect
+  select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+   except
+   (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1) order by id desc limit 1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+5,false,1,1,1,10,1.100000023841858,10.1,'03/01/09','1',2009-03-01 00:01:00,2009,3
+====
+---- QUERY
+# Large table constant selects and values statements
+select count(*) from (
+  select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypes
+  except
+  values(0,true,0,0,0,0,cast(0 as float),0,'01/01/09','0',cast('2009-01-01 00:00:00' as timestamp),2009,1)
+  except
+  select 1,false,1,1,1,10,cast(1.1 as float),10.1,'01/01/09','1',cast('2009-01-01 00:01:00' as timestamp),2009,1
+  except
+  values(2,true,2,2,2,20,cast(2.2 as float),cast(20.2 as double),'01/01/09','2',cast('2009-01-01 00:02:00.10' as timestamp),2009,1)
+) x
+---- TYPES
+bigint
+---- RESULTS
+7297
+====
+---- QUERY
+# only constant selects
+select 1, 'a', NULL, 10.0
+except
+select 2, 'b', NULL, 20.0
+except
+select 3, 'c', NULL, 30.0
+---- TYPES
+tinyint, string, null, decimal
+---- HS2_TYPES
+tinyint, string, boolean, decimal
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+1,'a',NULL,10.0
+====
+---- QUERY
+# except with values statements
+values(1, 'a', NULL, 10.0)
+except
+values(2, 'b', NULL, 20.0)
+except
+values(3, 'c', NULL, 30.0)
+---- TYPES
+tinyint, string, null, decimal
+---- HS2_TYPES
+tinyint, string, boolean, decimal
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+1,'a',NULL,10.0
+====
+---- QUERY
+# Test EXCEPT with GROUP BY
+select id, count(*) from alltypes where id < 3 group by id
+except
+select 2, 3
+---- TYPES
+int, bigint
+---- RESULTS
+0,1
+1,1
+2,1
+====
+---- QUERY
+# join on string column + except
+select count(*) from
+(select 1 FROM alltypes AS t1 JOIN alltypestiny AS t2 ON t1.string_col = t2.string_col
+EXCEPT SELECT 2 FROM tinytable AS t1) as t3
+---- TYPES
+bigint
+---- RESULTS
+1
+====
+---- QUERY
+# Test except where all operands are dropped because of constant conjuncts.
+select * from
+  (select 1 a, 2 b
+   except
+   select 3, 4
+   except
+   select 10, 20) t
+where a > b
+---- TYPES
+tinyint, tinyint
+---- RESULTS
+====
+---- QUERY
+# IMPALA-4336: Test proper result expr casting when unnesting operands.
+(select 10) union select double_col from alltypestiny except (select 0 union all select 90)
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+10.1
+10
+---- TYPES
+DOUBLE
+=====
+---- QUERY
+select bigint_col + 1 from alltypestiny
+except
+select bigint_col + 1 from alltypestiny
+---- RESULTS
+---- TYPES
+bigint
+=====
+---- QUERY
+# One operand is passed through, the other is not.
+select bigint_col + 1 from alltypestiny
+except
+select bigint_col from alltypestiny
+---- RESULTS
+11
+1
+---- TYPES
+bigint
+=====
+---- QUERY
+# Test implicit casts. The label comes from the leftmost operands however the return
+# types should match the widest compatible types across operands, not just the leftmost
+select tinyint_col, float_col, string_col from alltypestiny where year=2009
+except
+select int_col, double_col, string_col from alltypestiny where year=2009 and month=1
+---- LABELS
+tinyint_col, float_col, string_col
+---- TYPES
+int, double, string
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+1,1.100000023841858,'1'
+====
+---- QUERY
+select tinyint_col from alltypesagg minus select tinyint_col from alltypes
+---- TYPES
+TINYINT
+---- RESULTS
+NULL
+====
+---- QUERY
+select tinyint_col from alltypesagg minus select tinyint_col from alltypesagg where month = 1
+---- TYPES
+TINYINT
+---- RESULTS
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/intersect.test b/testdata/workloads/functional-query/queries/QueryTest/intersect.test
new file mode 100644
index 0000000..9751ae4
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/intersect.test
@@ -0,0 +1,536 @@
+====
+---- QUERY
+select 100 union select 101 intersect select 101
+---- RESULTS
+101
+====
+---- QUERY
+select 100 intersect select 101 union select 101
+---- RESULTS
+101
+====
+---- QUERY
+select 100 intersect (select 100 union select 102)
+---- RESULTS
+100
+====
+---- QUERY
+select 100 intersect (select 100 union all select 102) intersect select 102
+---- RESULTS
+====
+---- QUERY
+select 100 intersect (select 100 intersect select 100)
+---- RESULTS
+100
+====
+---- QUERY
+# intersect distinct before unions all with limit 2
+select 100 intersect distinct (select 100 intersect select 100 union all select 100)
+  union all select 100 limit 2
+---- RESULTS
+100
+100
+====
+---- QUERY
+# intersect distinct after unions with limit 2 but deduped by the intersect coming after
+# the unions as opposed to above
+select 100 union all (select 100 intersect select 100 union all select 100)
+  intersect distinct select 100 limit 2
+---- RESULTS
+100
+====
+---- QUERY
+# nulls are equal
+select group_str, null_str from nullrows where bool_nulls = true intersect
+select group_str, null_str from nullrows where bool_nulls = false
+---- TYPES
+STRING, STRING
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+'a','NULL'
+'f','NULL'
+'k','NULL'
+'u','NULL'
+'p','NULL'
+====
+---- QUERY
+# nulls are equal
+select id, null_str from nullrows where bool_nulls = true intersect
+select id, some_nulls from nullrows where bool_nulls = false
+---- TYPES
+STRING, STRING
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+'a','NULL'
+'f','NULL'
+'k','NULL'
+'u','NULL'
+'p','NULL'
+---- QUERY
+# Subquery rewrites
+select id from alltypestiny where int_col in (select int_col from alltypessmall) intersect
+select id from alltypessmall where int_col < (select max(int_col) from alltypestiny)
+---- TYPES
+INT
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0
+====
+---- QUERY
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+intersect
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# intersect has set semantics so each row should be unique
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+union all
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+intersect
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year from alltypestiny where year=2009 and month=1 order by id limit 3-2
+intersect
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year from alltypestiny where year=2009 and month=1 order by int_col limit 1*1*1
+intersect
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year from alltypestiny where year=2009 and month=1 order by id limit 1&1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009
+====
+---- QUERY
+# INTERSECT only, mixed selects with and without from clauses, no nesting
+select id as  i, bool_col as bet, tinyint_col as the, smallint_col as song, int_col as `is`, bigint_col as about, float_col as food, double_col as yummy, date_string_col as chocolate, string_col as cake, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+intersect
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+intersect
+select 1,false,1,1,1,10,1.1,10.1,'01/01/09','1',cast('2009-01-01 00:01:00' as timestamp), 2009,1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- LABELS
+i, bet, the, song, is, about, food, yummy, chocolate, cake, timestamp_col, year, month
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# intersect with order by and limit
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+intersect
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)
+intersect
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1)
+order by 1 limit 1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+====
+---- QUERY
+# Mixed UNION ALL/DISTINCT, no nested unions, with order by and limit
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+union distinct
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+union all
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+union all
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+order by 1,2 limit 3
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+====
+---- QUERY
+# Mixed UNION ALL/DISTINCT, no nested unions, with order by and limit
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+union all
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+union distinct
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+union all
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+order by 1,2 limit 4
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+====
+---- QUERY
+# intersect not unnesting first operand
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)
+intersect
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1)
+intersect
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# intersect unnesting second operand is nested
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+intersect
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+   intersect
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2))
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# first operand intersect union all outside
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)
+ intersect
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+union all
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# first operand intersect and union all, union all outside pulls up
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)
+ intersect
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+ union all
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+union all
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# first operand intersect and union, union outside no pull up
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)
+ intersect
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+ union distinct
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+union distinct
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# intersect unnest second operand is nested
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+intersect
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (2,1)
+   intersect
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# written without nesting should have the same results as above
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+intersect
+  select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (2,1)
+   intersect
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# intersect unnest second operand is nested
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+intersect
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+intersect
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (2,1)
+   intersect
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2))
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# union all absorbing nested all with intersect
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+ intersect
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+ union all
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1)
+union all
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# union all absorbing nested all with intersect
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+union all
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (2,1)
+ intersect
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+ union all
+ select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1)
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# INTERSECT rewrites to inner join for when we have select distincts
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+intersect
+  (select distinct id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+   intersect
+   select distinct id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)
+   intersect
+   select distinct id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# INTERSECT rewrites to inner join for when we have select distincts
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+intersect
+  (select distinct id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+   intersect
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)
+   intersect
+   select distinct id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# intersect doesn't absorb nested with order by and limit,
+# second operand is nested
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+intersect
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+   intersect
+   (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+   order by 1 limit 1)
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+====
+---- QUERY
+# manually unnest intersect with order by and limit; results same as above
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+intersect
+  select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+   intersect
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+   order by 1 limit 1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+====
+---- QUERY
+# mixed with except nested
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+intersect
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month = 1
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+   intersect
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1)
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# mixed with except as above manually unnested
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+intersect
+  select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month = 1
+   except
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2
+   intersect
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,true,0,0,0,0,0,0,'01/01/09','0',2009-01-01 00:00:00,2009,1
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# Complex intersect unnesting
+select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+intersect
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)
+   intersect
+     (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)
+      intersect
+        (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+         intersect
+         select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)))
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+3,false,1,1,1,10,1.100000023841858,10.1,'02/01/09','1',2009-02-01 00:01:00,2009,2
+====
+---- QUERY
+# intersect in subquery
+select x.id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2,3)
+   intersect 
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month in (1,2)) x
+intersect
+(select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=2)
+order by 1 limit 1
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS
+2,true,0,0,0,0,0,0,'02/01/09','0',2009-02-01 00:00:00,2009,2
+====
+---- QUERY
+# INTERSET in subquery with a WHERE condition in the outer select.
+select x.id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from
+  (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1
+   intersect
+   select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from alltypestiny where year=2009 and month=1) x
+where x.int_col < 5 and x.bool_col = false
+---- TYPES
+int, boolean, tinyint, smallint, int, bigint, float, double, string, string, timestamp, int, int
+---- RESULTS
+1,false,1,1,1,10,1.100000023841858,10.1,'01/01/09','1',2009-01-01 00:01:00,2009,1
+====
+---- QUERY
+# INTERSECT with values statements
+values(1, 'a', NULL, 10.0)
+intersect
+values(1, 'a', NULL, 10.0)
+intersect
+values(1, 'a', NULL, 10.0)
+---- TYPES
+tinyint, string, null, decimal
+---- HS2_TYPES
+tinyint, string, boolean, decimal
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+1,'a',NULL,10.0
+====
+---- QUERY
+# Test INTERSECT with group by
+select id, count(*) from alltypes where id = 1 group by id
+intersect 
+select 1, 1
+---- TYPES
+int, bigint
+---- RESULTS
+1,1
+====
+---- QUERY
+select * from
+  (select 1 a, 2 b
+   intersect
+   select 2-1, 2
+   intersect
+   select 10/10, 20/10) t
+where b > a
+---- TYPES
+double, double
+---- RESULTS
+1,2
+====
+---- QUERY
+# Verify that limit is obeyed
+select bigint_col from alltypestiny where bigint_col >= 0
+intersect
+(select bigint_col from alltypestiny where bigint_col >= 0)
+order by bigint_col limit 1
+---- RESULTS
+0
+---- TYPES
+bigint
+=====
+---- QUERY
+# Input tuples that have non-nullable slots
+select count(*) from alltypestiny
+intersect
+select count(*) from alltypestiny
+---- RESULTS
+8
+---- TYPES
+bigint
+=====
+---- QUERY
+select bigint_col + 1 from alltypestiny
+intersect
+select bigint_col + 1 from alltypestiny
+---- RESULTS
+11
+1
+---- TYPES
+bigint
+=====
+---- QUERY
+# Test implicit casts. The labels comes from the leftmost operands however the return
+# types should match the widest compatible types across operands, not just the leftmost
+select tinyint_col, float_col, string_col from alltypestiny where year=2009 and month=1
+intersect
+select int_col, double_col, string_col from alltypestiny where year=2009 and month=1
+---- LABELS
+tinyint_col, float_col, string_col
+---- TYPES
+int, double, string
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+0,0,'0'
+====
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q14-1.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q14-1.test
new file mode 100644
index 0000000..343b7b9
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q14-1.test
@@ -0,0 +1,207 @@
+====
+---- QUERY: TPCDS-Q14-1
+with  cross_items as
+ (select i_item_sk ss_item_sk
+ from item,
+ (select iss.i_brand_id brand_id
+     ,iss.i_class_id class_id
+     ,iss.i_category_id category_id
+ from store_sales
+     ,item iss
+     ,date_dim d1
+ where ss_item_sk = iss.i_item_sk
+   and ss_sold_date_sk = d1.d_date_sk
+   and d1.d_year between 1999 AND 1999 + 2
+ intersect 
+ select ics.i_brand_id
+     ,ics.i_class_id
+     ,ics.i_category_id
+ from catalog_sales
+     ,item ics
+     ,date_dim d2
+ where cs_item_sk = ics.i_item_sk
+   and cs_sold_date_sk = d2.d_date_sk
+   and d2.d_year between 1999 AND 1999 + 2
+ intersect
+ select iws.i_brand_id
+     ,iws.i_class_id
+     ,iws.i_category_id
+ from web_sales
+     ,item iws
+     ,date_dim d3
+ where ws_item_sk = iws.i_item_sk
+   and ws_sold_date_sk = d3.d_date_sk
+   and d3.d_year between 1999 AND 1999 + 2) t1
+ where i_brand_id = brand_id
+      and i_class_id = class_id
+      and i_category_id = category_id
+),
+ avg_sales as
+ (select avg(quantity*list_price) average_sales
+  from (select ss_quantity quantity
+             ,ss_list_price list_price
+       from store_sales
+           ,date_dim
+       where ss_sold_date_sk = d_date_sk
+         and d_year between 1999 and 1999 + 2
+       union all 
+       select cs_quantity quantity 
+             ,cs_list_price list_price
+       from catalog_sales
+           ,date_dim
+       where cs_sold_date_sk = d_date_sk
+         and d_year between 1999 and 1999 + 2 
+       union all
+       select ws_quantity quantity
+             ,ws_list_price list_price
+       from web_sales
+           ,date_dim
+       where ws_sold_date_sk = d_date_sk
+         and d_year between 1999 and 1999 + 2) x)
+ select channel, i_brand_id,i_class_id,i_category_id,sum(sales), sum(number_sales)
+ from(
+       select 'store' channel, i_brand_id,i_class_id
+             ,i_category_id,sum(ss_quantity*ss_list_price) sales
+             , count(*) number_sales
+       from store_sales
+           ,item
+           ,date_dim
+       where ss_item_sk in (select ss_item_sk from cross_items)
+         and ss_item_sk = i_item_sk
+         and ss_sold_date_sk = d_date_sk
+         and d_year = 1999+2 
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)
+       union all
+       select 'catalog' channel, i_brand_id,i_class_id,i_category_id, sum(cs_quantity*cs_list_price) sales, count(*) number_sales
+       from catalog_sales
+           ,item
+           ,date_dim
+       where cs_item_sk in (select ss_item_sk from cross_items)
+         and cs_item_sk = i_item_sk
+         and cs_sold_date_sk = d_date_sk
+         and d_year = 1999+2 
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(cs_quantity*cs_list_price) > (select average_sales from avg_sales)
+       union all
+       select 'web' channel, i_brand_id,i_class_id,i_category_id, sum(ws_quantity*ws_list_price) sales , count(*) number_sales
+       from web_sales
+           ,item
+           ,date_dim
+       where ws_item_sk in (select ss_item_sk from cross_items)
+         and ws_item_sk = i_item_sk
+         and ws_sold_date_sk = d_date_sk
+         and d_year = 1999+2
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(ws_quantity*ws_list_price) > (select average_sales from avg_sales)
+ ) y
+ group by rollup (channel, i_brand_id,i_class_id,i_category_id)
+ order by channel,i_brand_id,i_class_id,i_category_id
+LIMIT 100
+---- RESULTS
+'catalog',1001001,1,1,115019.61,20
+'catalog',1001001,1,2,146344.47,27
+'catalog',1001001,1,3,22597.19,3
+'catalog',1001001,1,4,107555.43,23
+'catalog',1001001,1,5,122521.31,25
+'catalog',1001001,1,6,16883.97,3
+'catalog',1001001,1,7,46329.78,9
+'catalog',1001001,1,8,77861.85,13
+'catalog',1001001,1,9,99985.35,21
+'catalog',1001001,1,10,100105.28,23
+'catalog',1001001,1,NULL,855204.24,167
+'catalog',1001001,2,2,43967.97,7
+'catalog',1001001,2,3,68565.38,14
+'catalog',1001001,2,5,12633.87,3
+'catalog',1001001,2,NULL,125167.22,24
+'catalog',1001001,3,1,11100.79,5
+'catalog',1001001,3,2,60551.64,14
+'catalog',1001001,3,4,28455.23,4
+'catalog',1001001,3,6,36821.61,7
+'catalog',1001001,3,7,17250.82,6
+'catalog',1001001,3,8,14426.92,4
+'catalog',1001001,3,9,30078.07,3
+'catalog',1001001,3,NULL,198685.08,43
+'catalog',1001001,4,2,45473.85,13
+'catalog',1001001,4,3,16558.92,8
+'catalog',1001001,4,4,47553.20,10
+'catalog',1001001,4,NULL,109585.97,31
+'catalog',1001001,5,9,30112.11,12
+'catalog',1001001,5,10,29678.50,5
+'catalog',1001001,5,NULL,59790.61,17
+'catalog',1001001,6,9,10261.82,3
+'catalog',1001001,6,NULL,10261.82,3
+'catalog',1001001,7,7,18244.94,3
+'catalog',1001001,7,NULL,18244.94,3
+'catalog',1001001,8,7,28872.49,7
+'catalog',1001001,8,10,26895.97,6
+'catalog',1001001,8,NULL,55768.46,13
+'catalog',1001001,9,6,30944.19,5
+'catalog',1001001,9,NULL,30944.19,5
+'catalog',1001001,11,9,82810.87,12
+'catalog',1001001,11,NULL,82810.87,12
+'catalog',1001001,12,10,38427.52,9
+'catalog',1001001,12,NULL,38427.52,9
+'catalog',1001001,15,9,53508.79,7
+'catalog',1001001,15,10,59329.31,13
+'catalog',1001001,15,NULL,112838.10,20
+'catalog',1001001,NULL,NULL,1697729.02,347
+'catalog',1001002,1,1,2673969.89,530
+'catalog',1001002,1,NULL,2673969.89,530
+'catalog',1001002,2,1,140831.91,29
+'catalog',1001002,2,NULL,140831.91,29
+'catalog',1001002,3,1,320175.87,67
+'catalog',1001002,3,NULL,320175.87,67
+'catalog',1001002,4,1,133287.96,21
+'catalog',1001002,4,NULL,133287.96,21
+'catalog',1001002,5,1,16606.90,9
+'catalog',1001002,5,NULL,16606.90,9
+'catalog',1001002,6,1,15133.01,4
+'catalog',1001002,6,NULL,15133.01,4
+'catalog',1001002,7,1,24471.26,10
+'catalog',1001002,7,NULL,24471.26,10
+'catalog',1001002,8,1,63773.05,12
+'catalog',1001002,8,NULL,63773.05,12
+'catalog',1001002,9,1,9167.19,3
+'catalog',1001002,9,NULL,9167.19,3
+'catalog',1001002,12,1,29108.42,7
+'catalog',1001002,12,NULL,29108.42,7
+'catalog',1001002,15,1,31143.45,6
+'catalog',1001002,15,NULL,31143.45,6
+'catalog',1001002,16,1,70162.42,8
+'catalog',1001002,16,NULL,70162.42,8
+'catalog',1001002,NULL,NULL,3527831.33,706
+'catalog',1002001,1,1,76392.13,14
+'catalog',1002001,1,2,118394.33,21
+'catalog',1002001,1,4,29395.79,5
+'catalog',1002001,1,5,35541.97,4
+'catalog',1002001,1,6,26104.36,3
+'catalog',1002001,1,9,18793.97,4
+'catalog',1002001,1,10,44071.42,4
+'catalog',1002001,1,NULL,348693.97,55
+'catalog',1002001,2,1,239511.02,51
+'catalog',1002001,2,2,147993.14,26
+'catalog',1002001,2,3,100086.93,17
+'catalog',1002001,2,4,53524.42,13
+'catalog',1002001,2,5,48494.06,10
+'catalog',1002001,2,6,142857.04,20
+'catalog',1002001,2,7,116557.98,16
+'catalog',1002001,2,8,92743.93,24
+'catalog',1002001,2,9,203943.99,38
+'catalog',1002001,2,10,88249.19,10
+'catalog',1002001,2,NULL,1233961.70,225
+'catalog',1002001,3,2,25171.13,6
+'catalog',1002001,3,7,27766.70,3
+'catalog',1002001,3,8,38116.49,8
+'catalog',1002001,3,NULL,91054.32,17
+'catalog',1002001,4,1,66896.68,15
+'catalog',1002001,4,3,43672.63,7
+'catalog',1002001,4,4,41393.52,6
+'catalog',1002001,4,5,30464.86,4
+'catalog',1002001,4,NULL,182427.69,32
+---- TYPES
+STRING, INT, INT, INT, DECIMAL, BIGINT
+====
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q14-2.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q14-2.test
new file mode 100644
index 0000000..adaeac9
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q14-2.test
@@ -0,0 +1,231 @@
+====
+---- QUERY: TPCDS-Q14-2
+WITH cross_items AS
+  (SELECT i_item_sk ss_item_sk
+   FROM item,
+     (SELECT iss.i_brand_id brand_id,
+             iss.i_class_id class_id,
+             iss.i_category_id category_id
+      FROM store_sales,
+           item iss,
+           date_dim d1
+      WHERE ss_item_sk = iss.i_item_sk
+        AND ss_sold_date_sk = d1.d_date_sk
+        AND d1.d_year BETWEEN 1999 AND 1999 + 2 INTERSECT
+        SELECT ics.i_brand_id,
+               ics.i_class_id,
+               ics.i_category_id
+        FROM catalog_sales,
+             item ics,
+             date_dim d2 WHERE cs_item_sk = ics.i_item_sk
+        AND cs_sold_date_sk = d2.d_date_sk
+        AND d2.d_year BETWEEN 1999 AND 1999 + 2 INTERSECT
+        SELECT iws.i_brand_id,
+               iws.i_class_id,
+               iws.i_category_id
+        FROM web_sales,
+             item iws,
+             date_dim d3 WHERE ws_item_sk = iws.i_item_sk
+        AND ws_sold_date_sk = d3.d_date_sk
+        AND d3.d_year BETWEEN 1999 AND 1999 + 2) x
+   WHERE i_brand_id = brand_id
+     AND i_class_id = class_id
+     AND i_category_id = category_id ),
+     avg_sales AS
+  (SELECT avg(quantity*list_price) average_sales
+   FROM
+     (SELECT ss_quantity quantity,
+             ss_list_price list_price
+      FROM store_sales,
+           date_dim
+      WHERE ss_sold_date_sk = d_date_sk
+        AND d_year BETWEEN 1999 AND 1999 + 2
+      UNION ALL SELECT cs_quantity quantity,
+                       cs_list_price list_price
+      FROM catalog_sales,
+           date_dim
+      WHERE cs_sold_date_sk = d_date_sk
+        AND d_year BETWEEN 1999 AND 1999 + 2
+      UNION ALL SELECT ws_quantity quantity,
+                       ws_list_price list_price
+      FROM web_sales,
+           date_dim
+      WHERE ws_sold_date_sk = d_date_sk
+        AND d_year BETWEEN 1999 AND 1999 + 2) x)
+SELECT this_year.channel ty_channel,
+       this_year.i_brand_id ty_brand,
+       this_year.i_class_id ty_class,
+       this_year.i_category_id ty_category,
+       this_year.sales ty_sales,
+       this_year.number_sales ty_number_sales,
+       last_year.channel ly_channel,
+       last_year.i_brand_id ly_brand,
+       last_year.i_class_id ly_class,
+       last_year.i_category_id ly_category,
+       last_year.sales ly_sales,
+       last_year.number_sales ly_number_sales
+FROM
+  (SELECT 'store' channel,
+                  i_brand_id,
+                  i_class_id,
+                  i_category_id,
+                  sum(ss_quantity*ss_list_price) sales,
+                  count(*) number_sales
+   FROM store_sales,
+        item,
+        date_dim
+   WHERE ss_item_sk IN
+       (SELECT ss_item_sk
+        FROM cross_items)
+     AND ss_item_sk = i_item_sk
+     AND ss_sold_date_sk = d_date_sk
+     AND d_week_seq =
+       (SELECT d_week_seq
+        FROM date_dim
+        WHERE d_year = 1999 + 1
+          AND d_moy = 12
+          AND d_dom = 11)
+   GROUP BY i_brand_id,
+            i_class_id,
+            i_category_id
+   HAVING sum(ss_quantity*ss_list_price) >
+     (SELECT average_sales
+      FROM avg_sales)) this_year,
+  (SELECT 'store' channel,
+                  i_brand_id,
+                  i_class_id,
+                  i_category_id,
+                  sum(ss_quantity*ss_list_price) sales,
+                  count(*) number_sales
+   FROM store_sales,
+        item,
+        date_dim
+   WHERE ss_item_sk IN
+       (SELECT ss_item_sk
+        FROM cross_items)
+     AND ss_item_sk = i_item_sk
+     AND ss_sold_date_sk = d_date_sk
+     AND d_week_seq =
+       (SELECT d_week_seq
+        FROM date_dim
+        WHERE d_year = 1999
+          AND d_moy = 12
+          AND d_dom = 11)
+   GROUP BY i_brand_id,
+            i_class_id,
+            i_category_id
+   HAVING sum(ss_quantity*ss_list_price) >
+     (SELECT average_sales
+      FROM avg_sales)) last_year
+WHERE this_year.i_brand_id= last_year.i_brand_id
+  AND this_year.i_class_id = last_year.i_class_id
+  AND this_year.i_category_id = last_year.i_category_id
+ORDER BY this_year.channel,
+         this_year.i_brand_id,
+         this_year.i_class_id,
+         this_year.i_category_id
+LIMIT 100
+---- RESULTS
+'store',1001001,1,1,372549.59,137,'store',1001001,1,1,1316197.43,340
+'store',1001002,1,1,799646.86,229,'store',1001002,1,1,775776.63,188
+'store',1002001,2,1,765555.91,202,'store',1002001,2,1,1218310.82,337
+'store',1002002,2,1,1030666.65,250,'store',1002002,2,1,605616.79,166
+'store',1003001,3,1,547136.96,167,'store',1003001,3,1,1172847.07,341
+'store',1003002,3,1,674007.12,188,'store',1003002,3,1,607135.04,154
+'store',1004001,4,1,682747.23,187,'store',1004001,4,1,1171699.76,311
+'store',1004002,4,1,739362.16,210,'store',1004002,4,1,649347.19,184
+'store',2001001,1,2,688211.08,216,'store',2001001,1,2,1342495.45,365
+'store',2001002,1,2,906330.70,237,'store',2001002,1,2,541063.50,153
+'store',2002001,2,2,868177.87,211,'store',2002001,2,2,1514792.19,402
+'store',2002002,2,2,648459.06,179,'store',2002002,2,2,588040.22,159
+'store',2003001,3,2,559505.35,151,'store',2003001,3,2,1171821.55,300
+'store',2003002,3,2,746714.11,219,'store',2003002,3,2,695877.16,171
+'store',2004001,4,2,772046.77,235,'store',2004001,4,2,1371151.70,362
+'store',2004002,4,2,685029.29,192,'store',2004002,4,2,646760.33,187
+'store',3001001,1,3,596684.04,165,'store',3001001,1,3,1059529.81,312
+'store',3001002,1,3,795882.47,202,'store',3001002,1,3,494739.47,142
+'store',3002001,2,3,661760.58,181,'store',3002001,2,3,1411948.74,380
+'store',3002002,2,3,864250.11,244,'store',3002002,2,3,630689.30,183
+'store',3003001,3,3,749054.16,224,'store',3003001,3,3,1513634.71,381
+'store',3003002,3,3,737460.71,194,'store',3003002,3,3,713930.86,164
+'store',3004001,4,3,508491.02,161,'store',3004001,4,3,1082215.49,293
+'store',3004002,4,3,867362.83,220,'store',3004002,4,3,630978.56,172
+'store',4001001,1,4,847845.40,207,'store',4001001,1,4,1291518.97,340
+'store',4001002,1,4,841877.80,232,'store',4001002,1,4,520226.86,143
+'store',4002001,2,4,543692.20,162,'store',4002001,2,4,1378081.90,362
+'store',4002002,2,4,912449.68,256,'store',4002002,2,4,713528.38,200
+'store',4003001,3,4,725565.22,189,'store',4003001,3,4,1285387.49,311
+'store',4003002,3,4,686780.88,197,'store',4003002,3,4,623656.88,160
+'store',4004001,4,4,665419.78,170,'store',4004001,4,4,1141222.24,334
+'store',4004002,4,4,866534.14,204,'store',4004002,4,4,644963.98,168
+'store',5001001,1,5,626379.58,167,'store',5001001,1,5,1258495.82,336
+'store',5001002,1,5,813592.70,198,'store',5001002,1,5,671017.15,179
+'store',5002001,2,5,468954.69,153,'store',5002001,2,5,1195162.42,297
+'store',5002002,2,5,885968.60,211,'store',5002002,2,5,691132.99,183
+'store',5003001,3,5,709102.64,206,'store',5003001,3,5,1369869.31,375
+'store',5003002,3,5,1019553.87,266,'store',5003002,3,5,671873.72,181
+'store',5004001,4,5,801416.97,219,'store',5004001,4,5,1405232.18,371
+'store',5004002,4,5,1061684.64,273,'store',5004002,4,5,790288.63,209
+'store',6001001,1,6,24105.89,9,'store',6001001,1,6,43566.03,11
+'store',6001002,1,6,40621.71,10,'store',6001002,1,6,31684.03,10
+'store',6001003,1,6,23365.00,8,'store',6001003,1,6,25117.82,10
+'store',6001004,1,6,29458.48,6,'store',6001004,1,6,29630.40,7
+'store',6001005,1,6,40555.76,13,'store',6001005,1,6,67204.16,14
+'store',6001006,1,6,49871.82,10,'store',6001006,1,6,17872.80,6
+'store',6001007,1,6,23526.21,8,'store',6001007,1,6,34341.06,13
+'store',6001008,1,6,74782.90,20,'store',6001008,1,6,127415.74,33
+'store',6002001,2,6,55244.23,15,'store',6002001,2,6,67018.71,20
+'store',6002002,2,6,24045.72,11,'store',6002002,2,6,73776.49,17
+'store',6002004,2,6,45590.44,15,'store',6002004,2,6,13288.01,5
+'store',6002005,2,6,53619.89,11,'store',6002005,2,6,57765.00,12
+'store',6002006,2,6,36339.64,12,'store',6002006,2,6,28988.99,8
+'store',6002007,2,6,33520.91,9,'store',6002007,2,6,55473.59,15
+'store',6002008,2,6,24710.01,7,'store',6002008,2,6,25085.33,11
+'store',6003001,3,6,33165.63,9,'store',6003001,3,6,84144.62,18
+'store',6003002,3,6,52293.57,15,'store',6003002,3,6,59151.70,15
+'store',6003003,3,6,33342.98,7,'store',6003003,3,6,43720.13,20
+'store',6003004,3,6,26344.41,9,'store',6003004,3,6,30930.85,9
+'store',6003005,3,6,89998.03,26,'store',6003005,3,6,95491.00,29
+'store',6003006,3,6,31862.34,10,'store',6003006,3,6,24121.07,3
+'store',6003007,3,6,92405.03,24,'store',6003007,3,6,129219.46,31
+'store',6003008,3,6,46145.62,13,'store',6003008,3,6,41948.28,19
+'store',6004001,4,6,12839.97,5,'store',6004001,4,6,74616.32,16
+'store',6004002,4,6,33700.51,11,'store',6004002,4,6,45202.40,11
+'store',6004003,4,6,31387.04,12,'store',6004003,4,6,103532.44,21
+'store',6004004,4,6,42644.68,8,'store',6004004,4,6,15287.71,6
+'store',6004005,4,6,15515.22,3,'store',6004005,4,6,60172.67,16
+'store',6004006,4,6,60466.48,15,'store',6004006,4,6,38426.04,10
+'store',6004007,4,6,48158.78,12,'store',6004007,4,6,55265.06,13
+'store',6004008,4,6,110528.92,26,'store',6004008,4,6,17409.99,6
+'store',6005001,5,6,38589.67,11,'store',6005001,5,6,135393.23,42
+'store',6005002,5,6,47495.55,8,'store',6005002,5,6,49267.29,12
+'store',6005003,5,6,14884.61,3,'store',6005003,5,6,74851.15,16
+'store',6005005,5,6,37483.24,12,'store',6005005,5,6,73876.82,18
+'store',6005006,5,6,61699.19,17,'store',6005006,5,6,52082.17,11
+'store',6005007,5,6,18764.89,5,'store',6005007,5,6,48445.43,16
+'store',6005008,5,6,25927.83,9,'store',6005008,5,6,16473.68,7
+'store',6006001,6,6,36995.31,14,'store',6006001,6,6,69739.81,24
+'store',6006002,6,6,85385.16,22,'store',6006002,6,6,87585.47,23
+'store',6006003,6,6,47063.87,11,'store',6006003,6,6,114431.48,29
+'store',6006004,6,6,20527.50,8,'store',6006004,6,6,23213.37,3
+'store',6006005,6,6,7442.61,4,'store',6006005,6,6,93912.14,18
+'store',6006006,6,6,25335.02,8,'store',6006006,6,6,24102.43,5
+'store',6006007,6,6,12901.85,6,'store',6006007,6,6,107194.04,30
+'store',6006008,6,6,39533.23,14,'store',6006008,6,6,48680.37,14
+'store',6007001,7,6,9032.59,3,'store',6007001,7,6,15183.24,6
+'store',6007002,7,6,57662.38,13,'store',6007002,7,6,42989.39,13
+'store',6007003,7,6,44207.42,17,'store',6007003,7,6,141091.04,39
+'store',6007004,7,6,147787.95,29,'store',6007004,7,6,69437.35,16
+'store',6007005,7,6,33681.87,9,'store',6007005,7,6,49926.37,15
+'store',6007006,7,6,50875.60,16,'store',6007006,7,6,75381.63,24
+'store',6007007,7,6,13622.22,7,'store',6007007,7,6,102650.94,31
+'store',6007008,7,6,34697.31,12,'store',6007008,7,6,49841.13,18
+'store',6008001,8,6,44993.33,10,'store',6008001,8,6,57972.98,22
+'store',6008002,8,6,39437.31,8,'store',6008002,8,6,48434.65,14
+'store',6008003,8,6,59905.77,14,'store',6008003,8,6,60930.66,20
+'store',6008004,8,6,36676.25,9,'store',6008004,8,6,31817.07,9
+'store',6008005,8,6,84238.28,27,'store',6008005,8,6,136071.70,27
+'store',6008007,8,6,14672.77,7,'store',6008007,8,6,59474.21,18
+---- TYPES
+STRING,INT,INT,INT,DECIMAL,BIGINT,STRING,INT,INT,INT,DECIMAL,BIGINT
+====
diff --git a/testdata/workloads/tpcds/queries/tpcds-q14-1.test b/testdata/workloads/tpcds/queries/tpcds-q14-1.test
new file mode 100644
index 0000000..343b7b9
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q14-1.test
@@ -0,0 +1,207 @@
+====
+---- QUERY: TPCDS-Q14-1
+with  cross_items as
+ (select i_item_sk ss_item_sk
+ from item,
+ (select iss.i_brand_id brand_id
+     ,iss.i_class_id class_id
+     ,iss.i_category_id category_id
+ from store_sales
+     ,item iss
+     ,date_dim d1
+ where ss_item_sk = iss.i_item_sk
+   and ss_sold_date_sk = d1.d_date_sk
+   and d1.d_year between 1999 AND 1999 + 2
+ intersect 
+ select ics.i_brand_id
+     ,ics.i_class_id
+     ,ics.i_category_id
+ from catalog_sales
+     ,item ics
+     ,date_dim d2
+ where cs_item_sk = ics.i_item_sk
+   and cs_sold_date_sk = d2.d_date_sk
+   and d2.d_year between 1999 AND 1999 + 2
+ intersect
+ select iws.i_brand_id
+     ,iws.i_class_id
+     ,iws.i_category_id
+ from web_sales
+     ,item iws
+     ,date_dim d3
+ where ws_item_sk = iws.i_item_sk
+   and ws_sold_date_sk = d3.d_date_sk
+   and d3.d_year between 1999 AND 1999 + 2) t1
+ where i_brand_id = brand_id
+      and i_class_id = class_id
+      and i_category_id = category_id
+),
+ avg_sales as
+ (select avg(quantity*list_price) average_sales
+  from (select ss_quantity quantity
+             ,ss_list_price list_price
+       from store_sales
+           ,date_dim
+       where ss_sold_date_sk = d_date_sk
+         and d_year between 1999 and 1999 + 2
+       union all 
+       select cs_quantity quantity 
+             ,cs_list_price list_price
+       from catalog_sales
+           ,date_dim
+       where cs_sold_date_sk = d_date_sk
+         and d_year between 1999 and 1999 + 2 
+       union all
+       select ws_quantity quantity
+             ,ws_list_price list_price
+       from web_sales
+           ,date_dim
+       where ws_sold_date_sk = d_date_sk
+         and d_year between 1999 and 1999 + 2) x)
+ select channel, i_brand_id,i_class_id,i_category_id,sum(sales), sum(number_sales)
+ from(
+       select 'store' channel, i_brand_id,i_class_id
+             ,i_category_id,sum(ss_quantity*ss_list_price) sales
+             , count(*) number_sales
+       from store_sales
+           ,item
+           ,date_dim
+       where ss_item_sk in (select ss_item_sk from cross_items)
+         and ss_item_sk = i_item_sk
+         and ss_sold_date_sk = d_date_sk
+         and d_year = 1999+2 
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)
+       union all
+       select 'catalog' channel, i_brand_id,i_class_id,i_category_id, sum(cs_quantity*cs_list_price) sales, count(*) number_sales
+       from catalog_sales
+           ,item
+           ,date_dim
+       where cs_item_sk in (select ss_item_sk from cross_items)
+         and cs_item_sk = i_item_sk
+         and cs_sold_date_sk = d_date_sk
+         and d_year = 1999+2 
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(cs_quantity*cs_list_price) > (select average_sales from avg_sales)
+       union all
+       select 'web' channel, i_brand_id,i_class_id,i_category_id, sum(ws_quantity*ws_list_price) sales , count(*) number_sales
+       from web_sales
+           ,item
+           ,date_dim
+       where ws_item_sk in (select ss_item_sk from cross_items)
+         and ws_item_sk = i_item_sk
+         and ws_sold_date_sk = d_date_sk
+         and d_year = 1999+2
+         and d_moy = 11
+       group by i_brand_id,i_class_id,i_category_id
+       having sum(ws_quantity*ws_list_price) > (select average_sales from avg_sales)
+ ) y
+ group by rollup (channel, i_brand_id,i_class_id,i_category_id)
+ order by channel,i_brand_id,i_class_id,i_category_id
+LIMIT 100
+---- RESULTS
+'catalog',1001001,1,1,115019.61,20
+'catalog',1001001,1,2,146344.47,27
+'catalog',1001001,1,3,22597.19,3
+'catalog',1001001,1,4,107555.43,23
+'catalog',1001001,1,5,122521.31,25
+'catalog',1001001,1,6,16883.97,3
+'catalog',1001001,1,7,46329.78,9
+'catalog',1001001,1,8,77861.85,13
+'catalog',1001001,1,9,99985.35,21
+'catalog',1001001,1,10,100105.28,23
+'catalog',1001001,1,NULL,855204.24,167
+'catalog',1001001,2,2,43967.97,7
+'catalog',1001001,2,3,68565.38,14
+'catalog',1001001,2,5,12633.87,3
+'catalog',1001001,2,NULL,125167.22,24
+'catalog',1001001,3,1,11100.79,5
+'catalog',1001001,3,2,60551.64,14
+'catalog',1001001,3,4,28455.23,4
+'catalog',1001001,3,6,36821.61,7
+'catalog',1001001,3,7,17250.82,6
+'catalog',1001001,3,8,14426.92,4
+'catalog',1001001,3,9,30078.07,3
+'catalog',1001001,3,NULL,198685.08,43
+'catalog',1001001,4,2,45473.85,13
+'catalog',1001001,4,3,16558.92,8
+'catalog',1001001,4,4,47553.20,10
+'catalog',1001001,4,NULL,109585.97,31
+'catalog',1001001,5,9,30112.11,12
+'catalog',1001001,5,10,29678.50,5
+'catalog',1001001,5,NULL,59790.61,17
+'catalog',1001001,6,9,10261.82,3
+'catalog',1001001,6,NULL,10261.82,3
+'catalog',1001001,7,7,18244.94,3
+'catalog',1001001,7,NULL,18244.94,3
+'catalog',1001001,8,7,28872.49,7
+'catalog',1001001,8,10,26895.97,6
+'catalog',1001001,8,NULL,55768.46,13
+'catalog',1001001,9,6,30944.19,5
+'catalog',1001001,9,NULL,30944.19,5
+'catalog',1001001,11,9,82810.87,12
+'catalog',1001001,11,NULL,82810.87,12
+'catalog',1001001,12,10,38427.52,9
+'catalog',1001001,12,NULL,38427.52,9
+'catalog',1001001,15,9,53508.79,7
+'catalog',1001001,15,10,59329.31,13
+'catalog',1001001,15,NULL,112838.10,20
+'catalog',1001001,NULL,NULL,1697729.02,347
+'catalog',1001002,1,1,2673969.89,530
+'catalog',1001002,1,NULL,2673969.89,530
+'catalog',1001002,2,1,140831.91,29
+'catalog',1001002,2,NULL,140831.91,29
+'catalog',1001002,3,1,320175.87,67
+'catalog',1001002,3,NULL,320175.87,67
+'catalog',1001002,4,1,133287.96,21
+'catalog',1001002,4,NULL,133287.96,21
+'catalog',1001002,5,1,16606.90,9
+'catalog',1001002,5,NULL,16606.90,9
+'catalog',1001002,6,1,15133.01,4
+'catalog',1001002,6,NULL,15133.01,4
+'catalog',1001002,7,1,24471.26,10
+'catalog',1001002,7,NULL,24471.26,10
+'catalog',1001002,8,1,63773.05,12
+'catalog',1001002,8,NULL,63773.05,12
+'catalog',1001002,9,1,9167.19,3
+'catalog',1001002,9,NULL,9167.19,3
+'catalog',1001002,12,1,29108.42,7
+'catalog',1001002,12,NULL,29108.42,7
+'catalog',1001002,15,1,31143.45,6
+'catalog',1001002,15,NULL,31143.45,6
+'catalog',1001002,16,1,70162.42,8
+'catalog',1001002,16,NULL,70162.42,8
+'catalog',1001002,NULL,NULL,3527831.33,706
+'catalog',1002001,1,1,76392.13,14
+'catalog',1002001,1,2,118394.33,21
+'catalog',1002001,1,4,29395.79,5
+'catalog',1002001,1,5,35541.97,4
+'catalog',1002001,1,6,26104.36,3
+'catalog',1002001,1,9,18793.97,4
+'catalog',1002001,1,10,44071.42,4
+'catalog',1002001,1,NULL,348693.97,55
+'catalog',1002001,2,1,239511.02,51
+'catalog',1002001,2,2,147993.14,26
+'catalog',1002001,2,3,100086.93,17
+'catalog',1002001,2,4,53524.42,13
+'catalog',1002001,2,5,48494.06,10
+'catalog',1002001,2,6,142857.04,20
+'catalog',1002001,2,7,116557.98,16
+'catalog',1002001,2,8,92743.93,24
+'catalog',1002001,2,9,203943.99,38
+'catalog',1002001,2,10,88249.19,10
+'catalog',1002001,2,NULL,1233961.70,225
+'catalog',1002001,3,2,25171.13,6
+'catalog',1002001,3,7,27766.70,3
+'catalog',1002001,3,8,38116.49,8
+'catalog',1002001,3,NULL,91054.32,17
+'catalog',1002001,4,1,66896.68,15
+'catalog',1002001,4,3,43672.63,7
+'catalog',1002001,4,4,41393.52,6
+'catalog',1002001,4,5,30464.86,4
+'catalog',1002001,4,NULL,182427.69,32
+---- TYPES
+STRING, INT, INT, INT, DECIMAL, BIGINT
+====
diff --git a/testdata/workloads/tpcds/queries/tpcds-q14-2.test b/testdata/workloads/tpcds/queries/tpcds-q14-2.test
new file mode 100644
index 0000000..adaeac9
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q14-2.test
@@ -0,0 +1,231 @@
+====
+---- QUERY: TPCDS-Q14-2
+WITH cross_items AS
+  (SELECT i_item_sk ss_item_sk
+   FROM item,
+     (SELECT iss.i_brand_id brand_id,
+             iss.i_class_id class_id,
+             iss.i_category_id category_id
+      FROM store_sales,
+           item iss,
+           date_dim d1
+      WHERE ss_item_sk = iss.i_item_sk
+        AND ss_sold_date_sk = d1.d_date_sk
+        AND d1.d_year BETWEEN 1999 AND 1999 + 2 INTERSECT
+        SELECT ics.i_brand_id,
+               ics.i_class_id,
+               ics.i_category_id
+        FROM catalog_sales,
+             item ics,
+             date_dim d2 WHERE cs_item_sk = ics.i_item_sk
+        AND cs_sold_date_sk = d2.d_date_sk
+        AND d2.d_year BETWEEN 1999 AND 1999 + 2 INTERSECT
+        SELECT iws.i_brand_id,
+               iws.i_class_id,
+               iws.i_category_id
+        FROM web_sales,
+             item iws,
+             date_dim d3 WHERE ws_item_sk = iws.i_item_sk
+        AND ws_sold_date_sk = d3.d_date_sk
+        AND d3.d_year BETWEEN 1999 AND 1999 + 2) x
+   WHERE i_brand_id = brand_id
+     AND i_class_id = class_id
+     AND i_category_id = category_id ),
+     avg_sales AS
+  (SELECT avg(quantity*list_price) average_sales
+   FROM
+     (SELECT ss_quantity quantity,
+             ss_list_price list_price
+      FROM store_sales,
+           date_dim
+      WHERE ss_sold_date_sk = d_date_sk
+        AND d_year BETWEEN 1999 AND 1999 + 2
+      UNION ALL SELECT cs_quantity quantity,
+                       cs_list_price list_price
+      FROM catalog_sales,
+           date_dim
+      WHERE cs_sold_date_sk = d_date_sk
+        AND d_year BETWEEN 1999 AND 1999 + 2
+      UNION ALL SELECT ws_quantity quantity,
+                       ws_list_price list_price
+      FROM web_sales,
+           date_dim
+      WHERE ws_sold_date_sk = d_date_sk
+        AND d_year BETWEEN 1999 AND 1999 + 2) x)
+SELECT this_year.channel ty_channel,
+       this_year.i_brand_id ty_brand,
+       this_year.i_class_id ty_class,
+       this_year.i_category_id ty_category,
+       this_year.sales ty_sales,
+       this_year.number_sales ty_number_sales,
+       last_year.channel ly_channel,
+       last_year.i_brand_id ly_brand,
+       last_year.i_class_id ly_class,
+       last_year.i_category_id ly_category,
+       last_year.sales ly_sales,
+       last_year.number_sales ly_number_sales
+FROM
+  (SELECT 'store' channel,
+                  i_brand_id,
+                  i_class_id,
+                  i_category_id,
+                  sum(ss_quantity*ss_list_price) sales,
+                  count(*) number_sales
+   FROM store_sales,
+        item,
+        date_dim
+   WHERE ss_item_sk IN
+       (SELECT ss_item_sk
+        FROM cross_items)
+     AND ss_item_sk = i_item_sk
+     AND ss_sold_date_sk = d_date_sk
+     AND d_week_seq =
+       (SELECT d_week_seq
+        FROM date_dim
+        WHERE d_year = 1999 + 1
+          AND d_moy = 12
+          AND d_dom = 11)
+   GROUP BY i_brand_id,
+            i_class_id,
+            i_category_id
+   HAVING sum(ss_quantity*ss_list_price) >
+     (SELECT average_sales
+      FROM avg_sales)) this_year,
+  (SELECT 'store' channel,
+                  i_brand_id,
+                  i_class_id,
+                  i_category_id,
+                  sum(ss_quantity*ss_list_price) sales,
+                  count(*) number_sales
+   FROM store_sales,
+        item,
+        date_dim
+   WHERE ss_item_sk IN
+       (SELECT ss_item_sk
+        FROM cross_items)
+     AND ss_item_sk = i_item_sk
+     AND ss_sold_date_sk = d_date_sk
+     AND d_week_seq =
+       (SELECT d_week_seq
+        FROM date_dim
+        WHERE d_year = 1999
+          AND d_moy = 12
+          AND d_dom = 11)
+   GROUP BY i_brand_id,
+            i_class_id,
+            i_category_id
+   HAVING sum(ss_quantity*ss_list_price) >
+     (SELECT average_sales
+      FROM avg_sales)) last_year
+WHERE this_year.i_brand_id= last_year.i_brand_id
+  AND this_year.i_class_id = last_year.i_class_id
+  AND this_year.i_category_id = last_year.i_category_id
+ORDER BY this_year.channel,
+         this_year.i_brand_id,
+         this_year.i_class_id,
+         this_year.i_category_id
+LIMIT 100
+---- RESULTS
+'store',1001001,1,1,372549.59,137,'store',1001001,1,1,1316197.43,340
+'store',1001002,1,1,799646.86,229,'store',1001002,1,1,775776.63,188
+'store',1002001,2,1,765555.91,202,'store',1002001,2,1,1218310.82,337
+'store',1002002,2,1,1030666.65,250,'store',1002002,2,1,605616.79,166
+'store',1003001,3,1,547136.96,167,'store',1003001,3,1,1172847.07,341
+'store',1003002,3,1,674007.12,188,'store',1003002,3,1,607135.04,154
+'store',1004001,4,1,682747.23,187,'store',1004001,4,1,1171699.76,311
+'store',1004002,4,1,739362.16,210,'store',1004002,4,1,649347.19,184
+'store',2001001,1,2,688211.08,216,'store',2001001,1,2,1342495.45,365
+'store',2001002,1,2,906330.70,237,'store',2001002,1,2,541063.50,153
+'store',2002001,2,2,868177.87,211,'store',2002001,2,2,1514792.19,402
+'store',2002002,2,2,648459.06,179,'store',2002002,2,2,588040.22,159
+'store',2003001,3,2,559505.35,151,'store',2003001,3,2,1171821.55,300
+'store',2003002,3,2,746714.11,219,'store',2003002,3,2,695877.16,171
+'store',2004001,4,2,772046.77,235,'store',2004001,4,2,1371151.70,362
+'store',2004002,4,2,685029.29,192,'store',2004002,4,2,646760.33,187
+'store',3001001,1,3,596684.04,165,'store',3001001,1,3,1059529.81,312
+'store',3001002,1,3,795882.47,202,'store',3001002,1,3,494739.47,142
+'store',3002001,2,3,661760.58,181,'store',3002001,2,3,1411948.74,380
+'store',3002002,2,3,864250.11,244,'store',3002002,2,3,630689.30,183
+'store',3003001,3,3,749054.16,224,'store',3003001,3,3,1513634.71,381
+'store',3003002,3,3,737460.71,194,'store',3003002,3,3,713930.86,164
+'store',3004001,4,3,508491.02,161,'store',3004001,4,3,1082215.49,293
+'store',3004002,4,3,867362.83,220,'store',3004002,4,3,630978.56,172
+'store',4001001,1,4,847845.40,207,'store',4001001,1,4,1291518.97,340
+'store',4001002,1,4,841877.80,232,'store',4001002,1,4,520226.86,143
+'store',4002001,2,4,543692.20,162,'store',4002001,2,4,1378081.90,362
+'store',4002002,2,4,912449.68,256,'store',4002002,2,4,713528.38,200
+'store',4003001,3,4,725565.22,189,'store',4003001,3,4,1285387.49,311
+'store',4003002,3,4,686780.88,197,'store',4003002,3,4,623656.88,160
+'store',4004001,4,4,665419.78,170,'store',4004001,4,4,1141222.24,334
+'store',4004002,4,4,866534.14,204,'store',4004002,4,4,644963.98,168
+'store',5001001,1,5,626379.58,167,'store',5001001,1,5,1258495.82,336
+'store',5001002,1,5,813592.70,198,'store',5001002,1,5,671017.15,179
+'store',5002001,2,5,468954.69,153,'store',5002001,2,5,1195162.42,297
+'store',5002002,2,5,885968.60,211,'store',5002002,2,5,691132.99,183
+'store',5003001,3,5,709102.64,206,'store',5003001,3,5,1369869.31,375
+'store',5003002,3,5,1019553.87,266,'store',5003002,3,5,671873.72,181
+'store',5004001,4,5,801416.97,219,'store',5004001,4,5,1405232.18,371
+'store',5004002,4,5,1061684.64,273,'store',5004002,4,5,790288.63,209
+'store',6001001,1,6,24105.89,9,'store',6001001,1,6,43566.03,11
+'store',6001002,1,6,40621.71,10,'store',6001002,1,6,31684.03,10
+'store',6001003,1,6,23365.00,8,'store',6001003,1,6,25117.82,10
+'store',6001004,1,6,29458.48,6,'store',6001004,1,6,29630.40,7
+'store',6001005,1,6,40555.76,13,'store',6001005,1,6,67204.16,14
+'store',6001006,1,6,49871.82,10,'store',6001006,1,6,17872.80,6
+'store',6001007,1,6,23526.21,8,'store',6001007,1,6,34341.06,13
+'store',6001008,1,6,74782.90,20,'store',6001008,1,6,127415.74,33
+'store',6002001,2,6,55244.23,15,'store',6002001,2,6,67018.71,20
+'store',6002002,2,6,24045.72,11,'store',6002002,2,6,73776.49,17
+'store',6002004,2,6,45590.44,15,'store',6002004,2,6,13288.01,5
+'store',6002005,2,6,53619.89,11,'store',6002005,2,6,57765.00,12
+'store',6002006,2,6,36339.64,12,'store',6002006,2,6,28988.99,8
+'store',6002007,2,6,33520.91,9,'store',6002007,2,6,55473.59,15
+'store',6002008,2,6,24710.01,7,'store',6002008,2,6,25085.33,11
+'store',6003001,3,6,33165.63,9,'store',6003001,3,6,84144.62,18
+'store',6003002,3,6,52293.57,15,'store',6003002,3,6,59151.70,15
+'store',6003003,3,6,33342.98,7,'store',6003003,3,6,43720.13,20
+'store',6003004,3,6,26344.41,9,'store',6003004,3,6,30930.85,9
+'store',6003005,3,6,89998.03,26,'store',6003005,3,6,95491.00,29
+'store',6003006,3,6,31862.34,10,'store',6003006,3,6,24121.07,3
+'store',6003007,3,6,92405.03,24,'store',6003007,3,6,129219.46,31
+'store',6003008,3,6,46145.62,13,'store',6003008,3,6,41948.28,19
+'store',6004001,4,6,12839.97,5,'store',6004001,4,6,74616.32,16
+'store',6004002,4,6,33700.51,11,'store',6004002,4,6,45202.40,11
+'store',6004003,4,6,31387.04,12,'store',6004003,4,6,103532.44,21
+'store',6004004,4,6,42644.68,8,'store',6004004,4,6,15287.71,6
+'store',6004005,4,6,15515.22,3,'store',6004005,4,6,60172.67,16
+'store',6004006,4,6,60466.48,15,'store',6004006,4,6,38426.04,10
+'store',6004007,4,6,48158.78,12,'store',6004007,4,6,55265.06,13
+'store',6004008,4,6,110528.92,26,'store',6004008,4,6,17409.99,6
+'store',6005001,5,6,38589.67,11,'store',6005001,5,6,135393.23,42
+'store',6005002,5,6,47495.55,8,'store',6005002,5,6,49267.29,12
+'store',6005003,5,6,14884.61,3,'store',6005003,5,6,74851.15,16
+'store',6005005,5,6,37483.24,12,'store',6005005,5,6,73876.82,18
+'store',6005006,5,6,61699.19,17,'store',6005006,5,6,52082.17,11
+'store',6005007,5,6,18764.89,5,'store',6005007,5,6,48445.43,16
+'store',6005008,5,6,25927.83,9,'store',6005008,5,6,16473.68,7
+'store',6006001,6,6,36995.31,14,'store',6006001,6,6,69739.81,24
+'store',6006002,6,6,85385.16,22,'store',6006002,6,6,87585.47,23
+'store',6006003,6,6,47063.87,11,'store',6006003,6,6,114431.48,29
+'store',6006004,6,6,20527.50,8,'store',6006004,6,6,23213.37,3
+'store',6006005,6,6,7442.61,4,'store',6006005,6,6,93912.14,18
+'store',6006006,6,6,25335.02,8,'store',6006006,6,6,24102.43,5
+'store',6006007,6,6,12901.85,6,'store',6006007,6,6,107194.04,30
+'store',6006008,6,6,39533.23,14,'store',6006008,6,6,48680.37,14
+'store',6007001,7,6,9032.59,3,'store',6007001,7,6,15183.24,6
+'store',6007002,7,6,57662.38,13,'store',6007002,7,6,42989.39,13
+'store',6007003,7,6,44207.42,17,'store',6007003,7,6,141091.04,39
+'store',6007004,7,6,147787.95,29,'store',6007004,7,6,69437.35,16
+'store',6007005,7,6,33681.87,9,'store',6007005,7,6,49926.37,15
+'store',6007006,7,6,50875.60,16,'store',6007006,7,6,75381.63,24
+'store',6007007,7,6,13622.22,7,'store',6007007,7,6,102650.94,31
+'store',6007008,7,6,34697.31,12,'store',6007008,7,6,49841.13,18
+'store',6008001,8,6,44993.33,10,'store',6008001,8,6,57972.98,22
+'store',6008002,8,6,39437.31,8,'store',6008002,8,6,48434.65,14
+'store',6008003,8,6,59905.77,14,'store',6008003,8,6,60930.66,20
+'store',6008004,8,6,36676.25,9,'store',6008004,8,6,31817.07,9
+'store',6008005,8,6,84238.28,27,'store',6008005,8,6,136071.70,27
+'store',6008007,8,6,14672.77,7,'store',6008007,8,6,59474.21,18
+---- TYPES
+STRING,INT,INT,INT,DECIMAL,BIGINT,STRING,INT,INT,INT,DECIMAL,BIGINT
+====
diff --git a/testdata/workloads/tpcds/queries/tpcds-q38.test b/testdata/workloads/tpcds/queries/tpcds-q38.test
new file mode 100644
index 0000000..3de28d1
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q38.test
@@ -0,0 +1,35 @@
+====
+---- QUERY: TPCDS-Q38
+SELECT count(*)
+FROM
+  (SELECT DISTINCT c_last_name,
+                   c_first_name,
+                   d_date
+   FROM store_sales,
+        date_dim,
+        customer
+   WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk
+     AND store_sales.ss_customer_sk = customer.c_customer_sk
+     AND d_month_seq BETWEEN 1200 AND 1200 + 11 INTERSECT
+     SELECT DISTINCT c_last_name,
+                     c_first_name,
+                     d_date
+     FROM catalog_sales,
+          date_dim,
+          customer WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
+     AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk
+     AND d_month_seq BETWEEN 1200 AND 1200 + 11 INTERSECT
+     SELECT DISTINCT c_last_name,
+                     c_first_name,
+                     d_date
+     FROM web_sales,
+          date_dim,
+          customer WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk
+     AND web_sales.ws_bill_customer_sk = customer.c_customer_sk
+     AND d_month_seq BETWEEN 1200 AND 1200 + 11 ) hot_cust
+LIMIT 100
+---- RESULTS
+107
+---- TYPES
+BIGINT
+====
diff --git a/testdata/workloads/tpcds/queries/tpcds-q87.test b/testdata/workloads/tpcds/queries/tpcds-q87.test
new file mode 100644
index 0000000..783915e
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q87.test
@@ -0,0 +1,37 @@
+====
+---- QUERY: TPCDS-Q87
+SELECT count(*)
+FROM ((SELECT DISTINCT c_last_name,
+                         c_first_name,
+                         d_date
+         FROM store_sales,
+              date_dim,
+              customer
+         WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk
+           AND store_sales.ss_customer_sk = customer.c_customer_sk
+           AND d_month_seq BETWEEN 1200 AND 1200+11)
+      EXCEPT
+        (SELECT DISTINCT c_last_name,
+                         c_first_name,
+                         d_date
+         FROM catalog_sales,
+              date_dim,
+              customer
+         WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk
+           AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk
+           AND d_month_seq BETWEEN 1200 AND 1200+11)
+      EXCEPT
+        (SELECT DISTINCT c_last_name,
+                         c_first_name,
+                         d_date
+         FROM web_sales,
+              date_dim,
+              customer
+         WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk
+           AND web_sales.ws_bill_customer_sk = customer.c_customer_sk
+           AND d_month_seq BETWEEN 1200 AND 1200+11)) cool_cust;
+---- RESULTS
+47298
+---- TYPES
+BIGINT
+====
diff --git a/tests/query_test/test_queries.py b/tests/query_test/test_queries.py
index 9d41fcf..bb73b28 100644
--- a/tests/query_test/test_queries.py
+++ b/tests/query_test/test_queries.py
@@ -122,6 +122,12 @@ class TestQueries(ImpalaTestSuite):
     result = self.execute_query(query_string, vector.get_value('exec_option'))
     assert result.data[0] == '60'
 
+  def test_intersect(self, vector):
+    self.run_test_case('QueryTest/intersect', vector)
+
+  def test_except(self, vector):
+    self.run_test_case('QueryTest/except', vector)
+
   def test_sort(self, vector):
     if vector.get_value('table_format').file_format == 'hbase':
       pytest.xfail(reason="IMPALA-283 - select count(*) produces inconsistent results")
diff --git a/tests/query_test/test_tpcds_queries.py b/tests/query_test/test_tpcds_queries.py
index 13d0ea0..75b8ce7 100644
--- a/tests/query_test/test_tpcds_queries.py
+++ b/tests/query_test/test_tpcds_queries.py
@@ -88,6 +88,12 @@ class TestTpcdsQuery(ImpalaTestSuite):
   def test_tpcds_q13(self, vector):
     self.run_test_case(self.get_workload() + '-q13', vector)
 
+  def test_tpcds_q14_1(self, vector):
+    self.run_test_case(self.get_workload() + '-q14-1', vector)
+
+  def test_tpcds_q14_2(self, vector):
+    self.run_test_case(self.get_workload() + '-q14-2', vector)
+
   def test_tpcds_q15(self, vector):
     self.run_test_case(self.get_workload() + '-q15', vector)
 
@@ -145,6 +151,9 @@ class TestTpcdsQuery(ImpalaTestSuite):
   def test_tpcds_q37(self, vector):
     self.run_test_case(self.get_workload() + '-q37', vector)
 
+  def test_tpcds_q38(self, vector):
+    self.run_test_case(self.get_workload() + '-q38', vector)
+
   def test_tpcds_q39_1(self, vector):
     self.run_test_case(self.get_workload() + '-q39-1', vector)
 
@@ -286,6 +295,9 @@ class TestTpcdsQuery(ImpalaTestSuite):
   def test_tpcds_q86a(self, vector):
     self.run_test_case(self.get_workload() + '-q86a', vector)
 
+  def test_tpcds_q87(self, vector):
+    self.run_test_case(self.get_workload() + '-q87', vector)
+
   def test_tpcds_q88(self, vector):
     self.run_test_case(self.get_workload() + '-q88', vector)
 
@@ -378,6 +390,12 @@ class TestTpcdsDecimalV2Query(ImpalaTestSuite):
   def test_tpcds_q13(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q13', vector)
 
+  def test_tpcds_q14_1(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q14-1', vector)
+
+  def test_tpcds_q14_2(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q14-2', vector)
+
   def test_tpcds_q15(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q15', vector)
 
diff --git a/tests/util/parse_util.py b/tests/util/parse_util.py
index 57f546e..bb19db7 100644
--- a/tests/util/parse_util.py
+++ b/tests/util/parse_util.py
@@ -22,7 +22,7 @@ from datetime import datetime
 # changed, and the stress test loses the ability to run the full set of queries. Set
 # these constants and assert that when a workload is used, all the queries we expect to
 # use are there.
-EXPECTED_TPCDS_QUERIES_COUNT = 97
+EXPECTED_TPCDS_QUERIES_COUNT = 99
 EXPECTED_TPCH_NESTED_QUERIES_COUNT = 22
 EXPECTED_TPCH_QUERIES_COUNT = 22
 # Add the number of stress test specific queries, i.e. in files like '*-stress-*.test'


[impala] 01/02: IMPALA-9959: Implement ds_kll_sketch() and ds_kll_quantile() functions

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tarmstrong pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 033a4607e2c9cd5a107a3af01f3fb3490bc5bc6e
Author: Gabor Kaszab <ga...@cloudera.com>
AuthorDate: Fri Jul 17 09:06:35 2020 +0200

    IMPALA-9959: Implement ds_kll_sketch() and ds_kll_quantile() functions
    
    ds_kll_sketch() is an aggregate function that receives a float
    parameter (e.g. a float column of a table) and returns a serialized
    Apache DataSketches KLL sketch of the input data set wrapped into
    STRING type. This sketch can be saved into a table or view and later
    used for quantile approximations. ds_kll_quantile() receives two
    parameters: a STRING parameter that contains a serialized KLL sketch
    and a DOUBLE that represents the rank of the quantile in the range of
    [0,1]. E.g. rank=0.1 means the approximate value in the sketch where
    10% of the sketched items are less than or equals to this value.
    
    Testing:
      - Added automated tests on small data sets to check the basic
        functionality of sketching and getting a quantile approximate.
      - Tested on TPCH25_parquet.lineitem to check that sketching and
        approximating works on bigger scale as well where serialize/merge
        phases are also required. On this scale the error range of the
        quantile approximation is within 1-1.5%
    
    Change-Id: I11de5fe10bb5d0dd42fb4ee45c4f21cb31963e52
    Reviewed-on: http://gerrit.cloudera.org:8080/16235
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exprs/aggregate-functions-ir.cc             |  95 +++++++++++++-
 be/src/exprs/aggregate-functions.h                 |   7 +
 be/src/exprs/datasketches-common.cc                |  18 ++-
 be/src/exprs/datasketches-common.h                 |  16 +--
 be/src/exprs/datasketches-functions-ir.cc          |  28 +++-
 be/src/exprs/datasketches-functions.h              |  16 ++-
 common/function-registry/impala_functions.py       |   5 +-
 .../java/org/apache/impala/catalog/BuiltinsDb.java |  10 ++
 testdata/data/README                               |  10 ++
 testdata/data/kll_sketches_from_hive.parquet       | Bin 0 -> 2501 bytes
 .../queries/QueryTest/datasketches-kll.test        | 146 +++++++++++++++++++++
 tests/query_test/test_datasketches.py              |   4 +
 12 files changed, 333 insertions(+), 22 deletions(-)

diff --git a/be/src/exprs/aggregate-functions-ir.cc b/be/src/exprs/aggregate-functions-ir.cc
index e3db0cc..1762860 100644
--- a/be/src/exprs/aggregate-functions-ir.cc
+++ b/be/src/exprs/aggregate-functions-ir.cc
@@ -40,6 +40,7 @@
 #include "runtime/timestamp-value.h"
 #include "runtime/timestamp-value.inline.h"
 #include "thirdparty/datasketches/hll.hpp"
+#include "thirdparty/datasketches/kll_sketch.hpp"
 #include "util/arithmetic-util.h"
 #include "util/mpfit-util.h"
 #include "util/pretty-printer.h"
@@ -54,6 +55,8 @@ using std::min_element;
 using std::nth_element;
 using std::pop_heap;
 using std::push_heap;
+using std::string;
+using std::stringstream;
 
 namespace {
 // Threshold for each precision where it's better to use linear counting instead
@@ -1612,6 +1615,14 @@ BigIntVal AggregateFunctions::HllFinalize(FunctionContext* ctx, const StringVal&
   return estimate;
 }
 
+StringVal StringStreamToStringVal(FunctionContext* ctx,
+    const stringstream& str_stream) {
+  string str = str_stream.str();
+  StringVal dst(ctx, str.size());
+  memcpy(dst.ptr, str.c_str(), str.size());
+  return dst;
+}
+
 /// Auxiliary function that receives a hll_sketch and returns the serialized version of
 /// it wrapped into a StringVal.
 /// Introducing this function in the .cc to avoid including the whole DataSketches HLL
@@ -1620,10 +1631,7 @@ StringVal SerializeCompactDsHllSketch(FunctionContext* ctx,
     const datasketches::hll_sketch& sketch) {
   std::stringstream serialized_input;
   sketch.serialize_compact(serialized_input);
-  std::string serialized_input_str = serialized_input.str();
-  StringVal dst(ctx, serialized_input_str.size());
-  memcpy(dst.ptr, serialized_input_str.c_str(), serialized_input_str.size());
-  return dst;
+  return StringStreamToStringVal(ctx, serialized_input);
 }
 
 /// Auxiliary function that receives a hll_union, gets the underlying HLL sketch from the
@@ -1637,6 +1645,17 @@ StringVal SerializeDsHllUnion(FunctionContext* ctx,
   return SerializeCompactDsHllSketch(ctx, sketch);
 }
 
+/// Auxiliary function that receives a kll_sketch<float> and returns the serialized
+/// version of it wrapped into a StringVal.
+/// Introducing this function in the .cc to avoid including the whole DataSketches HLL
+/// functionality into the header
+StringVal SerializeDsKllSketch(FunctionContext* ctx,
+    const datasketches::kll_sketch<float>& sketch) {
+  std::stringstream serialized_sketch;
+  sketch.serialize(serialized_sketch);
+  return StringStreamToStringVal(ctx, serialized_sketch);
+}
+
 void AggregateFunctions::DsHllInit(FunctionContext* ctx, StringVal* dst) {
   AllocBuffer(ctx, dst, sizeof(datasketches::hll_sketch));
   if (UNLIKELY(dst->is_null)) {
@@ -1743,10 +1762,10 @@ void AggregateFunctions::DsHllUnionUpdate(FunctionContext* ctx, const StringVal&
   if (src.is_null) return;
   DCHECK(!dst->is_null);
   DCHECK_EQ(dst->len, sizeof(datasketches::hll_union));
-  // These parameters might be overwritten by DeserializeHllSketch() to use the settings
+  // These parameters might be overwritten by DeserializeDsSketch() to use the settings
   // from the deserialized sketch from 'src'.
   datasketches::hll_sketch src_sketch(DS_SKETCH_CONFIG, DS_HLL_TYPE);
-  if (!DeserializeHllSketch(src, &src_sketch)) {
+  if (!DeserializeDsSketch(src, &src_sketch)) {
     LogSketchDeserializationError(ctx);
     return;
   }
@@ -1798,6 +1817,70 @@ StringVal AggregateFunctions::DsHllUnionFinalize(FunctionContext* ctx,
   return result;
 }
 
+void AggregateFunctions::DsKllInit(FunctionContext* ctx, StringVal* dst) {
+  AllocBuffer(ctx, dst, sizeof(datasketches::kll_sketch<float>));
+  if (UNLIKELY(dst->is_null)) {
+    DCHECK(!ctx->impl()->state()->GetQueryStatus().ok());
+    return;
+  }
+  // Note, that kll_sketch will always have the same size regardless of the amount of
+  // data it keeps track of. This is because it's a wrapper class that holds all the
+  // inserted data on heap. Here, we put only the wrapper class into a StringVal.
+  datasketches::kll_sketch<float>* sketch_ptr =
+      reinterpret_cast<datasketches::kll_sketch<float>*>(dst->ptr);
+  *sketch_ptr = datasketches::kll_sketch<float>();
+}
+
+void AggregateFunctions::DsKllUpdate(FunctionContext* ctx, const FloatVal& src,
+    StringVal* dst) {
+  if (src.is_null) return;
+  DCHECK(!dst->is_null);
+  DCHECK_EQ(dst->len, sizeof(datasketches::kll_sketch<float>));
+  datasketches::kll_sketch<float>* sketch_ptr =
+      reinterpret_cast<datasketches::kll_sketch<float>*>(dst->ptr);
+  sketch_ptr->update(src.val);
+}
+
+StringVal AggregateFunctions::DsKllSerialize(FunctionContext* ctx,
+    const StringVal& src) {
+  DCHECK(!src.is_null);
+  DCHECK_EQ(src.len, sizeof(datasketches::kll_sketch<float>));
+  datasketches::kll_sketch<float>* sketch_ptr =
+      reinterpret_cast<datasketches::kll_sketch<float>*>(src.ptr);
+  StringVal dst = SerializeDsKllSketch(ctx, *sketch_ptr);
+  ctx->Free(src.ptr);
+  return dst;
+}
+
+void AggregateFunctions::DsKllMerge(
+    FunctionContext* ctx, const StringVal& src, StringVal* dst) {
+  DCHECK(!src.is_null);
+  DCHECK(!dst->is_null);
+  DCHECK_EQ(dst->len, sizeof(datasketches::kll_sketch<float>));
+  datasketches::kll_sketch<float> src_sketch =
+      datasketches::kll_sketch<float>::deserialize((void*)src.ptr, src.len);
+
+  datasketches::kll_sketch<float>* dst_sketch_ptr =
+      reinterpret_cast<datasketches::kll_sketch<float>*>(dst->ptr);
+
+  dst_sketch_ptr->merge(src_sketch);
+}
+
+StringVal AggregateFunctions::DsKllFinalizeSketch(FunctionContext* ctx,
+    const StringVal& src) {
+  DCHECK(!src.is_null);
+  DCHECK_EQ(src.len, sizeof(datasketches::kll_sketch<float>));
+  datasketches::kll_sketch<float>* sketch_ptr =
+      reinterpret_cast<datasketches::kll_sketch<float>*>(src.ptr);
+  if (sketch_ptr->get_n() == 0) {
+    ctx->Free(src.ptr);
+    return StringVal::null();
+  }
+  StringVal dst = SerializeDsKllSketch(ctx, *sketch_ptr);
+  ctx->Free(src.ptr);
+  return dst;
+}
+
 /// Intermediate aggregation state for the SampledNdv() function.
 /// Stores NUM_HLL_BUCKETS of the form <row_count, hll_state>.
 /// The 'row_count' keeps track of how many input rows were aggregated into that
diff --git a/be/src/exprs/aggregate-functions.h b/be/src/exprs/aggregate-functions.h
index d7fb986..487c451 100644
--- a/be/src/exprs/aggregate-functions.h
+++ b/be/src/exprs/aggregate-functions.h
@@ -250,6 +250,13 @@ class AggregateFunctions {
   static void DsHllUnionMerge(FunctionContext*, const StringVal& src, StringVal* dst);
   static StringVal DsHllUnionFinalize(FunctionContext*, const StringVal& src);
 
+  /// These functions implement Apache DataSketches KLL support for sketching.
+  static void DsKllInit(FunctionContext*, StringVal* slot);
+  static void DsKllUpdate(FunctionContext*, const FloatVal& src, StringVal* dst);
+  static StringVal DsKllSerialize(FunctionContext*, const StringVal& src);
+  static void DsKllMerge(FunctionContext*, const StringVal& src, StringVal* dst);
+  static StringVal DsKllFinalizeSketch(FunctionContext*, const StringVal& src);
+
   /// Estimates the number of distinct values (NDV) based on a sample of data and the
   /// corresponding sampling rate. The main idea of this function is to collect several
   /// (x,y) data points where x is the number of rows and y is the corresponding NDV
diff --git a/be/src/exprs/datasketches-common.cc b/be/src/exprs/datasketches-common.cc
index 0fe278c..c9bdcaf 100644
--- a/be/src/exprs/datasketches-common.cc
+++ b/be/src/exprs/datasketches-common.cc
@@ -19,28 +19,36 @@
 
 #include "common/logging.h"
 #include "udf/udf-internal.h"
+#include "thirdparty/datasketches/kll_sketch.hpp"
 
 namespace impala {
 
 using datasketches::hll_sketch;
+using datasketches::kll_sketch;
 using impala_udf::StringVal;
 
 void LogSketchDeserializationError(FunctionContext* ctx) {
   ctx->SetError("Unable to deserialize sketch.");
 }
 
-bool DeserializeHllSketch(const StringVal& serialized_sketch, hll_sketch* sketch) {
+template<class T>
+bool DeserializeDsSketch(const StringVal& serialized_sketch, T* sketch) {
   DCHECK(sketch != nullptr);
   if (serialized_sketch.is_null || serialized_sketch.len == 0) return false;
   try {
-    *sketch = hll_sketch::deserialize((void*)serialized_sketch.ptr,
-        serialized_sketch.len);
+    *sketch = T::deserialize((void*)serialized_sketch.ptr, serialized_sketch.len);
     return true;
-  } catch (const std::invalid_argument&) {
-    // Deserialization throws if the input string is not a serialized sketch.
+  } catch (const std::exception&) {
+    // One reason of throwing from deserialization is that the input string is not a
+    // serialized sketch.
     return false;
   }
 }
 
+template bool DeserializeDsSketch(const StringVal& serialized_sketch,
+    hll_sketch* sketch);
+template bool DeserializeDsSketch(const StringVal& serialized_sketch,
+    kll_sketch<float>* sketch);
+
 }
 
diff --git a/be/src/exprs/datasketches-common.h b/be/src/exprs/datasketches-common.h
index 3d4f43c..7560692 100644
--- a/be/src/exprs/datasketches-common.h
+++ b/be/src/exprs/datasketches-common.h
@@ -37,13 +37,13 @@ const int DS_SKETCH_CONFIG = 12;
 /// Logs a common error message saying that sketch deserialization failed.
 void LogSketchDeserializationError(FunctionContext* ctx);
 
-/// Receives a serialized DataSketches HLL sketch in 'serialized_sketch', deserializes it
-/// and puts the deserialized sketch into 'sketch'. The outgoing 'sketch' will hold the
-/// same configs as 'serialized_sketch' regardless of what was provided when it was
-/// constructed before this function call. Returns false if the deserialization
-/// fails, true otherwise.
-bool DeserializeHllSketch(const StringVal& serialized_sketch,
-    datasketches::hll_sketch* sketch) WARN_UNUSED_RESULT;
-
+/// Receives a serialized DataSketches sketch  (either Hll or KLL) in
+/// 'serialized_sketch', deserializes it and puts the deserialized sketch into 'sketch'.
+/// The outgoing 'sketch' will hold the same configs as 'serialized_sketch' regardless of
+/// what was provided when it was constructed before this function call. Returns false if
+/// the deserialization fails, true otherwise.
+template<class T>
+bool DeserializeDsSketch(const StringVal& serialized_sketch, T* sketch)
+    WARN_UNUSED_RESULT;
 }
 
diff --git a/be/src/exprs/datasketches-functions-ir.cc b/be/src/exprs/datasketches-functions-ir.cc
index bba537d..d2898bc 100644
--- a/be/src/exprs/datasketches-functions-ir.cc
+++ b/be/src/exprs/datasketches-functions-ir.cc
@@ -18,20 +18,46 @@
 #include "exprs/datasketches-functions.h"
 
 #include "exprs/datasketches-common.h"
+#include "gutil/strings/substitute.h"
 #include "thirdparty/datasketches/hll.hpp"
+#include "thirdparty/datasketches/kll_sketch.hpp"
+#include "udf/udf-internal.h"
 
 namespace impala {
 
+using strings::Substitute;
+
 BigIntVal DataSketchesFunctions::DsHllEstimate(FunctionContext* ctx,
     const StringVal& serialized_sketch) {
   if (serialized_sketch.is_null || serialized_sketch.len == 0) return BigIntVal::null();
   datasketches::hll_sketch sketch(DS_SKETCH_CONFIG, DS_HLL_TYPE);
-  if (!DeserializeHllSketch(serialized_sketch, &sketch)) {
+  if (!DeserializeDsSketch(serialized_sketch, &sketch)) {
     LogSketchDeserializationError(ctx);
     return BigIntVal::null();
   }
   return sketch.get_estimate();
 }
 
+FloatVal DataSketchesFunctions::DsKllQuantile(FunctionContext* ctx,
+    const StringVal& serialized_sketch, const DoubleVal& rank) {
+  if (serialized_sketch.is_null || serialized_sketch.len == 0) return FloatVal::null();
+  if (rank.val < 0.0 || rank.val > 1.0) {
+    ctx->SetError("Rank parameter should be in the range of [0,1]");
+    return FloatVal::null();
+  }
+  datasketches::kll_sketch<float> sketch;
+  if (!DeserializeDsSketch(serialized_sketch, &sketch)) {
+    LogSketchDeserializationError(ctx);
+    return FloatVal::null();
+  }
+  try {
+    return sketch.get_quantile(rank.val);
+  } catch (const std::exception& e) {
+    ctx->SetError(Substitute("Error while getting quantile from DataSketches KLL. "
+        "Message: $0", e.what()).c_str());
+    return FloatVal::null();
+  }
+}
+
 }
 
diff --git a/be/src/exprs/datasketches-functions.h b/be/src/exprs/datasketches-functions.h
index bcbec89..143fd69 100644
--- a/be/src/exprs/datasketches-functions.h
+++ b/be/src/exprs/datasketches-functions.h
@@ -22,12 +22,26 @@
 namespace impala {
 
 using impala_udf::BigIntVal;
+using impala_udf::DoubleVal;
+using impala_udf::FloatVal;
 using impala_udf::FunctionContext;
 using impala_udf::StringVal;
 
 class DataSketchesFunctions {
 public:
-  static BigIntVal DsHllEstimate(FunctionContext*, const StringVal&);
+  /// 'serialized_sketch' is expected as a serialized Apache DataSketches HLL sketch. If
+  /// it is not then the query fails. Otherwise, returns the count(distinct) estimate
+  /// from the sketch.
+  static BigIntVal DsHllEstimate(FunctionContext* ctx,
+      const StringVal& serialized_sketch);
+
+  /// 'serialized_sketch' is expected as a serialized Apache DataSketches KLL sketch. If
+  /// it is not then the query fails. 'rank' is used to identify which item (estimate) to
+  /// return from the sketched dataset. E.g. 0.1 means the item where 10% of the sketched
+  /// dataset is lower or equals to this particular item. 'rank' should be in the range
+  /// of [0,1]. Otherwise this function returns error.
+  static FloatVal DsKllQuantile(FunctionContext* ctx, const StringVal& serialized_sketch,
+      const DoubleVal& rank);
 };
 
 }
diff --git a/common/function-registry/impala_functions.py b/common/function-registry/impala_functions.py
index c366552..8398785 100644
--- a/common/function-registry/impala_functions.py
+++ b/common/function-registry/impala_functions.py
@@ -931,7 +931,10 @@ visible_functions = [
   [['mask_hash'], 'DATE', ['DATE'], 'impala::MaskFunctions::MaskHash'],
 
   # Functions to use Apache DataSketches functionality
-  [['ds_hll_estimate'], 'BIGINT', ['STRING'], '_ZN6impala21DataSketchesFunctions13DsHllEstimateEPN10impala_udf15FunctionContextERKNS1_9StringValE'],
+  [['ds_hll_estimate'], 'BIGINT', ['STRING'],
+      '_ZN6impala21DataSketchesFunctions13DsHllEstimateEPN10impala_udf15FunctionContextERKNS1_9StringValE'],
+  [['ds_kll_quantile'], 'FLOAT', ['STRING', 'DOUBLE'],
+      '_ZN6impala21DataSketchesFunctions13DsKllQuantileEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_9DoubleValE'],
 ]
 
 invisible_functions = [
diff --git a/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java b/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
index 514e49d..5969549 100644
--- a/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
+++ b/fe/src/main/java/org/apache/impala/catalog/BuiltinsDb.java
@@ -1323,6 +1323,16 @@ public class BuiltinsDb extends Db {
         prefix + "10CountMergeEPN10impala_udf15FunctionContextERKNS1_9BigIntValEPS4_",
         null, null));
 
+    db.addBuiltin(AggregateFunction.createBuiltin(db, "ds_kll_sketch",
+        Lists.<Type>newArrayList(Type.FLOAT), Type.STRING, Type.STRING,
+        prefix + "9DsKllInitEPN10impala_udf15FunctionContextEPNS1_9StringValE",
+        prefix + "11DsKllUpdateEPN10impala_udf15FunctionContextERKNS1_8FloatValEPNS1_" +
+            "9StringValE",
+        prefix + "10DsKllMergeEPN10impala_udf15FunctionContextERKNS1_9StringValEPS4_",
+        prefix + "14DsKllSerializeEPN10impala_udf15FunctionContextERKNS1_9StringValE",
+        prefix + "19DsKllFinalizeSketchEPN10impala_udf15FunctionContextERKNS1_" +
+            "9StringValE", true, false, true));
+
     // The following 3 functions are never directly executed because they get rewritten
     db.addBuiltin(AggregateFunction.createAnalyticBuiltin(
         db, "percent_rank", Lists.<Type>newArrayList(), Type.DOUBLE, Type.STRING));
diff --git a/testdata/data/README b/testdata/data/README
index 63c2d7d..41ddeac 100644
--- a/testdata/data/README
+++ b/testdata/data/README
@@ -509,6 +509,16 @@ hll_sketches_from_impala.parquet:
 This holds the same sketches as hll_sketches_from_hive.parquet but these sketches were
 created by Impala instead of Hive.
 
+kll_sketches_from_hive.parquet:
+This file contains a table that has some string columns to store serialized Apache
+DataSketches KLL sketches created by Hive. Each column is for a different purpose:
+  - 'f': Float with distinct values.
+  - 'repetitions': Float with some repetition in the values.
+  - 'some_nulls': Float values and some NULLs.
+  - 'all_nulls': All values are NULLs.
+  - 'some_nans': Floats with some NaN values.
+  - 'all_nans': All values are NaNs.
+
 hudi_parquet:
 IMPALA-8778: Support read Apache Hudi tables
 Hudi parquet is a special format of parquet files managed by Apache Hudi
diff --git a/testdata/data/kll_sketches_from_hive.parquet b/testdata/data/kll_sketches_from_hive.parquet
new file mode 100644
index 0000000..8842981
Binary files /dev/null and b/testdata/data/kll_sketches_from_hive.parquet differ
diff --git a/testdata/workloads/functional-query/queries/QueryTest/datasketches-kll.test b/testdata/workloads/functional-query/queries/QueryTest/datasketches-kll.test
new file mode 100644
index 0000000..b7b734b
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/datasketches-kll.test
@@ -0,0 +1,146 @@
+====
+---- QUERY
+# Check that ds_kll_quantile returns error for strings that are not serialized sketches.
+select ds_kll_quantile(date_string_col, 0.5) from functional_parquet.alltypestiny;
+---- CATCH
+UDF ERROR: Unable to deserialize sketch
+====
+---- QUERY
+select ds_kll_quantile(ds_kll_sketch(float_col), -0.1) from functional_parquet.alltypestiny;
+---- CATCH
+UDF ERROR: Rank parameter should be in the range of [0,1]
+====
+---- QUERY
+select ds_kll_quantile(ds_kll_sketch(float_col), 1.1) from functional_parquet.alltypestiny;
+---- CATCH
+UDF ERROR: Rank parameter should be in the range of [0,1]
+====
+---- QUERY
+select
+    ds_kll_quantile(ds_kll_sketch(id), 0),
+    ds_kll_quantile(ds_kll_sketch(tinyint_col), 0),
+    ds_kll_quantile(ds_kll_sketch(smallint_col), 0),
+    ds_kll_quantile(ds_kll_sketch(int_col), 0),
+    ds_kll_quantile(ds_kll_sketch(bigint_col), 0),
+    ds_kll_quantile(ds_kll_sketch(float_col), 0)
+from functional_parquet.alltypestiny;
+---- RESULTS
+0,0,0,0,0,0
+---- TYPES
+FLOAT,FLOAT,FLOAT,FLOAT,FLOAT,FLOAT
+====
+---- QUERY
+select
+    ds_kll_quantile(ds_kll_sketch(id), 0.5),
+    ds_kll_quantile(ds_kll_sketch(tinyint_col), 0.5),
+    ds_kll_quantile(ds_kll_sketch(smallint_col), 0.5),
+    ds_kll_quantile(ds_kll_sketch(int_col), 0.5),
+    ds_kll_quantile(ds_kll_sketch(bigint_col), 0.5),
+    ds_kll_quantile(ds_kll_sketch(float_col), 0.5)
+from functional_parquet.alltypestiny;
+---- RESULTS
+4,1,1,1,10,1.100000023841858
+---- TYPES
+FLOAT,FLOAT,FLOAT,FLOAT,FLOAT,FLOAT
+====
+---- QUERY
+select
+    ds_kll_quantile(ds_kll_sketch(id), 1),
+    ds_kll_quantile(ds_kll_sketch(tinyint_col), 1),
+    ds_kll_quantile(ds_kll_sketch(smallint_col), 1),
+    ds_kll_quantile(ds_kll_sketch(int_col), 1),
+    ds_kll_quantile(ds_kll_sketch(bigint_col), 1),
+    ds_kll_quantile(ds_kll_sketch(float_col), 1)
+from functional_parquet.alltypestiny;
+---- RESULTS
+7,1,1,1,10,1.100000023841858
+---- TYPES
+FLOAT,FLOAT,FLOAT,FLOAT,FLOAT,FLOAT
+====
+---- QUERY
+select ds_kll_sketch(double_col) from functional_parquet.alltypestiny;
+---- CATCH
+AnalysisException: No matching function with signature: ds_kll_sketch(DOUBLE)
+====
+---- QUERY
+select ds_kll_sketch(string_col) from functional_parquet.alltypestiny;
+---- CATCH
+AnalysisException: No matching function with signature: ds_kll_sketch(STRING)
+====
+---- QUERY
+select ds_kll_sketch(timestamp_col) from functional_parquet.alltypestiny;
+---- CATCH
+AnalysisException: No matching function with signature: ds_kll_sketch(TIMESTAMP)
+====
+---- QUERY
+select ds_kll_sketch(cast(date_string_col as date format 'MM/DD/YYYY'))
+from functional_parquet.alltypestiny;
+---- CATCH
+AnalysisException: No matching function with signature: ds_kll_sketch(DATE)
+====
+---- QUERY
+# Check that ds_kll_quantile() returns null for null inputs.
+select ds_kll_quantile(c, 0.5) from functional_parquet.nulltable;
+---- RESULTS
+NULL
+---- TYPES
+FLOAT
+====
+---- QUERY
+# Check that ds_kll_sketch() returns null for null inputs.
+select ds_kll_sketch(d) from functional_parquet.nulltable;
+---- RESULTS
+'NULL'
+---- TYPES
+STRING
+====
+---- QUERY
+# Check that ds_kll_sketch() returns null for empty input.
+select ds_kll_sketch(f2) from functional_parquet.emptytable;
+---- RESULTS
+'NULL'
+---- TYPES
+STRING
+====
+---- QUERY
+# Write sketches to a table as string and get an estimate from the written sketch.
+# Note, the plan is to write sketches as binary instead of strings. For this we have to
+# wait for the binary support (IMPALA-9482).
+create table sketch_store
+    (year int, month int, float_sketch string)
+stored as parquet;
+insert into sketch_store
+    select
+        year,
+        month,
+        ds_kll_sketch(float_col)
+    from functional_parquet.alltypessmall
+    group by year, month;
+select
+    year,
+    month,
+    ds_kll_quantile(float_sketch, 0.5)
+from sketch_store;
+---- RESULTS
+2009,1,4.400000095367432
+2009,2,4.400000095367432
+2009,3,4.400000095367432
+2009,4,4.400000095367432
+---- TYPES
+INT,INT,FLOAT
+====
+---- QUERY
+# Check that sketches made by Hive can be read and used for estimating by Impala.
+select
+    ds_kll_quantile(f, 0.5) as f,
+    ds_kll_quantile(repetitions, 0.5) as r,
+    ds_kll_quantile(some_nulls, 0.5) as sn,
+    ds_kll_quantile(all_nulls, 0.5) as an,
+    ds_kll_quantile(some_nans, 0.5) as snan,
+    ds_kll_quantile(all_nans, 0.5) as anan
+from kll_sketches_from_hive;
+---- TYPES
+FLOAT,FLOAT,FLOAT,FLOAT,FLOAT,FLOAT
+---- RESULTS
+100.1999969482422,25000.099609375,50.90000152587891,NULL,50.5,NULL
+====
diff --git a/tests/query_test/test_datasketches.py b/tests/query_test/test_datasketches.py
index 53d051c..1634387 100644
--- a/tests/query_test/test_datasketches.py
+++ b/tests/query_test/test_datasketches.py
@@ -36,3 +36,7 @@ class TestDatasketches(ImpalaTestSuite):
     create_table_from_parquet(self.client, unique_database, 'hll_sketches_from_hive')
     create_table_from_parquet(self.client, unique_database, 'hll_sketches_from_impala')
     self.run_test_case('QueryTest/datasketches-hll', vector, unique_database)
+
+  def test_kll(self, vector, unique_database):
+    create_table_from_parquet(self.client, unique_database, 'kll_sketches_from_hive')
+    self.run_test_case('QueryTest/datasketches-kll', vector, unique_database)