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 2014/06/29 02:07:52 UTC

git commit: [OPTIQ-300] Support multiple parameters in COUNT(DISTINCT x, y, ...)

Repository: incubator-optiq
Updated Branches:
  refs/heads/master 2db452ecb -> 4f80e6424


[OPTIQ-300] Support multiple parameters in COUNT(DISTINCT x, y, ...)


Project: http://git-wip-us.apache.org/repos/asf/incubator-optiq/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-optiq/commit/4f80e642
Tree: http://git-wip-us.apache.org/repos/asf/incubator-optiq/tree/4f80e642
Diff: http://git-wip-us.apache.org/repos/asf/incubator-optiq/diff/4f80e642

Branch: refs/heads/master
Commit: 4f80e6424db2054a9163f2f96331d23acd7a4735
Parents: 2db452e
Author: Julian Hyde <ju...@gmail.com>
Authored: Sat Jun 28 17:06:52 2014 -0700
Committer: Julian Hyde <ju...@gmail.com>
Committed: Sat Jun 28 17:06:52 2014 -0700

----------------------------------------------------------------------
 .../net/hydromatic/optiq/BuiltinMethod.java     |   1 +
 .../main/java/org/eigenbase/rex/RexBuilder.java |  54 ++--
 .../eigenbase/sql/fun/SqlCountAggFunction.java  |   4 +-
 .../org/eigenbase/sql/type/OperandTypes.java    |  39 +--
 .../sql/type/SqlOperandCountRanges.java         |  20 +-
 .../eigenbase/sql/validate/SqlValidator.java    |   3 +
 .../net/hydromatic/optiq/test/JdbcTest.java     | 100 ++++++-
 .../eigenbase/sql/test/SqlOperatorBaseTest.java |   9 +-
 .../org/eigenbase/test/SqlValidatorTest.java    |  16 +-
 core/src/test/resources/sql/agg.oq              | 265 +++++++------------
 10 files changed, 272 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/4f80e642/core/src/main/java/net/hydromatic/optiq/BuiltinMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/BuiltinMethod.java b/core/src/main/java/net/hydromatic/optiq/BuiltinMethod.java
index 7730645..1aef089 100644
--- a/core/src/main/java/net/hydromatic/optiq/BuiltinMethod.java
+++ b/core/src/main/java/net/hydromatic/optiq/BuiltinMethod.java
@@ -96,6 +96,7 @@ public enum BuiltinMethod {
   ARRAYS_AS_LIST(FlatLists.class, "of", Object[].class),
   LIST2(FlatLists.class, "of", Object.class, Object.class),
   LIST3(FlatLists.class, "of", Object.class, Object.class, Object.class),
+  IDENTITY_COMPARER(Functions.class, "identityComparer"),
   IDENTITY_SELECTOR(Functions.class, "identitySelector"),
   AS_ENUMERABLE(Linq4j.class, "asEnumerable", Object[].class),
   AS_ENUMERABLE2(Linq4j.class, "asEnumerable", Iterable.class),

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/4f80e642/core/src/main/java/org/eigenbase/rex/RexBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rex/RexBuilder.java b/core/src/main/java/org/eigenbase/rex/RexBuilder.java
index a05df07..7d3f92a 100644
--- a/core/src/main/java/org/eigenbase/rex/RexBuilder.java
+++ b/core/src/main/java/org/eigenbase/rex/RexBuilder.java
@@ -262,32 +262,15 @@ public class RexBuilder {
 
   /**
    * Creates a reference to an aggregate call, checking for repeated calls.
+   *
+   * <p>Argument types help to optimize for repeated aggregates.
+   * For instance count(42) is equivalent to count(*).</p>
+   *
    * @param aggCall aggregate call to be added
    * @param groupCount number of groups in the aggregate relation
    * @param aggCalls destination list of aggregate calls
    * @param aggCallMapping the dictionary of already added calls
-   * @return Rex expression for the given aggregate call
-   * @deprecated Use {@link #addAggCall(org.eigenbase.rel.AggregateCall, int, java.util.List, java.util.Map, java.util.List)}
-   * Will be removed before optiq-0.9.
-   */
-  @Deprecated
-  public RexNode addAggCall(
-      AggregateCall aggCall,
-      int groupCount,
-      List<AggregateCall> aggCalls,
-      Map<AggregateCall, RexNode> aggCallMapping) {
-    Bug.upgrade("remove before optiq-0.9");
-    return addAggCall(aggCall, groupCount, aggCalls, aggCallMapping, null);
-  }
-
-  /**
-   * Creates a reference to an aggregate call, checking for repeated calls.
-   * Argument types help to optimize for repeated aggregates.
-   * For instance count(42) is equivalent to count(*)
-   * @param aggCall aggregate call to be added
-   * @param groupCount number of groups in the aggregate relation
-   * @param aggCalls destination list of aggregate calls
-   * @param aggCallMapping the dictionary of already added calls
+   * @param aggArgTypes Argument types, not null
    * @return Rex expression for the given aggregate call
    */
   public RexNode addAggCall(
@@ -295,20 +278,14 @@ public class RexBuilder {
       int groupCount,
       List<AggregateCall> aggCalls,
       Map<AggregateCall, RexNode> aggCallMapping,
-      List<RelDataType> aggArgTypes) {
+      final List<RelDataType> aggArgTypes) {
     if (aggCall.getAggregation() instanceof SqlCountAggFunction
-        && aggArgTypes != null && !aggArgTypes.isEmpty()
         && !aggCall.isDistinct()) {
-      boolean hasNotNullArg = false;
-      for (RelDataType type : aggArgTypes) {
-        if (!type.isNullable()) {
-          hasNotNullArg = true;
-          break;
-        }
-      }
-      if (hasNotNullArg) {
+      final List<Integer> notNullArgList =
+          nullableArgs(aggCall.getArgList(), aggArgTypes);
+      if (!notNullArgList.equals(aggCall.getArgList())) {
         aggCall = new AggregateCall(aggCall.getAggregation(),
-            aggCall.isDistinct(), ImmutableList.<Integer>of(),
+            aggCall.isDistinct(), notNullArgList,
             aggCall.getType(), aggCall.getName());
       }
     }
@@ -322,6 +299,17 @@ public class RexBuilder {
     return rex;
   }
 
+  private static List<Integer> nullableArgs(List<Integer> list0,
+      List<RelDataType> types) {
+    final List<Integer> list = new ArrayList<Integer>();
+    for (Pair<Integer, RelDataType> pair : Pair.zip(list0, types)) {
+      if (pair.right.isNullable()) {
+        list.add(pair.left);
+      }
+    }
+    return list;
+  }
+
   /**
    * Creates a call to a windowed agg.
    */

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/4f80e642/core/src/main/java/org/eigenbase/sql/fun/SqlCountAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql/fun/SqlCountAggFunction.java b/core/src/main/java/org/eigenbase/sql/fun/SqlCountAggFunction.java
index 4034394..5f1f4ce 100644
--- a/core/src/main/java/org/eigenbase/sql/fun/SqlCountAggFunction.java
+++ b/core/src/main/java/org/eigenbase/sql/fun/SqlCountAggFunction.java
@@ -42,7 +42,9 @@ public class SqlCountAggFunction extends SqlAggFunction {
         SqlKind.OTHER_FUNCTION,
         ReturnTypes.BIGINT,
         null,
-        OperandTypes.ANY,
+        SqlValidator.STRICT
+            ? OperandTypes.ANY
+            : OperandTypes.ONE_OR_MORE,
         SqlFunctionCategory.NUMERIC);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/4f80e642/core/src/main/java/org/eigenbase/sql/type/OperandTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql/type/OperandTypes.java b/core/src/main/java/org/eigenbase/sql/type/OperandTypes.java
index 2e17cac..a2e2d7d 100644
--- a/core/src/main/java/org/eigenbase/sql/type/OperandTypes.java
+++ b/core/src/main/java/org/eigenbase/sql/type/OperandTypes.java
@@ -99,21 +99,30 @@ public abstract class OperandTypes {
    * number or type of operands.
    */
   public static final SqlOperandTypeChecker VARIADIC =
-      new SqlOperandTypeChecker() {
-        public boolean checkOperandTypes(
-            SqlCallBinding callBinding,
-            boolean throwOnFailure) {
-          return true;
-        }
-
-        public SqlOperandCountRange getOperandCountRange() {
-          return SqlOperandCountRanges.any();
-        }
-
-        public String getAllowedSignatures(SqlOperator op, String opName) {
-          return opName + "(...)";
-        }
-      };
+      variadic(SqlOperandCountRanges.any());
+
+  /** Operand type-checking strategy that allows one or more operands. */
+  public static final SqlOperandTypeChecker ONE_OR_MORE =
+      variadic(SqlOperandCountRanges.from(1));
+
+  private static SqlOperandTypeChecker variadic(
+      final SqlOperandCountRange range) {
+    return new SqlOperandTypeChecker() {
+      public boolean checkOperandTypes(
+          SqlCallBinding callBinding,
+          boolean throwOnFailure) {
+        return range.isValidCount(callBinding.getOperandCount());
+      }
+
+      public SqlOperandCountRange getOperandCountRange() {
+        return range;
+      }
+
+      public String getAllowedSignatures(SqlOperator op, String opName) {
+        return opName + "(...)";
+      }
+    };
+  }
 
   public static final SqlSingleOperandTypeChecker BOOLEAN =
       family(SqlTypeFamily.BOOLEAN);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/4f80e642/core/src/main/java/org/eigenbase/sql/type/SqlOperandCountRanges.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql/type/SqlOperandCountRanges.java b/core/src/main/java/org/eigenbase/sql/type/SqlOperandCountRanges.java
index 585503a..2934bdd 100644
--- a/core/src/main/java/org/eigenbase/sql/type/SqlOperandCountRanges.java
+++ b/core/src/main/java/org/eigenbase/sql/type/SqlOperandCountRanges.java
@@ -32,22 +32,12 @@ public abstract class SqlOperandCountRanges {
     return new RangeImpl(min, max);
   }
 
-  public static SqlOperandCountRange any() {
-    return new VariadicImpl();
+  public static SqlOperandCountRange from(int min) {
+    return new RangeImpl(min, -1);
   }
 
-  private static class VariadicImpl implements SqlOperandCountRange {
-    public boolean isValidCount(int count) {
-      return true;
-    }
-
-    public int getMin() {
-      return 0;
-    }
-
-    public int getMax() {
-      return -1;
-    }
+  public static SqlOperandCountRange any() {
+    return new RangeImpl(0, -1);
   }
 
   private static class RangeImpl implements SqlOperandCountRange {
@@ -60,7 +50,7 @@ public abstract class SqlOperandCountRanges {
     }
 
     public boolean isValidCount(int count) {
-      return count >= min && count <= max;
+      return count >= min && (max == -1 || count <= max);
     }
 
     public int getMin() {

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/4f80e642/core/src/main/java/org/eigenbase/sql/validate/SqlValidator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/sql/validate/SqlValidator.java b/core/src/main/java/org/eigenbase/sql/validate/SqlValidator.java
index 7962465..e25ba22 100644
--- a/core/src/main/java/org/eigenbase/sql/validate/SqlValidator.java
+++ b/core/src/main/java/org/eigenbase/sql/validate/SqlValidator.java
@@ -73,6 +73,9 @@ import org.eigenbase.util.*;
  * names in a particular clause of a SQL statement.</p>
  */
 public interface SqlValidator {
+  /** Whether to follow the SQL standard strictly. */
+  boolean STRICT = Util.first(Boolean.getBoolean("optiq.strict.sql"), false);
+
   //~ Methods ----------------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/4f80e642/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java b/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
index 81198b6..2e9be1c 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
@@ -2335,13 +2335,83 @@ public class JdbcTest {
             "empid=110; deptno=10; name=Theodore; salary=11500.0; commission=250; deptno0=40; name0=HR; employees=[Employee [empid: 200, deptno: 20, name: Eric]]");
   }
 
+  @Test public void testDistinctCountSimple() {
+    final String s =
+        "select count(distinct \"sales_fact_1997\".\"unit_sales\") as \"m0\"\n"
+        + "from \"sales_fact_1997\" as \"sales_fact_1997\"";
+    OptiqAssert.that()
+        .with(OptiqAssert.Config.FOODMART_CLONE)
+        .query(s)
+        .explainContains(
+            "EnumerableAggregateRel(group=[{}], m0=[COUNT($0)])\n"
+            + "  EnumerableAggregateRel(group=[{0}])\n"
+            + "    EnumerableCalcRel(expr#0..7=[{inputs}], unit_sales=[$t7])\n"
+            + "      EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]])")
+        .returns("m0=6\n");
+  }
+
+  @Test public void testDistinctCount2() {
+    final String s =
+        "select cast(\"unit_sales\" as integer) as \"u\",\n"
+        + " count(distinct \"sales_fact_1997\".\"customer_id\") as \"m0\"\n"
+        + "from \"sales_fact_1997\" as \"sales_fact_1997\"\n"
+        + "group by \"unit_sales\"";
+    OptiqAssert.that()
+        .with(OptiqAssert.Config.FOODMART_CLONE)
+        .query(s)
+        .explainContains(
+            "EnumerableCalcRel(expr#0..1=[{inputs}], expr#2=[CAST($t0):INTEGER NOT NULL], u=[$t2], m0=[$t1])\n"
+            + "  EnumerableAggregateRel(group=[{0}], m0=[COUNT($1)])\n"
+            + "    EnumerableAggregateRel(group=[{0, 1}])\n"
+            + "      EnumerableCalcRel(expr#0..7=[{inputs}], unit_sales=[$t7], customer_id=[$t2])\n"
+            + "        EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]])")
+        .returnsUnordered(
+            "u=1; m0=523",
+            "u=5; m0=1059",
+            "u=4; m0=4459",
+            "u=6; m0=19",
+            "u=3; m0=4895",
+            "u=2; m0=4735");
+  }
+
   @Test public void testDistinctCount() {
     final String s =
-        "select \"time_by_day\".\"the_year\" as \"c0\", sum(\"sales_fact_1997\".\"unit_sales\") as \"m0\" from \"time_by_day\" as \"time_by_day\", \"sales_fact_1997\" as \"sales_fact_1997\" where \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\" and \"time_by_day\".\"the_year\" = 1997 group by \"time_by_day\".\"the_year\"";
+        "select \"time_by_day\".\"the_year\" as \"c0\",\n"
+        + " count(distinct \"sales_fact_1997\".\"unit_sales\") as \"m0\"\n"
+        + "from \"time_by_day\" as \"time_by_day\",\n"
+        + " \"sales_fact_1997\" as \"sales_fact_1997\"\n"
+        + "where \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\"\n"
+        + "and \"time_by_day\".\"the_year\" = 1997\n"
+        + "group by \"time_by_day\".\"the_year\"";
+    OptiqAssert.that()
+        .with(OptiqAssert.Config.FOODMART_CLONE)
+        .query(s)
+        .explainContains(
+            "EnumerableAggregateRel(group=[{0}], m0=[COUNT($1)])\n"
+            + "  EnumerableAggregateRel(group=[{0, 1}])\n"
+            + "    EnumerableCalcRel(expr#0..3=[{inputs}], c0=[$t3], unit_sales=[$t1])\n"
+            + "      EnumerableJoinRel(condition=[=($0, $2)], joinType=[inner])\n"
+            + "        EnumerableCalcRel(expr#0..7=[{inputs}], time_id=[$t1], unit_sales=[$t7])\n"
+            + "          EnumerableTableAccessRel(table=[[foodmart2, sales_fact_1997]])\n"
+            + "        EnumerableCalcRel(expr#0..9=[{inputs}], expr#10=[CAST($t4):INTEGER], expr#11=[1997], expr#12=[=($t10, $t11)], time_id=[$t0], the_year=[$t4], $condition=[$t12])\n"
+            + "          EnumerableTableAccessRel(table=[[foodmart2, time_by_day]])")
+        .returns("c0=1997; m0=6\n");
+  }
+
+  @Test public void testDistinctCountComposite() {
+    final String s =
+        "select \"time_by_day\".\"the_year\" as \"c0\",\n"
+        + " count(distinct \"sales_fact_1997\".\"product_id\",\n"
+        + "       \"sales_fact_1997\".\"customer_id\") as \"m0\"\n"
+        + "from \"time_by_day\" as \"time_by_day\",\n"
+        + " \"sales_fact_1997\" as \"sales_fact_1997\"\n"
+        + "where \"sales_fact_1997\".\"time_id\" = \"time_by_day\".\"time_id\"\n"
+        + "and \"time_by_day\".\"the_year\" = 1997\n"
+        + "group by \"time_by_day\".\"the_year\"";
     OptiqAssert.that()
         .with(OptiqAssert.Config.FOODMART_CLONE)
         .query(s)
-        .returns("c0=1997; m0=266773.0000\n");
+        .returns("c0=1997; m0=85452\n");
   }
 
   /** A difficult query: an IN list so large that the planner promotes it
@@ -2727,7 +2797,6 @@ public class JdbcTest {
             + "from \"hr\".\"emps\"\n")
         .returnsUnordered(
             "C=false; deptno=10",
-            "C=false; deptno=10",
             "C=true; deptno=10",
             "C=true; deptno=20")
         .planContains(".distinct(");
@@ -2826,6 +2895,22 @@ public class JdbcTest {
         .returns("CS=0; CS2=0\n");
   }
 
+  /** Tests that {@code count(deptno, commission, commission + 1)} is reduced to
+   * {@code count(commission, commission + 1)}, because deptno is NOT NULL. */
+  @Test public void testReduceCompositeCountNotNullable() {
+    OptiqAssert.that()
+        .with(OptiqAssert.Config.REGULAR)
+        .query(
+            "select\n"
+            + " count(\"deptno\", \"commission\", \"commission\" + 1) as cs\n"
+            + "from \"hr\".\"emps\"")
+        .explainContains(
+            "EnumerableAggregateRel(group=[{}], CS=[COUNT($0, $1)])\n"
+            + "  EnumerableCalcRel(expr#0..4=[{inputs}], expr#5=[1], expr#6=[+($t4, $t5)], commission=[$t4], $f2=[$t6])\n"
+            + "    EnumerableTableAccessRel(table=[[hr, emps]])")
+        .returns("CS=3\n");
+  }
+
   /** Tests sorting by a column that is already sorted. */
   @Test public void testOrderByOnSortedTable() {
     OptiqAssert.that()
@@ -4034,6 +4119,10 @@ public class JdbcTest {
         .returnsUnordered(lines);
   }
 
+  @Test public void testRunAgg() throws Exception {
+    checkRun("sql/agg.oq");
+  }
+
   @Test public void testRunOuter() throws Exception {
     checkRun("sql/outer.oq");
   }
@@ -4070,6 +4159,11 @@ public class JdbcTest {
                   .with(OptiqAssert.Config.REGULAR)
                   .connect();
             }
+            if (name.equals("foodmart")) {
+              return OptiqAssert.that()
+                  .with(OptiqAssert.Config.FOODMART_CLONE)
+                  .connect();
+            }
             if (name.equals("post")) {
               return OptiqAssert.that()
                   .with(OptiqAssert.Config.REGULAR)

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/4f80e642/core/src/test/java/org/eigenbase/sql/test/SqlOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/eigenbase/sql/test/SqlOperatorBaseTest.java b/core/src/test/java/org/eigenbase/sql/test/SqlOperatorBaseTest.java
index 547475f..616c638 100644
--- a/core/src/test/java/org/eigenbase/sql/test/SqlOperatorBaseTest.java
+++ b/core/src/test/java/org/eigenbase/sql/test/SqlOperatorBaseTest.java
@@ -4288,19 +4288,14 @@ public abstract class SqlOperatorBaseTest {
         "^COUNT()^",
         "Invalid number of arguments to function 'COUNT'. Was expecting 1 arguments",
         false);
-    tester.checkFails(
-        "^COUNT(1, 2)^",
-        "Invalid number of arguments to function 'COUNT'. Was expecting 1 arguments",
-        false);
+    tester.checkType("count(1, 2)", "BIGINT NOT NULL");
+    tester.checkType("count(1, 2, 'x', 'y')", "BIGINT NOT NULL");
     final String[] values = {"0", "CAST(null AS INTEGER)", "1", "0"};
     tester.checkAgg(
         "COUNT(x)",
         values,
         3,
         (double) 0);
-    if (!enable) {
-      return;
-    }
     tester.checkAgg(
         "COUNT(CASE x WHEN 0 THEN NULL ELSE -1 END)",
         values,

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/4f80e642/core/src/test/java/org/eigenbase/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/eigenbase/test/SqlValidatorTest.java b/core/src/test/java/org/eigenbase/test/SqlValidatorTest.java
index c609f82..374553a 100644
--- a/core/src/test/java/org/eigenbase/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/eigenbase/test/SqlValidatorTest.java
@@ -6023,10 +6023,24 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     check("select count(sal) from emp");
     check("select count(1) from emp");
     checkFails(
-        "select ^count(sal,ename)^ from emp",
+        "select ^count()^ from emp",
         "Invalid number of arguments to function 'COUNT'. Was expecting 1 arguments");
   }
 
+  @Test public void testCountCompositeFunction() {
+    check("select count(ename, deptno) from emp");
+    checkFails("select count(ename, deptno, ^gender^) from emp",
+        "Column 'GENDER' not found in any table");
+    check("select count(ename, 1, deptno) from emp");
+    check("select count(distinct ename, 1, deptno) from emp");
+    checkFails(
+        "select count(deptno, *) from emp",
+        "(?s).*Encountered \", \\*\" at .*");
+    checkFails(
+        "select count(*, deptno) from emp",
+        "(?s).*Encountered \",\" at .*");
+  }
+
   @Test public void testLastFunction() {
     check("select LAST_VALUE(sal) over (order by empno) from emp");
     check("select LAST_VALUE(ename) over (order by empno) from emp");

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/4f80e642/core/src/test/resources/sql/agg.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/agg.oq b/core/src/test/resources/sql/agg.oq
index b718949..5597cb2 100644
--- a/core/src/test/resources/sql/agg.oq
+++ b/core/src/test/resources/sql/agg.oq
@@ -1,200 +1,137 @@
-# outer.oq - Various kinds of outer join
+# agg.oq - Aggregate functions
 !use post
 !set outputformat mysql
 
-select * from emp;
-+-------+--------+--------+
-| ENAME | DEPTNO | GENDER |
-+-------+--------+--------+
-| Jane  |     10 | F      |
-| Bob   |     10 | M      |
-| Eric  |     20 | M      |
-| Susan |     30 | F      |
-| Alice |     30 | F      |
-| Adam  |     50 | M      |
-| Eve   |     50 | F      |
-| Grace |     60 | F      |
-| Wilma |        | F      |
-+-------+--------+--------+
-(9 rows)
+# count(*) returns number of rows in table
+select count(ename) as c from emp;
++---+
+| C |
++---+
+| 9 |
++---+
+(1 row)
 
 !ok
-select * from emp join dept on emp.deptno = dept.deptno;
-+-------+--------+--------+---------+-------------+
-| ENAME | DEPTNO | GENDER | DEPTNO0 | DNAME       |
-+-------+--------+--------+---------+-------------+
-| Jane  |     10 | F      |      10 | Sales       |
-| Bob   |     10 | M      |      10 | Sales       |
-| Eric  |     20 | M      |      20 | Marketing   |
-| Susan |     30 | F      |      30 | Engineering |
-| Alice |     30 | F      |      30 | Engineering |
-+-------+--------+--------+---------+-------------+
-(5 rows)
-
-!ok
-
-# The following test is disabled, because we cannot handle non-equi-join.
-# Following it are the results from MySQL.
-!if (false) {
-select * from emp join dept on emp.deptno = dept.deptno and emp.gender = 'F';
- ename | deptno | gender | deptno | dname
--------+--------+--------+--------+-------------
- Jane  |     10 | F      |     10 | Sales
- Susan |     30 | F      |     30 | Engineering
- Alice |     30 | F      |     30 | Engineering
-
-!ok
-!}
 
-select * from emp join dept on emp.deptno = dept.deptno where emp.gender = 'F';
-+-------+--------+--------+---------+-------------+
-| ENAME | DEPTNO | GENDER | DEPTNO0 | DNAME       |
-+-------+--------+--------+---------+-------------+
-| Jane  |     10 | F      |      10 | Sales       |
-| Susan |     30 | F      |      30 | Engineering |
-| Alice |     30 | F      |      30 | Engineering |
-+-------+--------+--------+---------+-------------+
-(3 rows)
+# count of not-nullable column same as count(*)
+select count(ename) as c from emp;
++---+
+| C |
++---+
+| 9 |
++---+
+(1 row)
 
 !ok
 
-select * from (select * from emp where gender ='F') as emp join dept on emp.deptno = dept.deptno;
-+-------+--------+--------+---------+-------------+
-| ENAME | DEPTNO | GENDER | DEPTNO0 | DNAME       |
-+-------+--------+--------+---------+-------------+
-| Jane  |     10 | F      |      10 | Sales       |
-| Susan |     30 | F      |      30 | Engineering |
-| Alice |     30 | F      |      30 | Engineering |
-+-------+--------+--------+---------+-------------+
-(3 rows)
+# count of nullable column
+select count(deptno) as c from emp;
++---+
+| C |
++---+
+| 8 |
++---+
+(1 row)
 
 !ok
 
-# The following test is disabled, because we cannot handle non-equi-join.
-# Following it are the results from MySQL.
+# composite count
 !if (false) {
-select * from emp left join dept on emp.deptno = dept.deptno and emp.gender = 'F';
- ename | deptno | gender | deptno | dname
--------+--------+--------+--------+-------------
- Jane  |     10 | F      |     10 | Sales
- Susan |     30 | F      |     30 | Engineering
- Alice |     30 | F      |     30 | Engineering
- Bob   |     10 | M      |   NULL | NULL
- Eric  |     20 | M      |   NULL | NULL
- Adam  |     50 | M      |   NULL | NULL
- Eve   |     50 | F      |   NULL | NULL
- Grace |     60 | F      |   NULL | NULL
+select count(deptno, ename, 1, deptno) as c from emp;
++---+
+| C |
++---+
+| 8 |
++---+
+(1 row)
+
 !ok
 !}
 
-select * from emp left join dept on emp.deptno = dept.deptno where emp.gender = 'F';
-+-------+--------+--------+---------+-------------+
-| ENAME | DEPTNO | GENDER | DEPTNO0 | DNAME       |
-+-------+--------+--------+---------+-------------+
-| Jane  |     10 | F      |      10 | Sales       |
-| Susan |     30 | F      |      30 | Engineering |
-| Alice |     30 | F      |      30 | Engineering |
-| Eve   |     50 | F      |         |             |
-| Grace |     60 | F      |         |             |
-| Wilma |        | F      |         |             |
-+-------+--------+--------+---------+-------------+
-(6 rows)
+!use foodmart
+
+select city, gender as c from "emps";
++---------------+---+
+| CITY          | C |
++---------------+---+
+| Vancouver     | F |
+| San Francisco | M |
+|               |   |
+| Vancouver     | M |
+|               | F |
++---------------+---+
+(5 rows)
 
 !ok
 
-select * from (select * from emp where gender ='F') as emp left join dept on emp.deptno = dept.deptno;
-+-------+--------+--------+---------+-------------+
-| ENAME | DEPTNO | GENDER | DEPTNO0 | DNAME       |
-+-------+--------+--------+---------+-------------+
-| Jane  |     10 | F      |      10 | Sales       |
-| Susan |     30 | F      |      30 | Engineering |
-| Alice |     30 | F      |      30 | Engineering |
-| Eve   |     50 | F      |         |             |
-| Grace |     60 | F      |         |             |
-| Wilma |        | F      |         |             |
-+-------+--------+--------+---------+-------------+
-(6 rows)
-
-!ok
+# SELECT DISTINCT includes fully and partially null rows
+select distinct city, gender from "emps";
++---------------+--------+
+| CITY          | GENDER |
++---------------+--------+
+|               |        |
+| Vancouver     | M      |
+|               | F      |
+| San Francisco | M      |
+| Vancouver     | F      |
++---------------+--------+
+(5 rows)
 
-# The following test is disabled, because we cannot handle non-equi-join.
-# Following it are the results from MySQL.
-!if (false) {
-select * from emp right join dept on emp.deptno = dept.deptno and emp.gender = 'F';
-+-------+--------+--------+--------+-------------+
-| ename | deptno | gender | deptno | dname       |
-+-------+--------+--------+--------+-------------+
-| Jane  |     10 | F      |     10 | Sales       |
-| Susan |     30 | F      |     30 | Engineering |
-| Alice |     30 | F      |     30 | Engineering |
-| NULL  |   NULL | NULL   |     20 | Marketing   |
-| NULL  |   NULL | NULL   |     40 | Empty       |
-+-------+--------+--------+--------+-------------+
 !ok
-!}
 
-select * from emp right join dept on emp.deptno = dept.deptno where emp.gender = 'F';
-+-------+--------+--------+---------+-------------+
-| ENAME | DEPTNO | GENDER | DEPTNO0 | DNAME       |
-+-------+--------+--------+---------+-------------+
-| Jane  |     10 | F      |      10 | Sales       |
-| Susan |     30 | F      |      30 | Engineering |
-| Alice |     30 | F      |      30 | Engineering |
-+-------+--------+--------+---------+-------------+
-(3 rows)
+# COUNT excludes fully or partially null rows
+select count(city, gender) as c from "emps";
++---+
+| C |
++---+
+| 3 |
++---+
+(1 row)
 
 !ok
 
-select * from (select * from emp where gender ='F') as emp right join dept on emp.deptno = dept.deptno;
-+-------+--------+--------+---------+-------------+
-| ENAME | DEPTNO | GENDER | DEPTNO0 | DNAME       |
-+-------+--------+--------+---------+-------------+
-| Jane  |     10 | F      |      10 | Sales       |
-| Susan |     30 | F      |      30 | Engineering |
-| Alice |     30 | F      |      30 | Engineering |
-|       |        |        |      20 | Marketing   |
-|       |        |        |      40 | Empty       |
-+-------+--------+--------+---------+-------------+
-(5 rows)
+# COUNT-DISTINCT excludes fully or partially null rows
+select count(distinct city, gender) as c from "emps";
++---+
+| C |
++---+
+| 3 |
++---+
+(1 row)
 
 !ok
 
-!if (false) {
-select * from emp full join dept on emp.deptno = dept.deptno and emp.gender = 'F';
-!ok
-!}
+select distinct mod(deptno, 20) as m, gender as c from "emps";
++----+---+
+| M  | C |
++----+---+
+|  0 | F |
+| 10 |   |
+|  0 | M |
++----+---+
+(3 rows)
 
+!ok
 
-select * from emp full join dept on emp.deptno = dept.deptno where emp.gender = 'F';
-+-------+--------+--------+---------+-------------+
-| ENAME | DEPTNO | GENDER | DEPTNO0 | DNAME       |
-+-------+--------+--------+---------+-------------+
-| Jane  |     10 | F      |      10 | Sales       |
-| Susan |     30 | F      |      30 | Engineering |
-| Alice |     30 | F      |      30 | Engineering |
-| Eve   |     50 | F      |         |             |
-| Grace |     60 | F      |         |             |
-| Wilma |        | F      |         |             |
-+-------+--------+--------+---------+-------------+
-(6 rows)
+# Partially null row (10, NULL) is excluded from count.
+select count(distinct mod(deptno, 20), gender) as c from "emps";
++---+
+| C |
++---+
+| 2 |
++---+
+(1 row)
 
 !ok
 
-select * from (select * from emp where gender ='F') as emp full join dept on emp.deptno = dept.deptno;
-+-------+--------+--------+---------+-------------+
-| ENAME | DEPTNO | GENDER | DEPTNO0 | DNAME       |
-+-------+--------+--------+---------+-------------+
-| Jane  |     10 | F      |      10 | Sales       |
-| Susan |     30 | F      |      30 | Engineering |
-| Alice |     30 | F      |      30 | Engineering |
-| Eve   |     50 | F      |         |             |
-| Grace |     60 | F      |         |             |
-| Wilma |        | F      |         |             |
-|       |        |        |      20 | Marketing   |
-|       |        |        |      40 | Empty       |
-+-------+--------+--------+---------+-------------+
-(8 rows)
+select count(mod(deptno, 20), gender) as c from "emps";
++---+
+| C |
++---+
+| 4 |
++---+
+(1 row)
 
 !ok
 
-# End outer.oq
+# End agg.oq