You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2019/10/17 08:13:24 UTC

[calcite] branch master updated (2ac4415 -> ec9947a)

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

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


    from 2ac4415  [CALCITE-3420] NullPointerException throws for implicit type coercion of nested SET operations
     new bfb716a  Refactor InterpreterTest
     new ec9947a  [CALCITE-3407] Implement MINUS and INTERSECT in interpreter (Wang Yanlin)

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:
 .../org/apache/calcite/interpreter/Bindables.java  |   2 +-
 .../apache/calcite/interpreter/Interpreter.java    |   7 +
 .../java/org/apache/calcite/interpreter/Nodes.java |   6 +-
 .../org/apache/calcite/interpreter/SetOpNode.java  |  87 ++++++
 .../org/apache/calcite/interpreter/UnionNode.java  |   3 +
 .../org/apache/calcite/test/InterpreterTest.java   | 321 ++++++++++++---------
 6 files changed, 291 insertions(+), 135 deletions(-)
 create mode 100644 core/src/main/java/org/apache/calcite/interpreter/SetOpNode.java


[calcite] 01/02: Refactor InterpreterTest

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

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

commit bfb716aefbb2673eaa3d91a7effdb7cc905546a2
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Wed Oct 16 15:51:13 2019 -0700

    Refactor InterpreterTest
---
 .../org/apache/calcite/test/InterpreterTest.java   | 223 +++++++++------------
 1 file changed, 90 insertions(+), 133 deletions(-)

diff --git a/core/src/test/java/org/apache/calcite/test/InterpreterTest.java b/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
index 7f1025d..de284dd 100644
--- a/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
@@ -21,6 +21,7 @@ import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.interpreter.Interpreter;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.parser.SqlParser;
@@ -74,6 +75,56 @@ public class InterpreterTest {
     }
   }
 
+  /** Fluent class that contains information necessary to run a test. */
+  private static class Sql {
+    private final String sql;
+    private final MyDataContext dataContext;
+    private final Planner planner;
+    private final boolean project;
+
+    Sql(String sql, MyDataContext dataContext, Planner planner,
+        boolean project) {
+      this.sql = sql;
+      this.dataContext = dataContext;
+      this.planner = planner;
+      this.project = project;
+    }
+
+    @SuppressWarnings("SameParameterValue")
+    Sql withProject(boolean project) {
+      return new Sql(sql, dataContext, planner, project);
+    }
+
+    /** Interprets the sql and checks result with specified rows, ordered. */
+    @SuppressWarnings("UnusedReturnValue")
+    Sql returnsRows(String... rows) throws Exception {
+      return returnsRows(false, rows);
+    }
+
+    /** Interprets the sql and checks result with specified rows, unordered. */
+    @SuppressWarnings("UnusedReturnValue")
+    Sql returnsRowsUnordered(String... rows) throws Exception {
+      return returnsRows(true, rows);
+    }
+
+    /** Interprets the sql and checks result with specified rows. */
+    private Sql returnsRows(boolean unordered, String[] rows)
+        throws Exception {
+      SqlNode parse = planner.parse(sql);
+      SqlNode validate = planner.validate(parse);
+      final RelRoot root = planner.rel(validate);
+      RelNode convert = project ? root.project() : root.rel;
+      final Interpreter interpreter = new Interpreter(dataContext, convert);
+      assertRows(interpreter, unordered, rows);
+      return this;
+    }
+  }
+
+  /** Creates a {@link Sql}. */
+  private Sql sql(String sql) {
+    return new Sql(sql, dataContext, planner, false);
+  }
+
   @Before public void setUp() {
     rootSchema = Frameworks.createRootSchema(true);
     final FrameworkConfig config = Frameworks.newConfigBuilder()
@@ -93,16 +144,10 @@ public class InterpreterTest {
 
   /** Tests executing a simple plan using an interpreter. */
   @Test public void testInterpretProjectFilterValues() throws Exception {
-    SqlNode parse =
-        planner.parse("select y, x\n"
-            + "from (values (1, 'a'), (2, 'b'), (3, 'c')) as t(x, y)\n"
-            + "where x > 1");
-
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).rel;
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRows(interpreter, "[b, 2]", "[c, 3]");
+    final String sql = "select y, x\n"
+        + "from (values (1, 'a'), (2, 'b'), (3, 'c')) as t(x, y)\n"
+        + "where x > 1";
+    sql(sql).returnsRows("[b, 2]", "[c, 3]");
   }
 
   /** Tests a plan where the sort field is projected away. */
@@ -110,31 +155,12 @@ public class InterpreterTest {
     final String sql = "select y\n"
         + "from (values (1, 'a'), (2, 'b'), (3, 'c')) as t(x, y)\n"
         + "order by -x";
-    SqlNode parse = planner.parse(sql);
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).project();
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRows(interpreter, "[c]", "[b]", "[a]");
+    sql(sql).withProject(true).returnsRows("[c]", "[b]", "[a]");
   }
 
   @Test public void testInterpretMultiset() throws Exception {
     final String sql = "select multiset['a', 'b', 'c']";
-    SqlNode parse = planner.parse(sql);
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).project();
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRows(interpreter, "[[a, b, c]]");
-  }
-
-  private static void assertRows(Interpreter interpreter, String... rows) {
-    assertRows(interpreter, false, rows);
-  }
-
-  private static void assertRowsUnordered(Interpreter interpreter,
-      String... rows) {
-    assertRows(interpreter, true, rows);
+    sql(sql).withProject(true).returnsRows("[[a, b, c]]");
   }
 
   private static void assertRows(Interpreter interpreter,
@@ -153,91 +179,44 @@ public class InterpreterTest {
 
   /** Tests executing a simple plan using an interpreter. */
   @Test public void testInterpretTable() throws Exception {
-    SqlNode parse =
-        planner.parse("select * from \"hr\".\"emps\" order by \"empid\"");
-
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).rel;
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRows(interpreter,
-        "[100, 10, Bill, 10000.0, 1000]",
-        "[110, 10, Theodore, 11500.0, 250]",
-        "[150, 10, Sebastian, 7000.0, null]",
-        "[200, 20, Eric, 8000.0, 500]");
+    sql("select * from \"hr\".\"emps\" order by \"empid\"")
+        .returnsRows("[100, 10, Bill, 10000.0, 1000]",
+            "[110, 10, Theodore, 11500.0, 250]",
+            "[150, 10, Sebastian, 7000.0, null]",
+            "[200, 20, Eric, 8000.0, 500]");
   }
 
   /** Tests executing a plan on a
    * {@link org.apache.calcite.schema.ScannableTable} using an interpreter. */
   @Test public void testInterpretScannableTable() throws Exception {
     rootSchema.add("beatles", new ScannableTableTest.BeatlesTable());
-    SqlNode parse =
-        planner.parse("select * from \"beatles\" order by \"i\"");
-
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).rel;
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRows(interpreter,
-        "[4, John]",
-        "[4, Paul]",
-        "[5, Ringo]",
-        "[6, George]");
+    sql("select * from \"beatles\" order by \"i\"")
+        .returnsRows("[4, John]", "[4, Paul]", "[5, Ringo]", "[6, George]");
   }
 
   @Test public void testAggregateCount() throws Exception {
     rootSchema.add("beatles", new ScannableTableTest.BeatlesTable());
-    SqlNode parse =
-        planner.parse("select  count(*) from \"beatles\"");
-
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).rel;
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRows(interpreter,
-        "[4]");
+    sql("select count(*) from \"beatles\"")
+        .returnsRows("[4]");
   }
 
   @Test public void testAggregateMax() throws Exception {
     rootSchema.add("beatles", new ScannableTableTest.BeatlesTable());
-    SqlNode parse =
-        planner.parse("select  max(\"i\") from \"beatles\"");
-
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).rel;
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRows(interpreter,
-        "[6]");
+    sql("select max(\"i\") from \"beatles\"")
+        .returnsRows("[6]");
   }
 
   @Test public void testAggregateMin() throws Exception {
     rootSchema.add("beatles", new ScannableTableTest.BeatlesTable());
-    SqlNode parse =
-        planner.parse("select  min(\"i\") from \"beatles\"");
-
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).rel;
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRows(interpreter,
-        "[4]");
+    sql("select min(\"i\") from \"beatles\"")
+        .returnsRows("[4]");
   }
 
   @Test public void testAggregateGroup() throws Exception {
     rootSchema.add("beatles", new ScannableTableTest.BeatlesTable());
-    SqlNode parse =
-        planner.parse("select \"j\", count(*) from \"beatles\" group by \"j\"");
-
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).rel;
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRowsUnordered(interpreter,
-        "[George, 1]",
-        "[Paul, 1]",
-        "[John, 1]",
-        "[Ringo, 1]");
+    sql("select \"j\", count(*) from \"beatles\" group by \"j\"")
+        .returnsRowsUnordered("[George, 1]", "[Paul, 1]", "[John, 1]",
+            "[Ringo, 1]");
   }
 
   @Test public void testAggregateGroupFilter() throws Exception {
@@ -245,62 +224,40 @@ public class InterpreterTest {
     final String sql = "select \"j\",\n"
         + "  count(*) filter (where char_length(\"j\") > 4)\n"
         + "from \"beatles\" group by \"j\"";
-    SqlNode parse = planner.parse(sql);
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).rel;
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRowsUnordered(interpreter,
-        "[George, 1]",
-        "[Paul, 0]",
-        "[John, 0]",
-        "[Ringo, 1]");
+    sql(sql)
+        .returnsRowsUnordered("[George, 1]",
+            "[Paul, 0]",
+            "[John, 0]",
+            "[Ringo, 1]");
   }
 
   /** Tests executing a plan on a single-column
    * {@link org.apache.calcite.schema.ScannableTable} using an interpreter. */
   @Test public void testInterpretSimpleScannableTable() throws Exception {
     rootSchema.add("simple", new ScannableTableTest.SimpleTable());
-    SqlNode parse =
-        planner.parse("select * from \"simple\" limit 2");
-
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).rel;
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRows(interpreter, "[0]", "[10]");
+    sql("select * from \"simple\" limit 2")
+        .returnsRows("[0]", "[10]");
   }
 
   /** Tests executing a UNION ALL query using an interpreter. */
   @Test public void testInterpretUnionAll() throws Exception {
     rootSchema.add("simple", new ScannableTableTest.SimpleTable());
-    SqlNode parse =
-        planner.parse("select * from \"simple\"\n"
-            + "union all\n"
-            + "select * from \"simple\"\n");
-
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).rel;
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRows(interpreter,
-        "[0]", "[10]", "[20]", "[30]", "[0]", "[10]", "[20]", "[30]");
+    final String sql = "select * from \"simple\"\n"
+        + "union all\n"
+        + "select * from \"simple\"";
+    sql(sql).returnsRowsUnordered("[0]", "[10]", "[20]", "[30]", "[0]", "[10]",
+        "[20]", "[30]");
   }
 
   /** Tests executing a UNION query using an interpreter. */
   @Test public void testInterpretUnion() throws Exception {
     rootSchema.add("simple", new ScannableTableTest.SimpleTable());
-    SqlNode parse =
-        planner.parse("select * from \"simple\"\n"
-            + "union\n"
-            + "select * from \"simple\"\n");
-
-    SqlNode validate = planner.validate(parse);
-    RelNode convert = planner.rel(validate).rel;
-
-    final Interpreter interpreter = new Interpreter(dataContext, convert);
-    assertRows(interpreter, "[0]", "[10]", "[20]", "[30]");
+    final String sql = "select * from \"simple\"\n"
+        + "union\n"
+        + "select * from \"simple\"";
+    sql(sql).returnsRowsUnordered("[0]", "[10]", "[20]", "[30]");
   }
+
 }
 
 // End InterpreterTest.java


[calcite] 02/02: [CALCITE-3407] Implement MINUS and INTERSECT in interpreter (Wang Yanlin)

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

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

commit ec9947ad8403b6b2d476b3b379f8f52776ebacc4
Author: yanlin-Lynn <19...@163.com>
AuthorDate: Mon Oct 14 11:08:33 2019 +0800

    [CALCITE-3407] Implement MINUS and INTERSECT in interpreter (Wang Yanlin)
    
    Obsolete UnionNode, replacing with SetOpNode.
    
    Close apache/calcite#1502
---
 .../org/apache/calcite/interpreter/Bindables.java  |   2 +-
 .../apache/calcite/interpreter/Interpreter.java    |   7 ++
 .../java/org/apache/calcite/interpreter/Nodes.java |   6 +-
 .../org/apache/calcite/interpreter/SetOpNode.java  |  87 ++++++++++++++++++
 .../org/apache/calcite/interpreter/UnionNode.java  |   3 +
 .../org/apache/calcite/test/InterpreterTest.java   | 102 +++++++++++++++++++++
 6 files changed, 203 insertions(+), 4 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/interpreter/Bindables.java b/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
index 11da5fc..9f7f6dc 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Bindables.java
@@ -566,7 +566,7 @@ public class Bindables {
     }
 
     public Node implement(InterpreterImplementor implementor) {
-      return new UnionNode(implementor.compiler, this);
+      return new SetOpNode(implementor.compiler, this);
     }
   }
 
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java b/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java
index cc50c04..0eef9aa 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Interpreter.java
@@ -536,6 +536,13 @@ public class Interpreter extends AbstractEnumerable<Object[]>
         for (Edge edge : edges2) {
           nodeInfo.sinks.put(edge, new ListSink(new ArrayDeque<>()));
         }
+      } else {
+        for (Edge edge : edges2) {
+          if (nodeInfo.sinks.containsKey(edge)) {
+            continue;
+          }
+          nodeInfo.sinks.put(edge, new ListSink(new ArrayDeque<>()));
+        }
       }
       if (edges.size() == 1) {
         return Iterables.getOnlyElement(nodeInfo.sinks.values());
diff --git a/core/src/main/java/org/apache/calcite/interpreter/Nodes.java b/core/src/main/java/org/apache/calcite/interpreter/Nodes.java
index 3010921..a169b05 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/Nodes.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/Nodes.java
@@ -23,9 +23,9 @@ import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.Match;
 import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SetOp;
 import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.core.Union;
 import org.apache.calcite.rel.core.Values;
 import org.apache.calcite.rel.core.Window;
 import org.apache.calcite.rex.RexNode;
@@ -75,8 +75,8 @@ public class Nodes {
       node = new SortNode(this, sort);
     }
 
-    public void visit(Union union) {
-      node = new UnionNode(this, union);
+    public void visit(SetOp setOp) {
+      node = new SetOpNode(this, setOp);
     }
 
     public void visit(Join join) {
diff --git a/core/src/main/java/org/apache/calcite/interpreter/SetOpNode.java b/core/src/main/java/org/apache/calcite/interpreter/SetOpNode.java
new file mode 100644
index 0000000..32b8adf
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/interpreter/SetOpNode.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.interpreter;
+
+import org.apache.calcite.rel.core.SetOp;
+
+import com.google.common.collect.HashMultiset;
+
+import java.util.Collection;
+import java.util.HashSet;
+
+/**
+ * Interpreter node that implements a
+ * {@link org.apache.calcite.rel.core.SetOp},
+ * including {@link org.apache.calcite.rel.core.Minus},
+ * {@link org.apache.calcite.rel.core.Union} and
+ * {@link org.apache.calcite.rel.core.Intersect}.
+ */
+public class SetOpNode implements Node {
+  private final Source leftSource;
+  private final Source rightSource;
+  private final Sink sink;
+  private final SetOp setOp;
+
+  public SetOpNode(Compiler compiler, SetOp setOp) {
+    leftSource = compiler.source(setOp, 0);
+    rightSource = compiler.source(setOp, 1);
+    sink = compiler.sink(setOp);
+    this.setOp = setOp;
+  }
+
+  @Override public void run() throws InterruptedException {
+    final Collection<Row> leftRows;
+    final Collection<Row> rightRows;
+    if (setOp.all) {
+      leftRows = HashMultiset.create();
+      rightRows = HashMultiset.create();
+    } else {
+      leftRows = new HashSet<>();
+      rightRows = new HashSet<>();
+    }
+    Row row;
+    while ((row = leftSource.receive()) != null) {
+      leftRows.add(row);
+    }
+    while ((row = rightSource.receive()) != null) {
+      rightRows.add(row);
+    }
+    switch (setOp.kind) {
+    case INTERSECT:
+      for (Row leftRow : leftRows) {
+        if (rightRows.remove(leftRow)) {
+          sink.send(leftRow);
+        }
+      }
+      break;
+    case EXCEPT:
+      for (Row leftRow : leftRows) {
+        if (!rightRows.remove(leftRow)) {
+          sink.send(leftRow);
+        }
+      }
+      break;
+    case UNION:
+      leftRows.addAll(rightRows);
+      for (Row r : leftRows) {
+        sink.send(r);
+      }
+    }
+  }
+}
+
+// End SetOpNode.java
diff --git a/core/src/main/java/org/apache/calcite/interpreter/UnionNode.java b/core/src/main/java/org/apache/calcite/interpreter/UnionNode.java
index 98d5c52..f31f1c7 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/UnionNode.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/UnionNode.java
@@ -26,7 +26,10 @@ import java.util.Set;
 /**
  * Interpreter node that implements a
  * {@link org.apache.calcite.rel.core.Union}.
+ *
+ * @deprecated Use {@link org.apache.calcite.interpreter.SetOpNode}
  */
+@Deprecated // to be removed before 2.0
 public class UnionNode implements Node {
   private final ImmutableList<Source> sources;
   private final Sink sink;
diff --git a/core/src/test/java/org/apache/calcite/test/InterpreterTest.java b/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
index de284dd..281717c 100644
--- a/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/InterpreterTest.java
@@ -258,6 +258,108 @@ public class InterpreterTest {
     sql(sql).returnsRowsUnordered("[0]", "[10]", "[20]", "[30]");
   }
 
+  @Test public void testInterpretUnionWithNullValue() throws Exception {
+    final String sql = "select * from\n"
+        + "(select x, y from (values (cast(NULL as int), cast(NULL as varchar(1))),\n"
+        + "(cast(NULL as int), cast(NULL as varchar(1)))) as t(x, y))\n"
+        + "union\n"
+        + "(select x, y from (values (cast(NULL as int), cast(NULL as varchar(1)))) as t2(x, y))";
+    sql(sql).returnsRows("[null, null]");
+  }
+
+  @Test public void testInterpretUnionAllWithNullValue() throws Exception {
+    final String sql = "select * from\n"
+        + "(select x, y from (values (cast(NULL as int), cast(NULL as varchar(1))),\n"
+        + "(cast(NULL as int), cast(NULL as varchar(1)))) as t(x, y))\n"
+        + "union all\n"
+        + "(select x, y from (values (cast(NULL as int), cast(NULL as varchar(1)))) as t2(x, y))";
+    sql(sql).returnsRows("[null, null]", "[null, null]", "[null, null]");
+  }
+
+  @Test public void testInterpretIntersect() throws Exception {
+    final String sql = "select * from\n"
+        + "(select x, y from (values (1, 'a'), (1, 'a'), (2, 'b'), (3, 'c')) as t(x, y))\n"
+        + "intersect\n"
+        + "(select x, y from (values (1, 'a'), (2, 'c'), (4, 'x')) as t2(x, y))";
+    sql(sql).returnsRows("[1, a]");
+  }
+
+  @Test public void testInterpretIntersectAll() throws Exception {
+    final String sql = "select * from\n"
+        + "(select x, y from (values (1, 'a'), (1, 'a'), (2, 'b'), (3, 'c')) as t(x, y))\n"
+        + "intersect all\n"
+        + "(select x, y from (values (1, 'a'), (2, 'c'), (4, 'x')) as t2(x, y))";
+    sql(sql).returnsRows("[1, a]");
+  }
+
+  @Test public void testInterpretIntersectWithNullValue() throws Exception {
+    final String sql = "select * from\n"
+        + "(select x, y from (values (cast(NULL as int), cast(NULL as varchar(1))),\n"
+        + " (cast(NULL as int), cast(NULL as varchar(1)))) as t(x, y))\n"
+        + "intersect\n"
+        + "(select x, y from (values (cast(NULL as int), cast(NULL as varchar(1)))) as t2(x, y))";
+    sql(sql).returnsRows("[null, null]");
+  }
+
+  @Test public void testInterpretIntersectAllWithNullValue() throws Exception {
+    final String sql = "select * from\n"
+        + "(select x, y from (values (cast(NULL as int), cast(NULL as varchar(1))),\n"
+        + " (cast(NULL as int), cast(NULL as varchar(1)))) as t(x, y))\n"
+        + "intersect all\n"
+        + "(select x, y from (values (cast(NULL as int), cast(NULL as varchar(1)))) as t2(x, y))";
+    sql(sql).returnsRows("[null, null]");
+  }
+
+  @Test public void testInterpretMinus() throws Exception {
+    final String sql = "select * from\n"
+        + "(select x, y from (values (1, 'a'), (2, 'b'), (2, 'b'), (3, 'c')) as t(x, y))\n"
+        + "except\n"
+        + "(select x, y from (values (1, 'a'), (2, 'c'), (4, 'x')) as t2(x, y))";
+    sql(sql).returnsRows("[2, b]", "[3, c]");
+  }
+
+  @Test public void testDuplicateRowInterpretMinus() throws Exception {
+    final String sql = "select * from\n"
+        + "(select x, y from (values (2, 'b'), (2, 'b')) as t(x, y))\n"
+        + "except\n"
+        + "(select x, y from (values (2, 'b')) as t2(x, y))";
+    sql(sql).returnsRows();
+  }
+
+  @Test public void testInterpretMinusAll() throws Exception {
+    final String sql = "select * from\n"
+        + "(select x, y from (values (1, 'a'), (2, 'b'), (2, 'b'), (3, 'c')) as t(x, y))\n"
+        + "except all\n"
+        + "(select x, y from (values (1, 'a'), (2, 'c'), (4, 'x')) as t2(x, y))";
+    sql(sql).returnsRows("[2, b]", "[2, b]", "[3, c]");
+  }
+
+  @Test public void testDuplicateRowInterpretMinusAll() throws Exception {
+    final String sql = "select * from\n"
+        + "(select x, y from (values (2, 'b'), (2, 'b')) as t(x, y))\n"
+        + "except all\n"
+        + "(select x, y from (values (2, 'b')) as t2(x, y))\n";
+    sql(sql).returnsRows("[2, b]");
+  }
+
+  @Test public void testInterpretMinusAllWithNullValue() throws Exception {
+    final String sql = "select * from\n"
+        + "(select x, y from (values (cast(NULL as int), cast(NULL as varchar(1))),\n"
+        + " (cast(NULL as int), cast(NULL as varchar(1)))) as t(x, y))\n"
+        + "except all\n"
+        + "(select x, y from (values (cast(NULL as int), cast(NULL as varchar(1)))) as t2(x, y))\n";
+    sql(sql).returnsRows("[null, null]");
+  }
+
+  @Test public void testInterpretMinusWithNullValue() throws Exception {
+    final String sql = "select * from\n"
+        + "(select x, y from (values (cast(NULL as int), cast(NULL as varchar(1))),\n"
+        + "(cast(NULL as int), cast(NULL as varchar(1)))) as t(x, y))\n"
+        + "except\n"
+        + "(select x, y from (values (cast(NULL as int), cast(NULL as varchar(1)))) as t2(x, y))\n";
+    sql(sql).returnsRows();
+  }
+
 }
 
 // End InterpreterTest.java