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 2020/11/01 17:46:15 UTC

[calcite] branch master updated (987cb7f -> d03ce47)

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 987cb7f  [CALCITE-4354] Allow ITEM operator on ROW/STRUCT data types (Alessandro Solimando)
     new 5f1d823  BigQuery dialect should allow GROUP BY ordinal
     new 1fec052  Add class SqlBasicAggFunction
     new a5801be  [CALCITE-4369] Support COUNTIF aggregate function for BigQuery (Aryeh Hillman)
     new d03ce47  [CALCITE-4352] RexSimplify incorrectly drops IS NULL and IS NOT NULL from SEARCH expressions

The 4 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:
 .../main/java/org/apache/calcite/rex/RexCall.java  |   9 ++
 .../java/org/apache/calcite/rex/RexSimplify.java   |  19 ++-
 .../main/java/org/apache/calcite/rex/RexUtil.java  |  84 ++++++++---
 .../org/apache/calcite/sql/SqlAggFunction.java     |   3 +
 .../main/java/org/apache/calcite/sql/SqlKind.java  |  16 +-
 .../main/java/org/apache/calcite/sql/SqlUtil.java  |  12 +-
 .../calcite/sql/fun/SqlBasicAggFunction.java       | 154 +++++++++++++++++++
 .../calcite/sql/fun/SqlLibraryOperators.java       |  68 ++++-----
 .../apache/calcite/sql/fun/SqlSearchOperator.java  |  26 +++-
 .../calcite/sql/fun/SqlStdOperatorTable.java       |  35 ++---
 .../calcite/sql/fun/SqlStringAggAggFunction.java   |  52 -------
 .../calcite/sql/validate/SqlConformance.java       |   1 +
 .../calcite/sql/validate/SqlConformanceEnum.java   |   1 +
 .../apache/calcite/sql2rel/SqlToRelConverter.java  |  15 +-
 .../java/org/apache/calcite/tools/RelBuilder.java  |   7 +-
 .../main/java/org/apache/calcite/util/Sarg.java    |  28 +++-
 .../calcite/materialize/LatticeSuggesterTest.java  |  20 +++
 .../org/apache/calcite/rex/RexProgramTest.java     | 168 ++++++++++++++-------
 .../calcite/sql/test/SqlOperatorBaseTest.java      |  27 ++++
 .../org/apache/calcite/test/RelBuilderTest.java    |   4 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml    |  10 +-
 core/src/test/resources/sql/agg.iq                 |  40 +++++
 core/src/test/resources/sql/conditions.iq          |  27 ++++
 core/src/test/resources/sql/sub-query.iq           |   2 +-
 site/_docs/reference.md                            |   1 +
 25 files changed, 616 insertions(+), 213 deletions(-)
 create mode 100644 core/src/main/java/org/apache/calcite/sql/fun/SqlBasicAggFunction.java
 delete mode 100644 core/src/main/java/org/apache/calcite/sql/fun/SqlStringAggAggFunction.java


[calcite] 01/04: BigQuery dialect should allow GROUP BY ordinal

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 5f1d8238da504d54a8d4d74252f344c4ebb77318
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Sat Oct 31 13:35:13 2020 -0700

    BigQuery dialect should allow GROUP BY ordinal
---
 .../apache/calcite/sql/validate/SqlConformance.java    |  1 +
 .../calcite/sql/validate/SqlConformanceEnum.java       |  1 +
 .../calcite/materialize/LatticeSuggesterTest.java      | 18 ++++++++++++++++++
 3 files changed, 20 insertions(+)

diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
index edb41fa..70bd4ce 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
@@ -106,6 +106,7 @@ public interface SqlConformance {
    *
    * <p>Among the built-in conformance levels, true in
    * {@link SqlConformanceEnum#BABEL},
+   * {@link SqlConformanceEnum#BIG_QUERY},
    * {@link SqlConformanceEnum#LENIENT},
    * {@link SqlConformanceEnum#MYSQL_5},
    * {@link SqlConformanceEnum#PRESTO};
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
index 30a41f1..d8b255b 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
@@ -117,6 +117,7 @@ public enum SqlConformanceEnum implements SqlConformance {
   @Override public boolean isGroupByOrdinal() {
     switch (this) {
     case BABEL:
+    case BIG_QUERY:
     case LENIENT:
     case MYSQL_5:
     case PRESTO:
diff --git a/core/src/test/java/org/apache/calcite/materialize/LatticeSuggesterTest.java b/core/src/test/java/org/apache/calcite/materialize/LatticeSuggesterTest.java
index 8e67d76..2fb2e4e 100644
--- a/core/src/test/java/org/apache/calcite/materialize/LatticeSuggesterTest.java
+++ b/core/src/test/java/org/apache/calcite/materialize/LatticeSuggesterTest.java
@@ -597,6 +597,8 @@ class LatticeSuggesterTest {
     assertThat(derivedColumns.get(1).tables, is(tables));
   }
 
+  /** Tests a number of features only available in Redshift: the {@code CONCAT}
+   * and {@code CONVERT_TIMEZONE} functions. */
   @Test void testRedshiftDialect() throws Exception {
     final Tester t = new Tester().foodmart().withEvolve(true)
         .withDialect(SqlDialect.DatabaseProduct.REDSHIFT.getDialect())
@@ -617,6 +619,22 @@ class LatticeSuggesterTest {
     assertThat(t.s.latticeMap.size(), is(1));
   }
 
+  /** Tests a number of features only available in BigQuery: back-ticks;
+   * GROUP BY ordinal; case-insensitive unquoted identifiers. */
+  @Test void testBigQueryDialect() throws Exception {
+    final Tester t = new Tester().foodmart().withEvolve(true)
+        .withDialect(SqlDialect.DatabaseProduct.BIG_QUERY.getDialect())
+        .withLibrary(SqlLibrary.BIG_QUERY);
+
+    final String q0 = "select `product_id`,\n"
+        + "  SUM(unit_sales)\n"
+        + "from\n"
+        + "  `sales_fact_1997`"
+        + "group by 1";
+    t.addQuery(q0);
+    assertThat(t.s.latticeMap.size(), is(1));
+  }
+
   /** A tricky case involving a CTE (WITH), a join condition that references an
    * expression, a complex WHERE clause, and some other queries. */
   @Test void testJoinUsingExpression() throws Exception {


[calcite] 04/04: [CALCITE-4352] RexSimplify incorrectly drops IS NULL and IS NOT NULL from SEARCH expressions

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 d03ce47536ddb2aaec6d445b3ea48464081a7ed0
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Sat Oct 24 17:23:16 2020 -0700

    [CALCITE-4352] RexSimplify incorrectly drops IS NULL and IS NOT NULL from SEARCH expressions
    
    Improve Sarg.toString() for IS NULL, IS NOT NULL, TRUE,
    FALSE.
    
    Simplify NOT SEARCH(x, sarg) to SEARCH(x, sarg2) where sarg2
    is the complement of sarg.
    
    Improve return type of SEARCH; result is NOT NULL if argument
    is NOT NULL of it Sarg allows IS NULL.
    
    Close apache/calcite#2233
---
 .../main/java/org/apache/calcite/rex/RexCall.java  |   9 ++
 .../java/org/apache/calcite/rex/RexSimplify.java   |  19 ++-
 .../main/java/org/apache/calcite/rex/RexUtil.java  |  84 ++++++++---
 .../main/java/org/apache/calcite/sql/SqlKind.java  |  11 ++
 .../apache/calcite/sql/fun/SqlSearchOperator.java  |  26 +++-
 .../java/org/apache/calcite/tools/RelBuilder.java  |   7 +-
 .../main/java/org/apache/calcite/util/Sarg.java    |  28 +++-
 .../org/apache/calcite/rex/RexProgramTest.java     | 168 ++++++++++++++-------
 .../org/apache/calcite/test/RelBuilderTest.java    |   4 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml    |  10 +-
 core/src/test/resources/sql/conditions.iq          |  27 ++++
 core/src/test/resources/sql/sub-query.iq           |   2 +-
 12 files changed, 302 insertions(+), 93 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rex/RexCall.java b/core/src/main/java/org/apache/calcite/rex/RexCall.java
index 5bea803..73795c0 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCall.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCall.java
@@ -25,6 +25,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Sarg;
 
 import com.google.common.collect.ImmutableList;
 
@@ -196,6 +197,10 @@ public class RexCall extends RexNode {
     case IS_TRUE:
     case CAST:
       return operands.get(0).isAlwaysTrue();
+    case SEARCH:
+      final Sarg sarg = ((RexLiteral) operands.get(1)).getValueAs(Sarg.class);
+      return sarg.isAll()
+          && (sarg.containsNull || !operands.get(0).getType().isNullable());
     default:
       return false;
     }
@@ -213,6 +218,10 @@ public class RexCall extends RexNode {
     case IS_TRUE:
     case CAST:
       return operands.get(0).isAlwaysFalse();
+    case SEARCH:
+      final Sarg sarg = ((RexLiteral) operands.get(1)).getValueAs(Sarg.class);
+      return sarg.isNone()
+          && (!sarg.containsNull || !operands.get(0).getType().isNullable());
     default:
       return false;
     }
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
index 9e2ab58..65b3ba3 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -583,6 +583,18 @@ public class RexSimplify {
     case NOT:
       // NOT NOT x ==> x
       return simplify(((RexCall) a).getOperands().get(0), unknownAs);
+    case SEARCH:
+      // NOT SEARCH(x, Sarg[(-inf, 10) OR NULL) ==> SEARCH(x, Sarg[[10, +inf)])
+      final RexCall call2 = (RexCall) a;
+      final RexNode ref = call2.operands.get(0);
+      final RexLiteral literal = (RexLiteral) call2.operands.get(1);
+      final Sarg sarg = literal.getValueAs(Sarg.class);
+      return simplifySearch(
+          call2.clone(call2.type,
+              ImmutableList.of(ref,
+                  rexBuilder.makeLiteral(sarg.negate(), literal.getType(),
+                      literal.getTypeName()))),
+          unknownAs.negate());
     case LITERAL:
       if (a.getType().getSqlTypeName() == SqlTypeName.BOOLEAN
           && !RexLiteral.isNullLiteral(a)) {
@@ -2651,7 +2663,7 @@ public class RexSimplify {
           map.computeIfAbsent(e, e2 ->
               addFluent(newTerms, new RexSargBuilder(e2, rexBuilder, negate)));
       if (negate) {
-        kind = kind.negateNullSafe();
+        kind = kind.negateNullSafe2();
       }
       final Comparable value = literal.getValueAs(Comparable.class);
       switch (kind) {
@@ -2687,6 +2699,7 @@ public class RexSimplify {
         } else {
           ++b.notNullTermCount;
         }
+        b.addAll();
         return true;
       case SEARCH:
         final Sarg sarg = literal.getValueAs(Sarg.class);
@@ -2790,6 +2803,10 @@ public class RexSimplify {
       throw new UnsupportedOperationException();
     }
 
+    void addAll() {
+      rangeSet.add(Range.all());
+    }
+
     void addRange(Range<Comparable> range, RelDataType type) {
       types.add(type);
       rangeSet.add(range);
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index bdda82b..91ee9f3 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -565,32 +565,36 @@ public class RexUtil {
     };
   }
 
+  /** Expands calls to {@link SqlStdOperatorTable#SEARCH} in an expression. */
   public static RexNode expandSearch(RexBuilder rexBuilder,
       @Nullable RexProgram program, RexNode node) {
-    final RexShuttle shuttle = new RexShuttle() {
-      @Override public RexNode visitCall(RexCall call) {
-        switch (call.getKind()) {
-        case SEARCH:
-          return visitSearch(rexBuilder, program, call);
-        default:
-          return super.visitCall(call);
-        }
-      }
-    };
-    return node.accept(shuttle);
+    return node.accept(searchShuttle(rexBuilder, program, -1));
+  }
+
+  /** Creates a shuttle that expands calls to
+   * {@link SqlStdOperatorTable#SEARCH}.
+   *
+   * <p>If {@code maxComplexity} is non-negative, a {@link Sarg} whose
+   * complexity is greater than {@code maxComplexity} is retained (not
+   * expanded); this gives a means to simplify simple expressions such as
+   * {@code x IS NULL} or {@code x > 10} while keeping more complex expressions
+   * such as {@code x IN (3, 5, 7) OR x IS NULL} as a Sarg. */
+  public static RexShuttle searchShuttle(RexBuilder rexBuilder,
+      RexProgram program, int maxComplexity) {
+    return new SearchExpandingShuttle(program, rexBuilder, maxComplexity);
   }
 
   @SuppressWarnings("BetaApi")
-  private static <C extends Comparable<C>> RexNode
-      visitSearch(RexBuilder rexBuilder,
-          @Nullable RexProgram program, RexCall call) {
-    final RexNode ref = call.operands.get(0);
-    final RexLiteral literal =
-        (RexLiteral) deref(program, call.operands.get(1));
-    @SuppressWarnings("unchecked")
-    final Sarg<C> sarg = literal.getValueAs(Sarg.class);
+  private static <C extends Comparable<C>> RexNode sargRef(
+      RexBuilder rexBuilder, RexNode ref, Sarg<C> sarg, RelDataType type) {
+    if (sarg.isAll()) {
+      if (sarg.containsNull) {
+        return rexBuilder.makeLiteral(true);
+      } else {
+        return rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, ref);
+      }
+    }
     final List<RexNode> orList = new ArrayList<>();
-
     if (sarg.containsNull) {
       orList.add(rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, ref));
     }
@@ -600,19 +604,19 @@ public class RexUtil {
           orList.add(
               rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, ref,
                   rexBuilder.makeLiteral(range.lowerEndpoint(),
-                      literal.getType(), true, true))));
+                      type, true, true))));
     } else if (sarg.isComplementedPoints()) {
       // Generate 'ref <> value1 AND ... AND ref <> valueN'
       final List<RexNode> list = sarg.rangeSet.complement().asRanges().stream()
           .map(range ->
               rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS, ref,
                   rexBuilder.makeLiteral(range.lowerEndpoint(),
-                      literal.getType(), true, true)))
+                      type, true, true)))
           .collect(Util.toImmutableList());
       orList.add(composeConjunction(rexBuilder, list));
     } else {
       final RangeSets.Consumer<C> consumer =
-          new RangeToRex<>(ref, orList, rexBuilder, literal.getType());
+          new RangeToRex<>(ref, orList, rexBuilder, type);
       RangeSets.forEach(sarg.rangeSet, consumer);
     }
     return composeDisjunction(rexBuilder, orList);
@@ -2987,4 +2991,38 @@ public class RexUtil {
           op(SqlStdOperatorTable.LESS_THAN, upper));
     }
   }
+
+  /** Shuttle that expands calls to
+   * {@link org.apache.calcite.sql.fun.SqlStdOperatorTable#SEARCH}.
+   *
+   * <p>Calls whose complexity is greater than {@link #maxComplexity}
+   * are retained (not expanded). */
+  private static class SearchExpandingShuttle extends RexShuttle {
+    private final RexBuilder rexBuilder;
+    private final RexProgram program;
+    private final int maxComplexity;
+
+    SearchExpandingShuttle(RexProgram program, RexBuilder rexBuilder,
+        int maxComplexity) {
+      this.program = program;
+      this.rexBuilder = rexBuilder;
+      this.maxComplexity = maxComplexity;
+    }
+
+    @Override public RexNode visitCall(RexCall call) {
+      switch (call.getKind()) {
+      case SEARCH:
+        final RexNode ref = call.operands.get(0);
+        final RexLiteral literal =
+            (RexLiteral) deref(program, call.operands.get(1));
+        final Sarg sarg = literal.getValueAs(Sarg.class);
+        if (maxComplexity < 0 || sarg.complexity() < maxComplexity) {
+          return sargRef(rexBuilder, ref, sarg, literal.getType());
+        }
+        // Sarg is complex (therefore useful); fall through
+      default:
+        return super.visitCall(call);
+      }
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index aa1332e..719bbf6 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -1385,6 +1385,17 @@ public enum SqlKind {
     }
   }
 
+  public SqlKind negateNullSafe2() {
+    switch (this) {
+    case IS_NOT_NULL:
+      return IS_NULL;
+    case IS_NULL:
+      return IS_NOT_NULL;
+    default:
+      return this.negateNullSafe();
+    }
+  }
+
   /**
    * Returns whether this {@code SqlKind} belongs to a given category.
    *
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSearchOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSearchOperator.java
index 5aa9f5f..6c5ab79 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSearchOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSearchOperator.java
@@ -16,18 +16,42 @@
  */
 package org.apache.calcite.sql.fun;
 
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlInternalOperator;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.type.InferTypes;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.util.Sarg;
 
 /** Operator that tests whether its left operand is included in the range of
  * values covered by search arguments. */
 class SqlSearchOperator extends SqlInternalOperator {
   SqlSearchOperator() {
-    super("SEARCH", SqlKind.SEARCH, 30, true, ReturnTypes.BOOLEAN_NULLABLE,
+    super("SEARCH", SqlKind.SEARCH, 30, true,
+        ReturnTypes.BOOLEAN.andThen(SqlSearchOperator::makeNullable),
         InferTypes.FIRST_KNOWN,
         OperandTypes.COMPARABLE_UNORDERED_COMPARABLE_UNORDERED);
   }
+
+  /** Sets whether a call to SEARCH should allow nulls.
+   *
+   * <p>For example, if the type of {@code x} is NOT NULL, then
+   * {@code SEARCH(x, Sarg[10])} will never return UNKNOWN.
+   * It is evident from the expansion, "x = 10", but holds for all Sarg
+   * values.
+   *
+   * <p>If {@link Sarg#containsNull} is true, SEARCH will never return
+   * UNKNOWN. For example, {@code SEARCH(x, Sarg[10 OR NULL])} expands to
+   * {@code x = 10 OR x IS NOT NULL}, which returns {@code TRUE} if
+   * {@code x} is NULL, {@code TRUE} if {@code x} is 10, and {@code FALSE}
+   * for all other values.
+   */
+  private static RelDataType makeNullable(SqlOperatorBinding binding,
+      RelDataType type) {
+    final boolean nullable = binding.getOperandType(0).isNullable()
+        && !binding.getOperandLiteralValue(1, Sarg.class).containsNull;
+    return binding.getTypeFactory().createTypeWithNullability(type, nullable);
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index 8209e76..11b77d8 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -1460,8 +1460,13 @@ public class RelBuilder {
 
     // Simplify expressions.
     if (config.simplify()) {
+      final RexShuttle shuttle =
+          RexUtil.searchShuttle(getRexBuilder(), null, 2);
       for (int i = 0; i < nodeList.size(); i++) {
-        nodeList.set(i, simplifier.simplifyPreservingType(nodeList.get(i)));
+        final RexNode node0 = nodeList.get(i);
+        final RexNode node1 = simplifier.simplifyPreservingType(node0);
+        final RexNode node2 = node1.accept(shuttle);
+        nodeList.set(i, node2);
       }
     }
 
diff --git a/core/src/main/java/org/apache/calcite/util/Sarg.java b/core/src/main/java/org/apache/calcite/util/Sarg.java
index b356b1e..ee80daa 100644
--- a/core/src/main/java/org/apache/calcite/util/Sarg.java
+++ b/core/src/main/java/org/apache/calcite/util/Sarg.java
@@ -90,7 +90,7 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
    *
    * and also null is printed as
    *
-   * <blockquote>{@code Sarg[7, 9, (10..+∞), null]}</blockquote>
+   * <blockquote>{@code Sarg[7, 9, (10..+∞) OR NULL]}</blockquote>
    */
   @Override public String toString() {
     final StringBuilder sb = new StringBuilder();
@@ -102,6 +102,12 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
    * with each embedded value. */
   public StringBuilder printTo(StringBuilder sb,
       BiConsumer<StringBuilder, C> valuePrinter) {
+    if (isAll()) {
+      return sb.append(containsNull ? "Sarg[TRUE]" : "Sarg[NOT NULL]");
+    }
+    if (isNone()) {
+      return sb.append(containsNull ? "Sarg[NULL]" : "Sarg[FALSE]");
+    }
     sb.append("Sarg[");
     final RangeSets.Consumer<C> printer = RangeSets.printer(sb, valuePrinter);
     Ord.forEach(rangeSet.asRanges(), (r, i) -> {
@@ -111,7 +117,7 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
       RangeSets.forEach(r, printer);
     });
     if (containsNull) {
-      sb.append(", null");
+      sb.append(" OR NULL");
     }
     return sb.append("]");
   }
@@ -131,6 +137,18 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
         && containsNull == ((Sarg) o).containsNull;
   }
 
+  /** Returns whether this Sarg includes all values (including or not including
+   * null). */
+  public boolean isAll() {
+    return rangeSet.equals(RangeSets.rangeSetAll());
+  }
+
+  /** Returns whether this Sarg includes no values (including or not including
+   * null). */
+  public boolean isNone() {
+    return rangeSet.isEmpty();
+  }
+
   /** Returns whether this Sarg is a collection of 1 or more points (and perhaps
    * an {@code IS NULL} if {@link #containsNull}).
    *
@@ -147,6 +165,7 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
    * or {@code ref NOT IN (value1, ...)}. */
   public boolean isComplementedPoints() {
     return rangeSet.span().encloses(Range.all())
+        && !rangeSet.equals(RangeSets.rangeSetAll())
         && rangeSet.complement().asRanges().stream()
             .allMatch(RangeSets::isPoint);
   }
@@ -182,4 +201,9 @@ public class Sarg<C extends Comparable<C>> implements Comparable<Sarg<C>> {
     }
     return complexity;
   }
+
+  /** Returns a Sarg that matches a value if and only this Sarg does not. */
+  public Sarg negate() {
+    return Sarg.of(!containsNull, rangeSet.complement());
+  }
 }
diff --git a/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java b/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
index e72550a..ba2f91b 100644
--- a/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/rex/RexProgramTest.java
@@ -52,6 +52,7 @@ import com.google.common.collect.Multimap;
 import com.google.common.collect.Range;
 import com.google.common.collect.RangeSet;
 
+import org.hamcrest.Matcher;
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
@@ -983,19 +984,22 @@ class RexProgramTest extends RexProgramTestBase {
     // "x <> x" simplifies to "false"
     checkSimplify(ne(literal(1), literal(1)), "false");
     checkSimplify(ne(hRef, hRef), "false");
-    checkSimplify3(ne(iRef, iRef), "AND(null, IS NULL(?0.i))", "false", "IS NULL(?0.i)");
+    checkSimplify3(ne(iRef, iRef), "AND(null, SEARCH(?0.i, Sarg[NULL]))",
+        "false", "SEARCH(?0.i, Sarg[NULL])");
     checkSimplifyUnchanged(ne(iRef, hRef));
 
     // "x < x" simplifies to "false"
     checkSimplify(lt(literal(1), literal(1)), "false");
     checkSimplify(lt(hRef, hRef), "false");
-    checkSimplify3(lt(iRef, iRef), "AND(null, IS NULL(?0.i))", "false", "IS NULL(?0.i)");
+    checkSimplify3(lt(iRef, iRef), "AND(null, SEARCH(?0.i, Sarg[NULL]))",
+        "false", "SEARCH(?0.i, Sarg[NULL])");
     checkSimplifyUnchanged(lt(iRef, hRef));
 
     // "x > x" simplifies to "false"
     checkSimplify(gt(literal(1), literal(1)), "false");
     checkSimplify(gt(hRef, hRef), "false");
-    checkSimplify3(gt(iRef, iRef), "AND(null, IS NULL(?0.i))", "false", "IS NULL(?0.i)");
+    checkSimplify3(gt(iRef, iRef), "AND(null, SEARCH(?0.i, Sarg[NULL]))",
+        "false", "SEARCH(?0.i, Sarg[NULL])");
     checkSimplifyUnchanged(gt(iRef, hRef));
 
     // "(not x) is null" to "x is null"
@@ -1555,7 +1559,7 @@ class RexProgramTest extends RexProgramTestBase {
             eq(aRef, literal(3)),
             or(eq(aRef, literal(3)),
                 eq(aRef, literal(4)))),
-        "AND(SEARCH(?0.b, Sarg[2]), SEARCH(?0.a, Sarg[3]))");
+        "AND(=(?0.b, 2), =(?0.a, 3))");
 
     checkSimplify3(
         or(lt(vInt(), nullInt),
@@ -1588,7 +1592,7 @@ class RexProgramTest extends RexProgramTestBase {
     // a is null or a >= 15
     RexNode expr = or(isNull(aRef),
         ge(aRef, literal(15)));
-    checkSimplify(expr, "SEARCH($0, Sarg[[15..+\u221e), null])")
+    checkSimplify(expr, "SEARCH($0, Sarg[[15..+\u221e) OR NULL])")
         .expandedSearch("OR(IS NULL($0), >=($0, 15))");
   }
 
@@ -1607,7 +1611,7 @@ class RexProgramTest extends RexProgramTestBase {
         ge(aRef, literal(15)));
     // [CALCITE-4190] causes "or a >= 15" to disappear from the simplified form.
     final String simplified =
-        "SEARCH($0, Sarg[(0..12), [15..+\u221e), null])";
+        "SEARCH($0, Sarg[(0..12), [15..+\u221e) OR NULL])";
     final String expanded =
         "OR(IS NULL($0), AND(>($0, 0), <($0, 12)), >=($0, 15))";
     checkSimplify(expr, simplified)
@@ -1636,7 +1640,7 @@ class RexProgramTest extends RexProgramTestBase {
                 eq(aRef, literal(5)))),
         isNull(aRef));
     final String simplified =
-        "SEARCH($0, Sarg[(-\u221e..3), (3..5), (5..+\u221e), null])";
+        "SEARCH($0, Sarg[(-\u221e..3), (3..5), (5..+\u221e) OR NULL])";
     final String expanded = "OR(IS NULL($0), AND(<>($0, 3), <>($0, 5)))";
     checkSimplify(expr, simplified)
         .expandedSearch(expanded);
@@ -1651,7 +1655,7 @@ class RexProgramTest extends RexProgramTestBase {
     // a in (1, 2) or b is null
     RexNode expr = or(eq(aRef, literal(1)), eq(aRef, literal(2)), isNull(bRef));
     final String simplified =
-        "OR(SEARCH($1, Sarg[, null]), SEARCH($0, Sarg[1, 2]))";
+        "OR(SEARCH($1, Sarg[NULL]), SEARCH($0, Sarg[1, 2]))";
     final String expanded = "OR(IS NULL($1), OR(=($0, 1), =($0, 2)))";
     checkSimplify(expr, simplified)
         .expandedSearch(expanded);
@@ -1670,6 +1674,44 @@ class RexProgramTest extends RexProgramTestBase {
         .expandedSearch(expanded);
   }
 
+  /** Unit test for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-4352">[CALCITE-4352]
+   * OR simplification incorrectly loses term</a>. */
+  @Test void testSimplifyAndIsNotNull() {
+    final RexNode aRef = input(tInt(true), 0);
+    final RexNode bRef = input(tInt(true), 1);
+    // (0 < a and a < 10) and b is not null
+    RexNode expr = and(
+        and(lt(literal(0), aRef),
+            lt(aRef, literal(10))),
+        isNotNull(bRef));
+    // [CALCITE-4352] causes "and b is not null" to disappear from the expanded
+    // form.
+    final String simplified = "AND(SEARCH($0, Sarg[(0..10)]),"
+        + " SEARCH($1, Sarg[NOT NULL]))";
+    final String expanded =
+        "AND(AND(>($0, 0), <($0, 10)), IS NOT NULL($1))";
+    checkSimplify(expr, simplified)
+        .expandedSearch(expanded);
+  }
+
+  @Test void testSimplifyAndIsNull() {
+    final RexNode aRef = input(tInt(true), 0);
+    final RexNode bRef = input(tInt(true), 1);
+    // (0 < a and a < 10) and b is null
+    RexNode expr = and(
+        and(lt(literal(0), aRef),
+            lt(aRef, literal(10))),
+        isNull(bRef));
+    // [CALCITE-4352] causes "and b is null" to disappear from the expanded
+    // form.
+    final String simplified =
+        "AND(SEARCH($0, Sarg[(0..10)]), SEARCH($1, Sarg[NULL]))";
+    final String expanded = "AND(AND(>($0, 0), <($0, 10)), IS NULL($1))";
+    checkSimplify(expr, simplified)
+        .expandedSearch(expanded);
+  }
+
   @Test void testSimplifyItemRangeTerms() {
     RexNode item = item(input(tArray(tInt()), 3), literal(1));
     // paranoid validation doesn't support array types, disable it for a moment
@@ -1717,7 +1759,7 @@ class RexProgramTest extends RexProgramTestBase {
             or(eq(vInt(), literal(20)),
                 isNull(vInt())),
         eq(vInt(), literal(10)));
-    checkSimplify2(e, "SEARCH(?0.int0, Sarg[])", "false");
+    checkSimplify(e, "false");
   }
 
   @Test void testSimplifyEqOrIsNullAndEqSame() {
@@ -2364,9 +2406,9 @@ class RexProgramTest extends RexProgramTestBase {
         "AND(IS NOT NULL(?0.int0), IS NOT NULL(?0.int1))",
         "true");
     checkSimplify3(and(ne(vInt(), vInt()), ne(vInt(1), vInt(1))),
-        "AND(null, IS NULL(?0.int0), IS NULL(?0.int1))",
+        "AND(null, SEARCH(?0.int0, Sarg[NULL]), SEARCH(?0.int1, Sarg[NULL]))",
         "false",
-        "AND(IS NULL(?0.int0), IS NULL(?0.int1))");
+        "AND(SEARCH(?0.int0, Sarg[NULL]), SEARCH(?0.int1, Sarg[NULL]))");
   }
 
   @Test void testBooleanComparisons() {
@@ -2658,15 +2700,17 @@ class RexProgramTest extends RexProgramTestBase {
     //    -> "x = x AND y < y" (treating unknown as unknown)
     //    -> false (treating unknown as false)
     checkSimplify3(and(eq(vInt(1), vInt(1)), not(ge(vInt(2), vInt(2)))),
-        "AND(OR(null, IS NOT NULL(?0.int1)), null, IS NULL(?0.int2))",
+        "AND(OR(null, IS NOT NULL(?0.int1)), null,"
+            + " SEARCH(?0.int2, Sarg[NULL]))",
         "false",
-        "IS NULL(?0.int2)");
+        "SEARCH(?0.int2, Sarg[NULL])");
 
     // "NOT(x = x AND NOT (y = y))"
     //   -> "OR(x <> x, y >= y)" (treating unknown as unknown)
     //   -> "y IS NOT NULL" (treating unknown as false)
     checkSimplify3(not(and(eq(vInt(1), vInt(1)), not(ge(vInt(2), vInt(2))))),
-        "OR(AND(null, IS NULL(?0.int1)), null, IS NOT NULL(?0.int2))",
+        "OR(AND(null, SEARCH(?0.int1, Sarg[NULL])), null,"
+            + " IS NOT NULL(?0.int2))",
         "IS NOT NULL(?0.int2)",
         "true");
   }
@@ -2674,10 +2718,10 @@ class RexProgramTest extends RexProgramTestBase {
   @Test void testSimplifyOrIsNull() {
     // x = 10 OR x IS NULL
     checkSimplify(or(eq(vInt(0), literal(10)), isNull(vInt(0))),
-        "SEARCH(?0.int0, Sarg[10, null])");
+        "SEARCH(?0.int0, Sarg[10 OR NULL])");
     // 10 = x OR x IS NULL
     checkSimplify(or(eq(literal(10), vInt(0)), isNull(vInt(0))),
-        "SEARCH(?0.int0, Sarg[10, null])");
+        "SEARCH(?0.int0, Sarg[10 OR NULL])");
   }
 
   @Test void testSimplifyOrNot() {
@@ -2689,7 +2733,8 @@ class RexProgramTest extends RexProgramTestBase {
     //    -> "x = x OR y < y" (treating unknown as unknown)
     //    -> "x IS NOT NULL" (treating unknown as false)
     checkSimplify3(or(eq(vInt(1), vInt(1)), not(ge(vInt(2), vInt(2)))),
-        "OR(null, IS NOT NULL(?0.int1), AND(null, IS NULL(?0.int2)))",
+        "OR(null, IS NOT NULL(?0.int1),"
+            + " AND(null, SEARCH(?0.int2, Sarg[NULL])))",
         "IS NOT NULL(?0.int1)",
         "true");
 
@@ -2697,60 +2742,69 @@ class RexProgramTest extends RexProgramTestBase {
     //   -> "AND(x <> x, y >= y)" (treating unknown as unknown)
     //   -> "FALSE" (treating unknown as false)
     checkSimplify3(not(or(eq(vInt(1), vInt(1)), not(ge(vInt(2), vInt(2))))),
-        "AND(null, IS NULL(?0.int1), OR(null, IS NOT NULL(?0.int2)))",
+        "AND(null, SEARCH(?0.int1, Sarg[NULL]),"
+            + " OR(null, IS NOT NULL(?0.int2)))",
         "false",
-        "IS NULL(?0.int1)");
+        "SEARCH(?0.int1, Sarg[NULL])");
+  }
+
+  private void checkSarg(String message, Sarg sarg,
+      Matcher<Integer> complexityMatcher, Matcher<String> stringMatcher) {
+    assertThat(message, sarg.complexity(), complexityMatcher);
+    assertThat(message, sarg.toString(), stringMatcher);
   }
 
   /** Tests {@link Sarg#complexity()}. */
   @SuppressWarnings("UnstableApiUsage")
   @Test void testSargComplexity() {
-    assertThat("complexity of 'x is not null'",
-        Sarg.of(false, RangeSets.<Integer>rangeSetAll()).complexity(),
-        is(1));
-    assertThat("complexity of 'x is null'",
-        Sarg.of(true, ImmutableRangeSet.<Integer>of()).complexity(),
-        is(1));
-
-    assertThat("complexity of 'x = 1'",
-        Sarg.of(false, ImmutableRangeSet.of(Range.singleton(1))).complexity(),
-        is(1));
-    assertThat("complexity of 'x > 1'",
-        Sarg.of(false, ImmutableRangeSet.of(Range.greaterThan(1)))
-            .complexity(),
-        is(1));
-    assertThat("complexity of 'x >= 1'",
-        Sarg.of(false, ImmutableRangeSet.of(Range.atLeast(1))).complexity(),
-        is(1));
-    assertThat("complexity of 'x > 1 or x is null'",
-        Sarg.of(true, ImmutableRangeSet.of(Range.greaterThan(1))).complexity(),
-        is(2));
-    assertThat("complexity of 'x <> 1'",
-        Sarg.of(false, ImmutableRangeSet.of(Range.singleton(1)).complement())
-            .complexity(),
-        is(1));
-    assertThat("complexity of 'x <> 1 or x is null'",
-        Sarg.of(true, ImmutableRangeSet.of(Range.singleton(1)).complement())
-            .complexity(),
-        is(2));
-    assertThat("complexity of 'x < 10 or x >= 20'",
+    checkSarg("complexity of 'x is not null'",
+        Sarg.of(false, RangeSets.<Integer>rangeSetAll()),
+        is(1), is("Sarg[NOT NULL]"));
+    checkSarg("complexity of 'x is null'",
+        Sarg.of(true, ImmutableRangeSet.<Integer>of()),
+        is(1), is("Sarg[NULL]"));
+    checkSarg("complexity of 'false'",
+        Sarg.of(false, ImmutableRangeSet.<Integer>of()),
+        is(0), is("Sarg[FALSE]"));
+    checkSarg("complexity of 'true'",
+        Sarg.of(true, RangeSets.<Integer>rangeSetAll()),
+        is(2), is("Sarg[TRUE]"));
+
+    checkSarg("complexity of 'x = 1'",
+        Sarg.of(false, ImmutableRangeSet.of(Range.singleton(1))),
+        is(1), is("Sarg[1]"));
+    checkSarg("complexity of 'x > 1'",
+        Sarg.of(false, ImmutableRangeSet.of(Range.greaterThan(1))),
+        is(1), is("Sarg[(1..+\u221E)]"));
+    checkSarg("complexity of 'x >= 1'",
+        Sarg.of(false, ImmutableRangeSet.of(Range.atLeast(1))),
+        is(1), is("Sarg[[1..+\u221E)]"));
+    checkSarg("complexity of 'x > 1 or x is null'",
+        Sarg.of(true, ImmutableRangeSet.of(Range.greaterThan(1))),
+        is(2), is("Sarg[(1..+\u221E) OR NULL]"));
+    checkSarg("complexity of 'x <> 1'",
+        Sarg.of(false, ImmutableRangeSet.of(Range.singleton(1)).complement()),
+        is(1), is("Sarg[(-\u221E..1), (1..+\u221E)]"));
+    checkSarg("complexity of 'x <> 1 or x is null'",
+        Sarg.of(true, ImmutableRangeSet.of(Range.singleton(1)).complement()),
+        is(2), is("Sarg[(-\u221E..1), (1..+\u221E) OR NULL]"));
+    checkSarg("complexity of 'x < 10 or x >= 20'",
         Sarg.of(false,
             ImmutableRangeSet.copyOf(
-                ImmutableList.of(Range.lessThan(10), Range.atLeast(20))))
-            .complexity(),
-        is(2));
-    assertThat("complexity of 'x in (2, 4, 6) or x > 20'",
+                ImmutableList.of(Range.lessThan(10), Range.atLeast(20)))),
+        is(2), is("Sarg[(-\u221E..10), [20..+\u221E)]"));
+    checkSarg("complexity of 'x in (2, 4, 6) or x > 20'",
         Sarg.of(false,
             ImmutableRangeSet.copyOf(
                 Arrays.asList(Range.singleton(2), Range.singleton(4),
-                    Range.singleton(6), Range.greaterThan(20)))).complexity(),
-        is(4));
-    assertThat("complexity of 'x between 3 and 8 or x between 10 and 20'",
+                    Range.singleton(6), Range.greaterThan(20)))),
+        is(4), is("Sarg[2, 4, 6, (20..+\u221E)]"));
+    checkSarg("complexity of 'x between 3 and 8 or x between 10 and 20'",
         Sarg.of(false,
             ImmutableRangeSet.copyOf(
                 Arrays.asList(Range.closed(3, 8),
-                    Range.closed(10, 20)))).complexity(),
-        is(2));
+                    Range.closed(10, 20)))),
+        is(2), is("Sarg[[3..8], [10..20]]"));
   }
 
   @Test void testInterpreter() {
diff --git a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
index 024f1f1..be5d678 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -681,7 +681,7 @@ public class RelBuilderTest {
                         builder.equals(builder.field("DEPTNO"),
                             builder.literal(10)),
                         builder.and(builder.isNull(builder.field(6)),
-                            builder.not(builder.isNotNull(builder.field(7))))),
+                            builder.not(builder.isNotNull(builder.field(5))))),
                     builder.equals(builder.field("DEPTNO"),
                         builder.literal(20)),
                     builder.equals(builder.field("DEPTNO"),
@@ -695,7 +695,7 @@ public class RelBuilderTest {
     final String expected = ""
         + "LogicalProject(DEPTNO=[$7], COMM=[CAST($6):SMALLINT NOT NULL], "
         + "$f2=[OR(SEARCH($7, Sarg[20, 30]), AND(null:NULL, =($7, 10), "
-        + "IS NULL($6), IS NULL($7)))], n2=[IS NULL($2)], "
+        + "IS NULL($6), IS NULL($5)))], n2=[IS NULL($2)], "
         + "nn2=[IS NOT NULL($3)], $f5=[20], COMM0=[$6], C=[$6])\n"
         + "  LogicalTableScan(table=[[scott, EMP]])\n";
     assertThat(root, hasTree(expected));
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index b68f023..3311dd9 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -9920,9 +9920,9 @@ LogicalAggregate(group=[{7}], EXPR$1=[COUNT(DISTINCT $10, $12)])
     <TestCase name="testAggregateJoinRemove11">
         <Resource name="sql">
             <![CDATA[select e.deptno, count(distinct e.slacker)
-        from sales.emp e
-        left outer join sales.dept d on e.deptno = d.deptno
-        group by e.deptno]]>
+from sales.emp e
+left outer join sales.dept d on e.deptno = d.deptno
+group by e.deptno]]>
         </Resource>
         <Resource name="planBefore">
             <![CDATA[
@@ -11180,7 +11180,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[10], ENAME=[$1], JOB=[$2], MGR=[null:INTEGER], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[7], SLACKER=[$8])
-  LogicalFilter(condition=[AND(=($7, 7), =($0, 10), IS NULL($3))])
+  LogicalFilter(condition=[AND(SEARCH($7, Sarg[7]), SEARCH($0, Sarg[10]), SEARCH($3, Sarg[NULL]))])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -11714,7 +11714,7 @@ LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null:INTEGER)])
             <![CDATA[
 LogicalProject(EMPNO=[$0])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    LogicalFilter(condition=[<($0, CASE(OR(AND(IS NOT NULL($12), <>($9, 0)), AND(<($10, $9), null, <>($9, 0), IS NULL($12))), 10, AND(OR(IS NULL($12), =($9, 0)), OR(>=($10, $9), =($9, 0), IS NOT NULL($12))), 20, 30))])
+    LogicalFilter(condition=[<($0, CASE(OR(AND(IS NOT NULL($12), <>($9, 0)), AND(<($10, $9), null, SEARCH($9, Sarg[(-∞..0), (0..+∞)]), SEARCH($12, Sarg[NULL]))), 10, AND(OR(IS NULL($12), =($9, 0)), OR(>=($10, $9), =($9, 0), SEARCH($12, Sarg[NOT NULL]))), 20, 30))])
       LogicalJoin(condition=[=($7, $11)], joinType=[left])
         LogicalJoin(condition=[true], joinType=[inner])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
diff --git a/core/src/test/resources/sql/conditions.iq b/core/src/test/resources/sql/conditions.iq
index 8fb40f9..e35b5f3 100644
--- a/core/src/test/resources/sql/conditions.iq
+++ b/core/src/test/resources/sql/conditions.iq
@@ -299,5 +299,32 @@ select s, (s or s is null), (s and s is not null) from ax;
 
 !ok
 
+!use scott
+
+# Test case for [CALCITE-4352] "IS NOT NULL" condition gets dropped
+select *
+from "scott".emp
+where deptno > 5 AND deptno < 20 AND mgr IS NOT NULL;
++-------+--------+---------+------+------------+---------+------+--------+
+| EMPNO | ENAME  | JOB     | MGR  | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+--------+---------+------+------------+---------+------+--------+
+|  7782 | CLARK  | MANAGER | 7839 | 1981-06-09 | 2450.00 |      |     10 |
+|  7934 | MILLER | CLERK   | 7782 | 1982-01-23 | 1300.00 |      |     10 |
++-------+--------+---------+------+------------+---------+------+--------+
+(2 rows)
+
+!ok
+
+select *
+from "scott".emp
+where deptno > 5 AND deptno < 20 AND mgr IS NULL;
++-------+-------+-----------+-----+------------+---------+------+--------+
+| EMPNO | ENAME | JOB       | MGR | HIREDATE   | SAL     | COMM | DEPTNO |
++-------+-------+-----------+-----+------------+---------+------+--------+
+|  7839 | KING  | PRESIDENT |     | 1981-11-17 | 5000.00 |      |     10 |
++-------+-------+-----------+-----+------------+---------+------+--------+
+(1 row)
+
+!ok
 
 # End conditions.iq
diff --git a/core/src/test/resources/sql/sub-query.iq b/core/src/test/resources/sql/sub-query.iq
index a7fc8a3..e5cd646 100644
--- a/core/src/test/resources/sql/sub-query.iq
+++ b/core/src/test/resources/sql/sub-query.iq
@@ -32,7 +32,7 @@ where t1.x not in (select t2.x from t2);
 (0 rows)
 
 !ok
-EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t1, $t5)], expr#7=[IS NULL($t4)], expr#8=[>=($t2, $t1)], expr#9=[IS NOT NULL($t0)], expr#10=[AND($t7, $t8, $t9)], expr#11=[OR($t6, $t10)], X=[$t0], $condition=[$t11])
+EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[=($t1, $t5)], expr#7=[Sarg[NOT NULL]], expr#8=[SEARCH($t0, $t7)], expr#9=[Sarg[NULL]], expr#10=[SEARCH($t4, $t9)], expr#11=[>=($t2, $t1)], expr#12=[AND($t8, $t10, $t11)], expr#13=[OR($t6, $t12)], X=[$t0], $condition=[$t13])
   EnumerableHashJoin(condition=[=($0, $3)], joinType=[left])
     EnumerableNestedLoopJoin(condition=[true], joinType=[inner])
       EnumerableUnion(all=[true])


[calcite] 03/04: [CALCITE-4369] Support COUNTIF aggregate function for BigQuery (Aryeh Hillman)

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 a5801bed0d0d74ad12b6742d64ffb07b4a05f674
Author: Aryeh Hillman <ar...@google.com>
AuthorDate: Mon Oct 12 17:29:57 2020 -0700

    [CALCITE-4369] Support COUNTIF aggregate function for BigQuery (Aryeh Hillman)
    
    In SQL reference, move COUNTIF to the list of dialect-specific
    functions; during SQL-to-Rel, transform to 'COUNT(*) FILTER
    (WHERE b)' rather than 'COUNT(CASE WHEN b THEN 1 END)' (Julian Hyde).
    
    Close apache/calcite#2235
---
 .../main/java/org/apache/calcite/sql/SqlKind.java  |  5 ++-
 .../main/java/org/apache/calcite/sql/SqlUtil.java  | 12 +++++--
 .../calcite/sql/fun/SqlLibraryOperators.java       | 12 +++++++
 .../apache/calcite/sql2rel/SqlToRelConverter.java  | 15 +++++---
 .../calcite/materialize/LatticeSuggesterTest.java  |  4 ++-
 .../calcite/sql/test/SqlOperatorBaseTest.java      | 27 +++++++++++++++
 core/src/test/resources/sql/agg.iq                 | 40 ++++++++++++++++++++++
 site/_docs/reference.md                            |  1 +
 8 files changed, 107 insertions(+), 9 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index e76727b..aa1332e 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -805,6 +805,9 @@ public enum SqlKind {
   /** The {@code STRING_AGG} aggregate function. */
   STRING_AGG,
 
+  /** The {@code COUNTIF} aggregate function. */
+  COUNTIF,
+
   /** The {@code ARRAY_AGG} aggregate function. */
   ARRAY_AGG,
 
@@ -1040,7 +1043,7 @@ public enum SqlKind {
           AVG, STDDEV_POP, STDDEV_SAMP, VAR_POP, VAR_SAMP, NTILE, COLLECT,
           FUSION, SINGLE_VALUE, ROW_NUMBER, RANK, PERCENT_RANK, DENSE_RANK,
           CUME_DIST, JSON_ARRAYAGG, JSON_OBJECTAGG, BIT_AND, BIT_OR, BIT_XOR,
-          LISTAGG, STRING_AGG, ARRAY_AGG, ARRAY_CONCAT_AGG,
+          LISTAGG, STRING_AGG, ARRAY_AGG, ARRAY_CONCAT_AGG, COUNTIF,
           INTERSECTION, ANY_VALUE);
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
index 1176fbd..9f9175d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -66,6 +66,8 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
@@ -75,9 +77,13 @@ import static org.apache.calcite.util.Static.RESOURCE;
 public abstract class SqlUtil {
   //~ Methods ----------------------------------------------------------------
 
-  static SqlNode andExpressions(
-      SqlNode node1,
-      SqlNode node2) {
+  /** Returns the AND of two expressions.
+   *
+   * <p>If {@code node1} is null, returns {@code node2}.
+   * Flattens if either node is an AND. */
+  public static @Nonnull SqlNode andExpressions(
+      @Nullable SqlNode node1,
+      @Nonnull SqlNode node2) {
     if (node1 == null) {
       return node2;
     }
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
index 9cb9e0f..8ff1b46 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
@@ -35,6 +35,7 @@ import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeTransforms;
 import org.apache.calcite.util.Litmus;
+import org.apache.calcite.util.Optionality;
 
 import com.google.common.collect.ImmutableList;
 
@@ -269,6 +270,17 @@ public abstract class SqlLibraryOperators {
   public static final SqlAggFunction LOGICAL_OR =
       new SqlMinMaxAggFunction("LOGICAL_OR", SqlKind.MAX, OperandTypes.BOOLEAN);
 
+  /** The "COUNTIF(condition) [OVER (...)]" function, in BigQuery,
+   * returns the count of TRUE values for expression.
+   *
+   * <p>{@code COUNTIF(b)} is equivalent to
+   * {@code COUNT(*) FILTER (WHERE b)}. */
+  @LibraryOperator(libraries = {BIG_QUERY})
+  public static final SqlAggFunction COUNTIF =
+      SqlBasicAggFunction
+          .create(SqlKind.COUNTIF, ReturnTypes.BIGINT, OperandTypes.BOOLEAN)
+          .withDistinct(Optionality.FORBIDDEN);
+
   /** The "ARRAY_AGG(value [ ORDER BY ...])" aggregate function,
    * in BigQuery and PostgreSQL, gathers values into arrays. */
   @LibraryOperator(libraries = {POSTGRESQL, BIG_QUERY})
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index fab36f2..39ca7dd 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -5437,6 +5437,7 @@ public class SqlToRelConverter {
       assert bb.agg == this;
       assert outerCall != null;
       final List<SqlNode> operands = call.getOperandList();
+      final SqlParserPos pos = call.getParserPosition();
       switch (call.getKind()) {
       case FILTER:
         assert filter == null;
@@ -5455,6 +5456,14 @@ public class SqlToRelConverter {
         translateAgg(call.operand(0), filter, orderList, ignoreNulls,
             outerCall);
         return;
+      case COUNTIF:
+        // COUNTIF(b)  ==> COUNT(*) FILTER (WHERE b)
+        // COUNTIF(b) FILTER (WHERE b2)  ==> COUNT(*) FILTER (WHERE b2 AND b)
+        final SqlCall call4 =
+            SqlStdOperatorTable.COUNT.createCall(pos, SqlIdentifier.star(pos));
+        final SqlNode filter2 = SqlUtil.andExpressions(filter, call.operand(0));
+        translateAgg(call4, filter2, orderList, ignoreNulls, outerCall);
+        return;
       case STRING_AGG:
         // Translate "STRING_AGG(s, sep ORDER BY x, y)"
         // as if it were "LISTAGG(s, sep) WITHIN GROUP (ORDER BY x, y)";
@@ -5469,8 +5478,7 @@ public class SqlToRelConverter {
         }
         final SqlCall call2 =
             SqlStdOperatorTable.LISTAGG.createCall(
-                call.getFunctionQuantifier(), call.getParserPosition(),
-                operands2);
+                call.getFunctionQuantifier(), pos, operands2);
         translateAgg(call2, filter, orderList, ignoreNulls, outerCall);
         return;
       case ARRAY_AGG:
@@ -5483,8 +5491,7 @@ public class SqlToRelConverter {
           orderList = (SqlNodeList) Util.last(operands);
           final SqlCall call3 =
               call.getOperator().createCall(
-                  call.getFunctionQuantifier(), call.getParserPosition(),
-                  Util.skipLast(operands));
+                  call.getFunctionQuantifier(), pos, Util.skipLast(operands));
           translateAgg(call3, filter, orderList, ignoreNulls, outerCall);
           return;
         }
diff --git a/core/src/test/java/org/apache/calcite/materialize/LatticeSuggesterTest.java b/core/src/test/java/org/apache/calcite/materialize/LatticeSuggesterTest.java
index 2fb2e4e..5ff1e95 100644
--- a/core/src/test/java/org/apache/calcite/materialize/LatticeSuggesterTest.java
+++ b/core/src/test/java/org/apache/calcite/materialize/LatticeSuggesterTest.java
@@ -620,13 +620,15 @@ class LatticeSuggesterTest {
   }
 
   /** Tests a number of features only available in BigQuery: back-ticks;
-   * GROUP BY ordinal; case-insensitive unquoted identifiers. */
+   * GROUP BY ordinal; case-insensitive unquoted identifiers;
+   * the {@code COUNTIF} aggregate function. */
   @Test void testBigQueryDialect() throws Exception {
     final Tester t = new Tester().foodmart().withEvolve(true)
         .withDialect(SqlDialect.DatabaseProduct.BIG_QUERY.getDialect())
         .withLibrary(SqlLibrary.BIG_QUERY);
 
     final String q0 = "select `product_id`,\n"
+        + "  countif(unit_sales > 1000) as num_over_thousand,\n"
         + "  SUM(unit_sales)\n"
         + "from\n"
         + "  `sales_fact_1997`"
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
index 7aeda68..827ec84 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
@@ -8602,6 +8602,33 @@ public abstract class SqlOperatorBaseTest {
     tester.checkAgg("COUNT(DISTINCT 123)", stringValues, 1, 0d);
   }
 
+  @Test void testCountifFunc() {
+    tester.setFor(SqlLibraryOperators.COUNTIF, VM_FENNEL, VM_JAVA);
+    final SqlTester tester = libraryTester(SqlLibrary.BIG_QUERY);
+    tester.checkType("countif(true)", "BIGINT NOT NULL");
+    tester.checkType("countif(nullif(true,true))", "BIGINT NOT NULL");
+    tester.checkType("countif(false) filter (where true)", "BIGINT NOT NULL");
+
+    final String expectedError = "Invalid number of arguments to function "
+        + "'COUNTIF'. Was expecting 1 arguments";
+    tester.checkFails("^COUNTIF()^", expectedError, false);
+    tester.checkFails("^COUNTIF(true, false)^", expectedError, false);
+    final String expectedError2 = "Cannot apply 'COUNTIF' to arguments of "
+        + "type 'COUNTIF\\(<INTEGER>\\)'\\. Supported form\\(s\\): "
+        + "'COUNTIF\\(<BOOLEAN>\\)'";
+    tester.checkFails("^COUNTIF(1)^", expectedError2, false);
+
+    final String[] values = {"1", "2", "CAST(NULL AS INTEGER)", "1"};
+    tester.checkAgg("countif(x > 0)", values, 3, 0d);
+    tester.checkAgg("countif(x < 2)", values, 2, 0d);
+    tester.checkAgg("countif(x is not null) filter (where x < 2)",
+        values, 2, 0d);
+    tester.checkAgg("countif(x < 2) filter (where x is not null)",
+        values, 2, 0d);
+    tester.checkAgg("countif(x between 1 and 2)", values, 3, 0d);
+    tester.checkAgg("countif(x < 0)", values, 0, 0d);
+  }
+
   @Test void testApproxCountDistinctFunc() {
     tester.setFor(SqlStdOperatorTable.COUNT, VM_EXPAND);
     tester.checkFails("approx_count_distinct(^*^)", "Unknown identifier '\\*'",
diff --git a/core/src/test/resources/sql/agg.iq b/core/src/test/resources/sql/agg.iq
index 25e627b..2dba1c3 100644
--- a/core/src/test/resources/sql/agg.iq
+++ b/core/src/test/resources/sql/agg.iq
@@ -2890,4 +2890,44 @@ from emp group by gender;
 
 !ok
 
+# COUNTIF(b) (BigQuery) is equivalent to COUNT(*) FILTER (WHERE b)
+select deptno, countif(gender = 'F') as f
+from emp
+group by deptno;
++--------+---+
+| DEPTNO | F |
++--------+---+
+|     10 | 1 |
+|     20 | 0 |
+|     30 | 2 |
+|     50 | 1 |
+|     60 | 1 |
+|        | 1 |
++--------+---+
+(6 rows)
+
+!ok
+
+select countif(gender = 'F') filter (where deptno = 30) as f
+from emp;
++---+
+| F |
++---+
+| 2 |
++---+
+(1 row)
+
+!ok
+
+select countif(a > 0) + countif(a > 1) + countif(c > 1) as c
+from (select 1 as a, 2 as b, 3 as c);
++---+
+| C |
++---+
+| 2 |
++---+
+(1 row)
+
+!ok
+
 # End agg.iq
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 3f995e7..260f8a3 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -2522,6 +2522,7 @@ Dialect-specific aggregate functions.
 | b p | ARRAY_CONCAT_AGG( [ ALL &#124; DISTINCT ] value [ ORDER BY orderItem [, orderItem ]* ] ) | Concatenates arrays into arrays
 | p | BOOL_AND(condition)                            | Synonym for `EVERY`
 | p | BOOL_OR(condition)                             | Synonym for `SOME`
+| b | COUNTIF(condition)                             | Returns the number of rows for which *condition* is TRUE; equivalent to `COUNT(*) FILTER (WHERE condition)`
 | b | LOGICAL_AND(condition)                         | Synonym for `EVERY`
 | b | LOGICAL_OR(condition)                          | Synonym for `SOME`
 | b p | STRING_AGG( [ ALL &#124; DISTINCT ] value [, separator] [ ORDER BY orderItem [, orderItem ]* ] ) | Synonym for `LISTAGG`


[calcite] 02/04: Add class SqlBasicAggFunction

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 1fec052330168e0d0b416ed884b1db03ca76ceff
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Sat Oct 31 23:40:17 2020 -0700

    Add class SqlBasicAggFunction
---
 .../org/apache/calcite/sql/SqlAggFunction.java     |   3 +
 .../calcite/sql/fun/SqlBasicAggFunction.java       | 154 +++++++++++++++++++++
 .../calcite/sql/fun/SqlLibraryOperators.java       |  58 +++-----
 .../calcite/sql/fun/SqlStdOperatorTable.java       |  35 ++---
 .../calcite/sql/fun/SqlStringAggAggFunction.java   |  52 -------
 5 files changed, 189 insertions(+), 113 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
index cdff639..099c08a 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
@@ -19,6 +19,7 @@ package org.apache.calcite.sql;
 import org.apache.calcite.plan.Context;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.fun.SqlBasicAggFunction;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
@@ -33,6 +34,8 @@ import javax.annotation.Nonnull;
 /**
  * Abstract base class for the definition of an aggregate function: an operator
  * which aggregates sets of values into a result.
+ *
+ * @see SqlBasicAggFunction
  */
 public abstract class SqlAggFunction extends SqlFunction implements Context {
   private final boolean requiresOrder;
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlBasicAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlBasicAggFunction.java
new file mode 100644
index 0000000..f553f13
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlBasicAggFunction.java
@@ -0,0 +1,154 @@
+/*
+ * 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.sql.fun;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Optionality;
+
+import java.util.Objects;
+import javax.annotation.Nonnull;
+
+/**
+ * Concrete implementation of {@link SqlAggFunction}.
+ *
+ * <p>The class is final, and instances are immutable.
+ *
+ * <p>Instances are created only by {@link SqlBasicAggFunction#create} and are
+ * "modified" by "wither" methods such as {@link #withDistinct} to create a new
+ * instance with one property changed. Since the class is final, you can modify
+ * behavior only by providing strategy objects, not by overriding methods in a
+ * sub-class.
+ */
+public final class SqlBasicAggFunction extends SqlAggFunction {
+  private final Optionality distinctOptionality;
+  private final SqlSyntax syntax;
+  private final boolean allowsNullTreatment;
+
+  //~ Constructors -----------------------------------------------------------
+
+  private SqlBasicAggFunction(String name, SqlIdentifier sqlIdentifier,
+      SqlKind kind, SqlReturnTypeInference returnTypeInference,
+      SqlOperandTypeInference operandTypeInference,
+      SqlOperandTypeChecker operandTypeChecker, SqlFunctionCategory funcType,
+      boolean requiresOrder, boolean requiresOver,
+      Optionality requiresGroupOrder, Optionality distinctOptionality,
+      SqlSyntax syntax, boolean allowsNullTreatment) {
+    super(name, sqlIdentifier, kind,
+        Objects.requireNonNull(returnTypeInference), operandTypeInference,
+        Objects.requireNonNull(operandTypeChecker),
+        Objects.requireNonNull(funcType), requiresOrder, requiresOver,
+        requiresGroupOrder);
+    this.distinctOptionality = Objects.requireNonNull(distinctOptionality);
+    this.syntax = Objects.requireNonNull(syntax);
+    this.allowsNullTreatment = allowsNullTreatment;
+  }
+
+  /** Creates a SqlBasicAggFunction whose name is the same as its kind. */
+  public static SqlBasicAggFunction create(SqlKind kind,
+      SqlReturnTypeInference returnTypeInference,
+      SqlOperandTypeChecker operandTypeChecker) {
+    return create(kind.name(), kind, returnTypeInference, operandTypeChecker);
+  }
+
+  /** Creates a SqlBasicAggFunction. */
+  public static SqlBasicAggFunction create(String name, SqlKind kind,
+      SqlReturnTypeInference returnTypeInference,
+      SqlOperandTypeChecker operandTypeChecker) {
+    return new SqlBasicAggFunction(name, null, kind, returnTypeInference, null,
+        operandTypeChecker, SqlFunctionCategory.NUMERIC, false, false,
+        Optionality.FORBIDDEN, Optionality.OPTIONAL, SqlSyntax.FUNCTION, false);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public RelDataType deriveType(SqlValidator validator,
+      SqlValidatorScope scope, SqlCall call) {
+    if (syntax == SqlSyntax.ORDERED_FUNCTION) {
+      call = ReturnTypes.stripOrderBy(call);
+    }
+    return super.deriveType(validator, scope, call);
+  }
+
+  @Override @Nonnull public Optionality getDistinctOptionality() {
+    return distinctOptionality;
+  }
+
+  /** Sets {@link #getDistinctOptionality()}. */
+  SqlBasicAggFunction withDistinct(@Nonnull Optionality distinctOptionality) {
+    return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
+        getReturnTypeInference(), getOperandTypeInference(),
+        getOperandTypeChecker(), getFunctionType(), requiresOrder(),
+        requiresOver(), requiresGroupOrder(), distinctOptionality, syntax,
+        allowsNullTreatment);
+  }
+
+  /** Sets {@link #getFunctionType()}. */
+  public SqlBasicAggFunction withFunctionType(SqlFunctionCategory category) {
+    return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
+        getReturnTypeInference(), getOperandTypeInference(),
+        getOperandTypeChecker(), category, requiresOrder(),
+        requiresOver(), requiresGroupOrder(), distinctOptionality, syntax,
+        allowsNullTreatment);
+  }
+
+  @Override public SqlSyntax getSyntax() {
+    return syntax;
+  }
+
+  /** Sets {@link #getSyntax()}. */
+  public SqlBasicAggFunction withSyntax(SqlSyntax syntax) {
+    return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
+        getReturnTypeInference(), getOperandTypeInference(),
+        getOperandTypeChecker(), getFunctionType(), requiresOrder(),
+        requiresOver(), requiresGroupOrder(), distinctOptionality, syntax,
+        allowsNullTreatment);
+  }
+
+  @Override public boolean allowsNullTreatment() {
+    return allowsNullTreatment;
+  }
+
+  /** Sets {@link #allowsNullTreatment()}. */
+  public SqlBasicAggFunction withAllowsNullTreatment(boolean allowsNullTreatment) {
+    return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
+        getReturnTypeInference(), getOperandTypeInference(),
+        getOperandTypeChecker(), getFunctionType(), requiresOrder(),
+        requiresOver(), requiresGroupOrder(), distinctOptionality, syntax,
+        allowsNullTreatment);
+  }
+
+  /** Sets {@link #requiresGroupOrder()}. */
+  public SqlBasicAggFunction withGroupOrder(Optionality groupOrder) {
+    return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
+        getReturnTypeInference(), getOperandTypeInference(),
+        getOperandTypeChecker(), getFunctionType(), requiresOrder(),
+        requiresOver(), groupOrder, distinctOptionality, syntax,
+        allowsNullTreatment);
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
index cc9cb2b..9cb9e0f 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
@@ -19,7 +19,6 @@ package org.apache.calcite.sql.fun;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
-import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
@@ -35,10 +34,7 @@ import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeTransforms;
-import org.apache.calcite.sql.validate.SqlValidator;
-import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.Litmus;
-import org.apache.calcite.util.Optionality;
 
 import com.google.common.collect.ImmutableList;
 
@@ -51,7 +47,6 @@ import static org.apache.calcite.sql.fun.SqlLibrary.MYSQL;
 import static org.apache.calcite.sql.fun.SqlLibrary.ORACLE;
 import static org.apache.calcite.sql.fun.SqlLibrary.POSTGRESQL;
 import static org.apache.calcite.sql.fun.SqlLibrary.SPARK;
-import static org.apache.calcite.sql.type.ReturnTypes.stripOrderBy;
 
 /**
  * Defines functions and operators that are not part of standard SQL but
@@ -278,46 +273,37 @@ public abstract class SqlLibraryOperators {
    * in BigQuery and PostgreSQL, gathers values into arrays. */
   @LibraryOperator(libraries = {POSTGRESQL, BIG_QUERY})
   public static final SqlAggFunction ARRAY_AGG =
-      new SqlAggFunction("ARRAY_AGG", null, SqlKind.ARRAY_AGG,
-          ReturnTypes.andThen(ReturnTypes::stripOrderBy, ReturnTypes.TO_ARRAY),
-          null, OperandTypes.ANY, SqlFunctionCategory.SYSTEM, false, false,
-          Optionality.OPTIONAL) {
-        @Override public SqlSyntax getSyntax() {
-          return SqlSyntax.ORDERED_FUNCTION;
-        }
-
-        @Override public boolean allowsNullTreatment() {
-          return true;
-        }
-
-        @Override public RelDataType deriveType(SqlValidator validator,
-            SqlValidatorScope scope, SqlCall call) {
-          return super.deriveType(validator, scope, stripOrderBy(call));
-        }
-      };
+      SqlBasicAggFunction
+          .create(SqlKind.ARRAY_AGG,
+              ReturnTypes.andThen(ReturnTypes::stripOrderBy,
+                  ReturnTypes.TO_ARRAY), OperandTypes.ANY)
+          .withFunctionType(SqlFunctionCategory.SYSTEM)
+          .withSyntax(SqlSyntax.ORDERED_FUNCTION)
+          .withAllowsNullTreatment(true);
 
   /** The "ARRAY_CONCAT_AGG(value [ ORDER BY ...])" aggregate function,
    * in BigQuery and PostgreSQL, concatenates array values into arrays. */
   @LibraryOperator(libraries = {POSTGRESQL, BIG_QUERY})
   public static final SqlAggFunction ARRAY_CONCAT_AGG =
-      new SqlAggFunction("ARRAY_CONCAT_AGG", null, SqlKind.ARRAY_CONCAT_AGG,
-          ReturnTypes.ARG0, null, OperandTypes.ARRAY,
-          SqlFunctionCategory.SYSTEM, false, false, Optionality.OPTIONAL) {
-        @Override public SqlSyntax getSyntax() {
-          return SqlSyntax.ORDERED_FUNCTION;
-        }
-
-        @Override public RelDataType deriveType(SqlValidator validator,
-            SqlValidatorScope scope, SqlCall call) {
-          return super.deriveType(validator, scope, stripOrderBy(call));
-        }
-      };
+      SqlBasicAggFunction
+          .create(SqlKind.ARRAY_CONCAT_AGG, ReturnTypes.ARG0,
+              OperandTypes.ARRAY)
+          .withFunctionType(SqlFunctionCategory.SYSTEM)
+          .withSyntax(SqlSyntax.ORDERED_FUNCTION);
 
   /** The "STRING_AGG(value [, separator ] [ ORDER BY ...])" aggregate function,
    * BigQuery and PostgreSQL's equivalent of
-   * {@link SqlStdOperatorTable#LISTAGG}. */
+   * {@link SqlStdOperatorTable#LISTAGG}.
+   *
+   * <p>{@code STRING_AGG(v, sep ORDER BY x, y)} is implemented by
+   * rewriting to {@code LISTAGG(v, sep) WITHIN GROUP (ORDER BY x, y)}. */
   @LibraryOperator(libraries = {POSTGRESQL, BIG_QUERY})
-  public static final SqlAggFunction STRING_AGG = new SqlStringAggAggFunction();
+  public static final SqlAggFunction STRING_AGG =
+      SqlBasicAggFunction
+          .create(SqlKind.STRING_AGG, ReturnTypes.ARG0_NULLABLE,
+              OperandTypes.or(OperandTypes.STRING, OperandTypes.STRING_STRING))
+          .withFunctionType(SqlFunctionCategory.SYSTEM)
+          .withSyntax(SqlSyntax.ORDERED_FUNCTION);
 
   /** The "DATE(string)" function, equivalent to "CAST(string AS DATE). */
   @LibraryOperator(libraries = {BIG_QUERY})
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index 7bc2459..b8f90ac 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -2200,15 +2200,10 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * The COLLECT operator. Multiset aggregator function.
    */
   public static final SqlAggFunction COLLECT =
-      new SqlAggFunction("COLLECT",
-          null,
-          SqlKind.COLLECT,
-          ReturnTypes.TO_MULTISET,
-          null,
-          OperandTypes.ANY,
-          SqlFunctionCategory.SYSTEM, false, false,
-          Optionality.OPTIONAL) {
-      };
+      SqlBasicAggFunction
+          .create(SqlKind.COLLECT, ReturnTypes.TO_MULTISET, OperandTypes.ANY)
+      .withFunctionType(SqlFunctionCategory.SYSTEM)
+      .withGroupOrder(Optionality.OPTIONAL);
 
   /**
    * The LISTAGG operator. String aggregator function.
@@ -2220,27 +2215,17 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * The FUSION operator. Multiset aggregator function.
    */
   public static final SqlAggFunction FUSION =
-      new SqlAggFunction("FUSION", null,
-          SqlKind.FUSION,
-          ReturnTypes.ARG0,
-          null,
-          OperandTypes.MULTISET,
-          SqlFunctionCategory.SYSTEM, false, false,
-          Optionality.FORBIDDEN) {
-      };
+      SqlBasicAggFunction
+          .create(SqlKind.FUSION, ReturnTypes.ARG0, OperandTypes.MULTISET)
+          .withFunctionType(SqlFunctionCategory.SYSTEM);
 
   /**
    * The INTERSECTION operator. Multiset aggregator function.
    */
   public static final SqlAggFunction INTERSECTION =
-      new SqlAggFunction("INTERSECTION", null,
-          SqlKind.INTERSECTION,
-          ReturnTypes.ARG0,
-          null,
-          OperandTypes.MULTISET,
-          SqlFunctionCategory.SYSTEM, false, false,
-          Optionality.FORBIDDEN) {
-      };
+      SqlBasicAggFunction
+          .create(SqlKind.INTERSECTION, ReturnTypes.ARG0, OperandTypes.MULTISET)
+          .withFunctionType(SqlFunctionCategory.SYSTEM);
 
   /** The sequence next value function: <code>NEXT VALUE FOR sequence</code>. */
   public static final SqlOperator NEXT_VALUE =
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStringAggAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStringAggAggFunction.java
deleted file mode 100644
index 7938e91..0000000
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStringAggAggFunction.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.sql.fun;
-
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlSyntax;
-import org.apache.calcite.sql.type.ReturnTypes;
-import org.apache.calcite.sql.validate.SqlValidator;
-import org.apache.calcite.sql.validate.SqlValidatorScope;
-
-import static org.apache.calcite.sql.type.ReturnTypes.stripOrderBy;
-
-/**
- * <code>STRING_AGG</code> aggregate function
- * returns the concatenation of its group rows;
- * it is the PostgreSQL and BigQuery equivalent of {@code LISTAGG}.
- *
- * <p>{@code STRING_AGG(v, sep ORDER BY x, y)} is implemented by
- * rewriting to {@code LISTAGG(v, sep) WITHIN GROUP (ORDER BY x, y)}.
- *
- * @see SqlListaggAggFunction
- */
-class SqlStringAggAggFunction extends SqlListaggAggFunction {
-  SqlStringAggAggFunction() {
-    super(SqlKind.STRING_AGG, ReturnTypes.ARG0_NULLABLE);
-  }
-
-  @Override public SqlSyntax getSyntax() {
-    return SqlSyntax.ORDERED_FUNCTION;
-  }
-
-  @Override public RelDataType deriveType(SqlValidator validator,
-      SqlValidatorScope scope, SqlCall call) {
-    return super.deriveType(validator, scope, stripOrderBy(call));
-  }
-}