[CALCITE-4334] LITERAL_AGG, an internal aggregate function that returns a constant value

LITERAL_AGG is internal, so you cannot use it from SQL. A
call to LITERAL_AGG(true) is created implicitly when you
write

  SELECT deptno, SUM(sal), true
  FROM Emp
  GROUP BY deptno

Add RelBuilder.literalAgg.

RelBuilder.aggregate now adds an aggregate function to ensure
that there is at least one field. The aggregate function is
LITERAL_AGG; thus the pseudo-query

  SELECT /* no fields */
  FROM t
  GROUP BY ()

becomes the valid query

  SELECT TRUE AS dummy
  FROM t
  GROUP BY ().

Solves that problem that an Aggregate query that uses no
columns throws AssertionError in RelFieldTrimmer.trimFields.

Add an option RelBuilder.Config.preventEmptyFieldList to
turn off the behavior we just added.

Close apache/calcite#3209
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java
index ab130a0..2c4b1b4 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/AggResultContext.java
@@ -40,5 +40,10 @@
    * cast to the appropriate type. */
   Expression keyField(int i);
 
+  /** Returns the aggregate call. */
   AggregateCall call();
+
+  /** Returns a {@code RexToLixTranslator}
+   * suitable to transform the result. */
+  RexToLixTranslator resultTranslator();
 }
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
index 4f4eeaf..4ee0836 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysType.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.adapter.enumerable;
 
+import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.ParameterExpression;
 import org.apache.calcite.rel.RelCollation;
@@ -50,6 +51,9 @@
    * field type is {@code Object} even if the field is not nullable.</p> */
   Type getJavaFieldType(int field);
 
+  /** Returns the type factory. */
+  JavaTypeFactory getTypeFactory();
+
   /** Returns the physical type of a field. */
   PhysType field(int ordinal);
 
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
index 966b050..55a44b2 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/PhysTypeImpl.java
@@ -117,6 +117,10 @@
     return format;
   }
 
+  @Override public JavaTypeFactory getTypeFactory() {
+    return typeFactory;
+  }
+
   @Override public PhysType project(List<Integer> integers, JavaRowFormat format) {
     return project(integers, false, format);
   }
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index 4d8e51b..7c94ed0 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -98,6 +98,8 @@
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import static org.apache.calcite.adapter.enumerable.EnumUtils.generateCollatorExpression;
 import static org.apache.calcite.linq4j.tree.ExpressionType.Add;
 import static org.apache.calcite.linq4j.tree.ExpressionType.Divide;
@@ -111,6 +113,7 @@
 import static org.apache.calcite.linq4j.tree.ExpressionType.NotEqual;
 import static org.apache.calcite.linq4j.tree.ExpressionType.Subtract;
 import static org.apache.calcite.linq4j.tree.ExpressionType.UnaryPlus;
+import static org.apache.calcite.sql.fun.SqlInternalOperators.LITERAL_AGG;
 import static org.apache.calcite.sql.fun.SqlInternalOperators.THROW_UNLESS;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.ACOSH;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.ARRAY;
@@ -885,6 +888,7 @@
           constructorSupplier(GroupingImplementor.class);
       aggMap.put(GROUPING, grouping);
       aggMap.put(GROUPING_ID, grouping);
+      aggMap.put(LITERAL_AGG, constructorSupplier(LiteralAggImplementor.class));
       winAggMap.put(RANK, constructorSupplier(RankImplementor.class));
       winAggMap.put(DENSE_RANK, constructorSupplier(DenseRankImplementor.class));
       winAggMap.put(ROW_NUMBER, constructorSupplier(RowNumberImplementor.class));
@@ -1702,6 +1706,27 @@
     }
   }
 
+  /** Implementor for the {@code LITERAL_AGG} aggregate function. */
+  static class LiteralAggImplementor implements AggImplementor {
+    @Override public List<Type> getStateType(AggContext info) {
+      return ImmutableList.of();
+    }
+
+    @Override public void implementReset(AggContext info, AggResetContext reset) {
+    }
+
+    @Override public void implementAdd(AggContext info, AggAddContext add) {
+    }
+
+    @Override public Expression implementResult(AggContext info,
+        AggResultContext result) {
+      checkArgument(info.aggregation().kind == SqlKind.LITERAL_AGG);
+      checkArgument(result.call().rexList.size() == 1);
+      final RexNode rexNode = result.call().rexList.get(0);
+      return result.resultTranslator().translate(rexNode);
+    }
+  }
+
   /** Implementor for user-defined aggregate functions. */
   public static class UserDefinedAggReflectiveImplementor
       extends StrictAggImplementor {
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
index b36d670..17a6843 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/impl/AggResultContextImpl.java
@@ -18,10 +18,12 @@
 
 import org.apache.calcite.adapter.enumerable.AggResultContext;
 import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
 import org.apache.calcite.linq4j.tree.BlockBuilder;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.ParameterExpression;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 
 import org.checkerframework.checker.nullness.qual.Nullable;
 
@@ -69,4 +71,10 @@
   @Override public AggregateCall call() {
     return requireNonNull(call, "call");
   }
+
+  @Override public RexToLixTranslator resultTranslator() {
+    requireNonNull(keyPhysType, "keyPhysType");
+    return RexToLixTranslator.forAggregation(keyPhysType.getTypeFactory(),
+        currentBlock(), null, SqlConformanceEnum.DEFAULT);
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
index c860c7c..96a7b0d 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java
@@ -25,6 +25,7 @@
 import org.apache.calcite.plan.volcano.RelSubset;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Correlate;
 import org.apache.calcite.rel.core.Exchange;
 import org.apache.calcite.rel.core.Filter;
@@ -49,7 +50,7 @@
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlInternalOperators;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.Bug;
@@ -197,19 +198,11 @@
         int sIdx = ((RexInputRef) expr.e).getIndex();
         m.set(sIdx, expr.i);
         columnsMappedBuilder.set(sIdx);
-      // Project can also generate constants. We need to include them.
-      } else if (RexLiteral.isNullLiteral(expr.e)) {
-        projectPullUpPredicates.add(
-            rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL,
-                rexBuilder.makeInputRef(project, expr.i)));
       } else if (RexUtil.isConstant(expr.e)) {
-        final List<RexNode> args =
-            ImmutableList.of(rexBuilder.makeInputRef(project, expr.i), expr.e);
-        final SqlOperator op = args.get(0).getType().isNullable()
-            || args.get(1).getType().isNullable()
-            ? SqlStdOperatorTable.IS_NOT_DISTINCT_FROM
-            : SqlStdOperatorTable.EQUALS;
-        projectPullUpPredicates.add(rexBuilder.makeCall(op, args));
+        // Project can also generate constants (including NULL). We need to
+        // include them.
+        projectPullUpPredicates.add(
+            eqConstant(project, rexBuilder, expr.i, expr.e));
       }
     }
 
@@ -226,6 +219,23 @@
     return RelOptPredicateList.of(rexBuilder, projectPullUpPredicates);
   }
 
+  /** Returns a predicate that field {@code i} of relational expression
+   * {@code r} is equal to a constant expression (using
+   * {@code IS NOT DISTINCT FROM} if the expression is nullable, or
+   * {@code IS NULL} if it is literal null. */
+  private static RexNode eqConstant(RelNode r, RexBuilder rexBuilder, int i,
+      RexNode e) {
+    final RexInputRef ref = rexBuilder.makeInputRef(r, i);
+    if (RexLiteral.isNullLiteral(e)) {
+      return rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, ref);
+    } else if (ref.getType().isNullable() || e.getType().isNullable()) {
+      return rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_DISTINCT_FROM, ref,
+          e);
+    } else {
+      return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, ref, e);
+    }
+  }
+
   /** Converts a predicate on a particular set of columns into a predicate on
    * a subset of those columns, weakening if necessary.
    *
@@ -373,6 +383,18 @@
         aggPullUpPredicates.add(r);
       }
     }
+
+    i = agg.getGroupCount();
+    for (AggregateCall aggregateCall : agg.getAggCallList()) {
+      if (aggregateCall.getAggregation() == SqlInternalOperators.LITERAL_AGG) {
+        // The query
+        //   SELECT x, LITERAL_AGG[42]() AS y FROM t GROUP BY x
+        // has predicate "y = 42"
+        aggPullUpPredicates.add(
+            eqConstant(agg, rexBuilder, i, aggregateCall.rexList.get(0)));
+      }
+      ++i;
+    }
     return RelOptPredicateList.of(rexBuilder, aggPullUpPredicates);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
index 35dd660..d702ff9 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
@@ -460,7 +460,17 @@
         addSelect(selectList, sqlExpr, e.getRowType());
       }
 
-      builder.setSelect(new SqlNodeList(selectList, POS));
+      final SqlNodeList selectNodeList = new SqlNodeList(selectList, POS);
+      if (builder.select.getGroup() == null
+          && builder.select.getHaving() == null
+          && SqlUtil.containsAgg(builder.select.getSelectList())
+          && !SqlUtil.containsAgg(selectNodeList)) {
+        // We are just about to remove the last aggregate function from the
+        // SELECT clause. The "GROUP BY ()" was implicit, but we now need to
+        // make it explicit.
+        builder.setGroupBy(SqlNodeList.EMPTY);
+      }
+      builder.setSelect(selectNodeList);
     }
     return builder.result();
   }
@@ -568,7 +578,8 @@
     builder.setSelect(new SqlNodeList(selectList, POS));
     if (!groupByList.isEmpty() || e.getAggCallList().isEmpty()) {
       // Some databases don't support "GROUP BY ()". We can omit it as long
-      // as there is at least one aggregate function.
+      // as there is at least one aggregate function. (We have to take care
+      // if we later prune away that last aggregate function.)
       builder.setGroupBy(new SqlNodeList(groupByList, POS));
     }
 
diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
index 1b2151b..1f5fa0f 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
@@ -80,6 +80,7 @@
 import org.apache.calcite.sql.SqlWindow;
 import org.apache.calcite.sql.fun.SqlCase;
 import org.apache.calcite.sql.fun.SqlCountAggFunction;
+import org.apache.calcite.sql.fun.SqlInternalOperators;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.fun.SqlSumEmptyIsZeroAggFunction;
 import org.apache.calcite.sql.parser.SqlParserPos;
@@ -1168,21 +1169,28 @@
     /** Converts a call to an aggregate function to an expression. */
     public SqlNode toSql(AggregateCall aggCall) {
       return toSql(aggCall.getAggregation(), aggCall.isDistinct(),
+          Util.transform(aggCall.rexList, e -> toSql((RexProgram) null, e)),
           Util.transform(aggCall.getArgList(), this::field),
           aggCall.filterArg, aggCall.collation, aggCall.isApproximate());
     }
 
     /** Converts a call to an aggregate function, with a given list of operands,
      * to an expression. */
-    private SqlCall toSql(SqlOperator op, boolean distinct,
-        List<SqlNode> operandList, int filterArg, RelCollation collation,
+    private SqlNode toSql(SqlOperator op, boolean distinct,
+        List<SqlNode> preOperandList, List<SqlNode> operandList,
+        int filterArg, RelCollation collation,
         boolean approximate) {
       final SqlLiteral qualifier =
           distinct ? SqlSelectKeyword.DISTINCT.symbol(POS) : null;
+
+      if (op == SqlInternalOperators.LITERAL_AGG) {
+        return preOperandList.get(0);
+      }
+
       if (op instanceof SqlSumEmptyIsZeroAggFunction) {
         final SqlNode node =
-            toSql(SqlStdOperatorTable.SUM, distinct, operandList, filterArg,
-                collation, approximate);
+            toSql(SqlStdOperatorTable.SUM, distinct, preOperandList,
+                operandList, filterArg, collation, approximate);
         return SqlStdOperatorTable.COALESCE.createCall(POS, node, ZERO);
       }
 
@@ -1206,7 +1214,8 @@
         if (operandList.size() > 1) {
           newOperandList.addAll(Util.skip(operandList));
         }
-        return toSql(op, distinct, newOperandList, -1, collation, approximate);
+        return toSql(op, distinct, preOperandList, newOperandList, -1,
+            collation, approximate);
       }
 
       if (op instanceof SqlCountAggFunction && operandList.isEmpty()) {
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
index ee85686..14a894e 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateRemoveRule.java
@@ -29,10 +29,12 @@
 import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlSplittableAggFunction;
+import org.apache.calcite.sql.SqlSingletonAggFunction;
+import org.apache.calcite.sql.SqlStaticAggFunction;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 
+import org.checkerframework.checker.nullness.qual.Nullable;
 import org.immutables.value.Value;
 
 import java.util.ArrayList;
@@ -78,18 +80,32 @@
       return false;
     }
     // If any aggregate functions do not support splitting, bail out.
-    for (AggregateCall aggregateCall : aggregate.getAggCallList()) {
-      if (aggregateCall.filterArg >= 0
-          || !aggregateCall.getAggregation()
-              .maybeUnwrap(SqlSplittableAggFunction.class).isPresent()) {
-        return false;
-      }
-    }
-    return true;
+    return aggregate.getAggCallList().stream()
+        .allMatch(AggregateRemoveRule::canFlatten);
   }
 
   //~ Methods ----------------------------------------------------------------
 
+  /** Returns whether an aggregate call can be converted to a single-row
+   * expression.
+   *
+   * <p>For example, 'SUM(x)' can be converted to 'x' if we know that each
+   * group contains only one row. */
+  static boolean canFlatten(AggregateCall aggregateCall) {
+    return aggregateCall.filterArg < 0
+        && (aggregateCall.getAggregation()
+                .maybeUnwrap(SqlSingletonAggFunction.class).isPresent()
+            || aggregateCall.getAggregation()
+                .maybeUnwrap(SqlStaticAggFunction.class).isPresent());
+  }
+
+  /** As {@link #canFlatten}, but only allows static aggregate functions. */
+  public static boolean canFlattenStatic(AggregateCall aggregateCall) {
+    return aggregateCall.filterArg < 0
+        && aggregateCall.getAggregation()
+            .maybeUnwrap(SqlStaticAggFunction.class).isPresent();
+  }
+
   @Override public void onMatch(RelOptRuleCall call) {
     final Aggregate aggregate = call.rel(0);
     final RelNode input = aggregate.getInput();
@@ -109,8 +125,21 @@
         // function to SUM0 and COUNT.
         return;
       }
-      final SqlSplittableAggFunction splitter =
-          aggregation.unwrapOrThrow(SqlSplittableAggFunction.class);
+      final @Nullable SqlStaticAggFunction staticAggFunction =
+          aggregation.unwrap(SqlStaticAggFunction.class);
+      if (staticAggFunction != null) {
+        final RexNode constant =
+            staticAggFunction.constant(rexBuilder,
+                aggregate.getGroupSet(), aggregate.groupSets, aggCall);
+        if (constant != null) {
+          final RexNode cast =
+              rexBuilder.ensureType(aggCall.type, constant, false);
+          projects.add(cast);
+          continue;
+        }
+      }
+      final SqlSingletonAggFunction splitter =
+          aggregation.unwrapOrThrow(SqlSingletonAggFunction.class);
       final RexNode singleton =
           splitter.singleton(rexBuilder, input.getRowType(), aggCall);
       final RexNode cast =
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java b/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
index 2dcd493..b005123 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/SubQueryRemoveRule.java
@@ -720,9 +720,8 @@
         builder.push(e.rel);
         // fall through
       default:
-        fields.add(builder.alias(trueLiteral, "i"));
-        builder.project(fields);
-        builder.distinct();
+        builder.aggregate(builder.groupKey(fields),
+            builder.literalAgg(true).as("i"));
       }
     }
 
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 d52322b..b34c9d2 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -772,6 +772,14 @@
 
   // internal operators (evaluated in validator) 200-299
 
+  /** The {@code LITERAL_AGG} aggregate function that always returns the same
+   * literal (even if the group is empty).
+   *
+   * <p>Useful during optimization because it allows you to, say, generate a
+   * non-null value (to detect outer joins) in an Aggregate without an extra
+   * Project. */
+  LITERAL_AGG,
+
   /**
    * Literal chain operator (for composite string literals).
    * An internal operator that does not appear in SQL syntax.
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSingletonAggFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlSingletonAggFunction.java
new file mode 100644
index 0000000..df05666
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSingletonAggFunction.java
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+
+/** Aggregate function that knows how to convert itself to a scalar value
+ * when applied to a single row. */
+public interface SqlSingletonAggFunction {
+  /** Generates an expression for the value of the aggregate function when
+   * applied to a single row.
+   *
+   * <p>For example, if there is one row:
+   * <ul>
+   *   <li>{@code SUM(x)} is {@code x}
+   *   <li>{@code MIN(x)} is {@code x}
+   *   <li>{@code MAX(x)} is {@code x}
+   *   <li>{@code COUNT(x)} is {@code CASE WHEN x IS NOT NULL THEN 1 ELSE 0 END 1}
+   *   which can be simplified to {@code 1} if {@code x} is never null
+   *   <li>{@code COUNT(*)} is 1
+   *   <li>{@code GROUPING(deptno)} if 0 if {@code deptno} is being grouped,
+   *       1 otherwise
+   * </ul>
+   *
+   * @param rexBuilder Rex builder
+   * @param inputRowType Input row type
+   * @param aggregateCall Aggregate call
+   *
+   * @return Expression for single row
+   */
+  RexNode singleton(RexBuilder rexBuilder, RelDataType inputRowType,
+      AggregateCall aggregateCall);
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSplittableAggFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlSplittableAggFunction.java
index 452ff6e..da4b310 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSplittableAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSplittableAggFunction.java
@@ -44,7 +44,7 @@
  * <p>For example, {@code COUNT(x)} can be split into {@code COUNT(x)} on
  * subsets followed by {@code SUM} to combine those counts.
  */
-public interface SqlSplittableAggFunction {
+public interface SqlSplittableAggFunction extends SqlSingletonAggFunction {
   AggregateCall split(AggregateCall aggregateCall,
       Mappings.TargetMapping mapping);
 
@@ -75,28 +75,6 @@
       int offset, RelDataType inputRowType, AggregateCall aggregateCall,
       int leftSubTotal, int rightSubTotal);
 
-  /** Generates an expression for the value of the aggregate function when
-   * applied to a single row.
-   *
-   * <p>For example, if there is one row:
-   * <ul>
-   *   <li>{@code SUM(x)} is {@code x}
-   *   <li>{@code MIN(x)} is {@code x}
-   *   <li>{@code MAX(x)} is {@code x}
-   *   <li>{@code COUNT(x)} is {@code CASE WHEN x IS NOT NULL THEN 1 ELSE 0 END 1}
-   *   which can be simplified to {@code 1} if {@code x} is never null
-   *   <li>{@code COUNT(*)} is 1
-   * </ul>
-   *
-   * @param rexBuilder Rex builder
-   * @param inputRowType Input row type
-   * @param aggregateCall Aggregate call
-   *
-   * @return Expression for single row
-   */
-  RexNode singleton(RexBuilder rexBuilder, RelDataType inputRowType,
-      AggregateCall aggregateCall);
-
   /**
    * Merge top and bottom aggregate calls into a single aggregate call,
    * if they are legit to merge.
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlStaticAggFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlStaticAggFunction.java
new file mode 100644
index 0000000..6ad5e88
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/SqlStaticAggFunction.java
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import com.google.common.collect.ImmutableList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/** Aggregate function whose value may be a constant expression, based on
+ * only the contents of the GROUP BY clause. */
+public interface SqlStaticAggFunction {
+  /** Generates an expression for the aggregate function; or null if the value
+   * is not constant.
+   *
+   * <p>For example:
+   * <ul>
+   *   <li>{@code GROUPING(deptno)} expands to literal {@code 1}
+   *       if the aggregate has {@code GROUP BY deptno}
+   * </ul>
+   *
+   * @param rexBuilder Rex builder
+   * @param groupSet Group set
+   * @param groupSets Group sets
+   * @param aggregateCall Aggregate call
+   *
+   * @return Expression for single row
+   */
+  @Nullable RexNode constant(RexBuilder rexBuilder, ImmutableBitSet groupSet,
+      ImmutableList<ImmutableBitSet> groupSets, AggregateCall aggregateCall);
+}
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 37c5afb..26362e9 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -29,6 +29,7 @@
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.sql.fun.SqlInOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlOperandMetadata;
@@ -37,6 +38,7 @@
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlNameMatcher;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.BarfingInvocationHandler;
@@ -1224,6 +1226,50 @@
     return op.createCall(pos, leftNode, rightNode);
   }
 
+  /**
+   * Returns whether a given node contains a {@link SqlInOperator}.
+   *
+   * @param node AST tree
+   */
+  public static boolean containsIn(SqlNode node) {
+    final Predicate<SqlCall> callPredicate = call ->
+        call.getOperator() instanceof SqlInOperator;
+    return containsCall(node, callPredicate);
+  }
+
+  /**
+   * Returns whether an AST tree contains a call to an aggregate function.
+   *
+   * @param node AST tree
+   */
+  public static boolean containsAgg(SqlNode node) {
+    final Predicate<SqlCall> callPredicate = call ->
+        call.getOperator().isAggregator();
+    return containsCall(node, callPredicate);
+  }
+
+  /** Returns whether an AST tree contains a call that matches a given
+   * predicate. */
+  private static boolean containsCall(SqlNode node,
+      Predicate<SqlCall> callPredicate) {
+    try {
+      SqlVisitor<Void> visitor =
+          new SqlBasicVisitor<Void>() {
+            @Override public Void visit(SqlCall call) {
+              if (callPredicate.test(call)) {
+                throw new Util.FoundOne(call);
+              }
+              return super.visit(call);
+            }
+          };
+      node.accept(visitor);
+      return false;
+    } catch (Util.FoundOne e) {
+      Util.swallow(e, null);
+      return true;
+    }
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlAnyValueAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlAnyValueAggFunction.java
index df9e17e..4117a84 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlAnyValueAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlAnyValueAggFunction.java
@@ -56,7 +56,7 @@
   //~ Methods ----------------------------------------------------------------
 
   @Override public <T extends Object> @Nullable T unwrap(Class<T> clazz) {
-    if (clazz == SqlSplittableAggFunction.class) {
+    if (clazz.isInstance(SqlSplittableAggFunction.SelfSplitter.INSTANCE)) {
       return clazz.cast(SqlSplittableAggFunction.SelfSplitter.INSTANCE);
     }
     return super.unwrap(clazz);
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
index 33f9d88..677998c 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlBasicAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlBasicAggFunction.java
@@ -22,6 +22,7 @@
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlStaticAggFunction;
 import org.apache.calcite.sql.SqlSyntax;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
@@ -47,6 +48,7 @@
  * sub-class.
  */
 public final class SqlBasicAggFunction extends SqlAggFunction {
+  private final @Nullable SqlStaticAggFunction staticFun;
   private final Optionality distinctOptionality;
   private final SqlSyntax syntax;
   private final boolean allowsNullTreatment;
@@ -59,6 +61,7 @@
       SqlKind kind, SqlReturnTypeInference returnTypeInference,
       @Nullable SqlOperandTypeInference operandTypeInference,
       SqlOperandTypeChecker operandTypeChecker,
+      @Nullable SqlStaticAggFunction staticFun,
       SqlFunctionCategory funcType,
       boolean requiresOrder, boolean requiresOver,
       Optionality requiresGroupOrder, Optionality distinctOptionality,
@@ -69,6 +72,7 @@
         requireNonNull(operandTypeChecker, "operandTypeChecker"),
         requireNonNull(funcType, "funcType"), requiresOrder, requiresOver,
         requiresGroupOrder);
+    this.staticFun = staticFun;
     this.distinctOptionality =
         requireNonNull(distinctOptionality, "distinctOptionality");
     this.syntax = requireNonNull(syntax, "syntax");
@@ -89,14 +93,17 @@
       SqlReturnTypeInference returnTypeInference,
       SqlOperandTypeChecker operandTypeChecker) {
     return new SqlBasicAggFunction(name, null, kind, returnTypeInference, null,
-        operandTypeChecker, SqlFunctionCategory.NUMERIC, false, false,
+        operandTypeChecker, null, SqlFunctionCategory.NUMERIC, false, false,
         Optionality.FORBIDDEN, Optionality.OPTIONAL, SqlSyntax.FUNCTION, false,
         false, false);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override public <T> @Nullable T unwrap(Class<T> clazz) {
+  @Override public <T extends Object> @Nullable T unwrap(Class<T> clazz) {
+    if (clazz.isInstance(staticFun)) {
+      return clazz.cast(staticFun);
+    }
     return super.unwrap(clazz);
   }
 
@@ -138,7 +145,7 @@
   public SqlAggFunction withName(String name) {
     return new SqlBasicAggFunction(name, getSqlIdentifier(), kind,
         getReturnTypeInference(), getOperandTypeInference(),
-        getOperandTypeChecker(), getFunctionType(), requiresOrder(),
+        getOperandTypeChecker(), staticFun, getFunctionType(), requiresOrder(),
         requiresOver(), requiresGroupOrder(), distinctOptionality, syntax,
         allowsNullTreatment, allowsSeparator, percentile);
   }
@@ -147,7 +154,7 @@
   SqlBasicAggFunction withDistinct(Optionality distinctOptionality) {
     return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
         getReturnTypeInference(), getOperandTypeInference(),
-        getOperandTypeChecker(), getFunctionType(), requiresOrder(),
+        getOperandTypeChecker(), staticFun, getFunctionType(), requiresOrder(),
         requiresOver(), requiresGroupOrder(), distinctOptionality, syntax,
         allowsNullTreatment, allowsSeparator, percentile);
   }
@@ -156,7 +163,7 @@
   public SqlBasicAggFunction withFunctionType(SqlFunctionCategory category) {
     return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
         getReturnTypeInference(), getOperandTypeInference(),
-        getOperandTypeChecker(), category, requiresOrder(),
+        getOperandTypeChecker(), staticFun, category, requiresOrder(),
         requiresOver(), requiresGroupOrder(), distinctOptionality, syntax,
         allowsNullTreatment, allowsSeparator, percentile);
   }
@@ -169,7 +176,7 @@
   public SqlBasicAggFunction withSyntax(SqlSyntax syntax) {
     return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
         getReturnTypeInference(), getOperandTypeInference(),
-        getOperandTypeChecker(), getFunctionType(), requiresOrder(),
+        getOperandTypeChecker(), staticFun, getFunctionType(), requiresOrder(),
         requiresOver(), requiresGroupOrder(), distinctOptionality, syntax,
         allowsNullTreatment, allowsSeparator, percentile);
   }
@@ -182,7 +189,7 @@
   public SqlBasicAggFunction withAllowsNullTreatment(boolean allowsNullTreatment) {
     return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
         getReturnTypeInference(), getOperandTypeInference(),
-        getOperandTypeChecker(), getFunctionType(), requiresOrder(),
+        getOperandTypeChecker(), staticFun, getFunctionType(), requiresOrder(),
         requiresOver(), requiresGroupOrder(), distinctOptionality, syntax,
         allowsNullTreatment, allowsSeparator, percentile);
   }
@@ -197,7 +204,7 @@
   public SqlBasicAggFunction withAllowsSeparator(boolean allowsSeparator) {
     return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
         getReturnTypeInference(), getOperandTypeInference(),
-        getOperandTypeChecker(), getFunctionType(), requiresOrder(),
+        getOperandTypeChecker(), staticFun, getFunctionType(), requiresOrder(),
         requiresOver(), requiresGroupOrder(), distinctOptionality, syntax,
         allowsNullTreatment, allowsSeparator, percentile);
   }
@@ -210,7 +217,7 @@
   public SqlBasicAggFunction withPercentile(boolean percentile) {
     return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
         getReturnTypeInference(), getOperandTypeInference(),
-        getOperandTypeChecker(), getFunctionType(), requiresOrder(),
+        getOperandTypeChecker(), staticFun, getFunctionType(), requiresOrder(),
         requiresOver(), requiresGroupOrder(), distinctOptionality, syntax,
         allowsNullTreatment, allowsSeparator, percentile);
   }
@@ -219,9 +226,18 @@
   public SqlBasicAggFunction withGroupOrder(Optionality groupOrder) {
     return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
         getReturnTypeInference(), getOperandTypeInference(),
-        getOperandTypeChecker(), getFunctionType(), requiresOrder(),
+        getOperandTypeChecker(), staticFun, getFunctionType(), requiresOrder(),
         requiresOver(), groupOrder, distinctOptionality, syntax,
         allowsNullTreatment, allowsSeparator, percentile);
   }
 
+  /** Sets that value to be returned when {@link #unwrap} is applied to
+   * {@link SqlStaticAggFunction}{@code .class}. */
+  public SqlBasicAggFunction withStatic(SqlStaticAggFunction staticFun) {
+    return new SqlBasicAggFunction(getName(), getSqlIdentifier(), kind,
+        getReturnTypeInference(), getOperandTypeInference(),
+        getOperandTypeChecker(), staticFun, getFunctionType(), requiresOrder(),
+        requiresOver(), requiresGroupOrder(), distinctOptionality, syntax,
+        allowsNullTreatment, allowsSeparator, percentile);
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlBitOpAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlBitOpAggFunction.java
index 30af5d4..17eacae 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlBitOpAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlBitOpAggFunction.java
@@ -57,7 +57,7 @@
   }
 
   @Override public <T extends Object> @Nullable T unwrap(Class<T> clazz) {
-    if (clazz == SqlSplittableAggFunction.class) {
+    if (clazz.isInstance(SqlSplittableAggFunction.SelfSplitter.INSTANCE)) {
       return clazz.cast(SqlSplittableAggFunction.SelfSplitter.INSTANCE);
     }
     return super.unwrap(clazz);
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
index c0f50c1..3007ad3 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCountAggFunction.java
@@ -92,7 +92,7 @@
   }
 
   @Override public <T extends Object> @Nullable T unwrap(Class<T> clazz) {
-    if (clazz == SqlSplittableAggFunction.class) {
+    if (clazz.isInstance(SqlSplittableAggFunction.CountSplitter.INSTANCE)) {
       return clazz.cast(SqlSplittableAggFunction.CountSplitter.INSTANCE);
     }
     return super.unwrap(clazz);
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupingFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupingFunction.java
index 3fed6a6..caf4aef 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupingFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlGroupingFunction.java
@@ -16,10 +16,21 @@
  */
 package org.apache.calcite.sql.fun;
 
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlStaticAggFunction;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import com.google.common.collect.ImmutableList;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.math.BigDecimal;
 
 /**
  * The {@code GROUPING} function. It accepts 1 or more arguments and they must be
@@ -39,8 +50,44 @@
  * <p>Some examples are in {@code agg.iq}.
  */
 class SqlGroupingFunction extends SqlAbstractGroupFunction {
+  private static final SqlStaticAggFunction STATIC =
+      SqlGroupingFunction::constant;
+
   SqlGroupingFunction(String name) {
     super(name, SqlKind.GROUPING, ReturnTypes.BIGINT, null,
         OperandTypes.ONE_OR_MORE, SqlFunctionCategory.SYSTEM);
   }
+
+  /** Implements {@link SqlStaticAggFunction}. */
+  private static @Nullable RexNode constant(RexBuilder rexBuilder,
+      ImmutableBitSet groupSet, ImmutableList<ImmutableBitSet> groupSets,
+      AggregateCall aggregateCall) {
+    // GROUPING(c1, ..., cN) evaluates to zero if every grouping set contains
+    // all of c1, ..., cN. For example,
+    //
+    //   SELECT GROUPING(deptno) AS gd, GROUPING(job) AS gj
+    //   FROM Emp
+    //   GROUP BY GROUPING SETS (deptno), (deptno, job);
+    //
+    // "gd" is zero for all rows, because both grouping sets contain "deptno";
+    // "gj" is 0 for some rows and 1 for others.
+    //
+    // Internally we allow GROUPING() with no arguments; it always
+    // evaluates to zero.
+    final ImmutableBitSet argSet =
+        ImmutableBitSet.of(aggregateCall.getArgList());
+    if (groupSets.stream().allMatch(set -> set.contains(argSet))) {
+      return rexBuilder.makeExactLiteral(BigDecimal.ZERO);
+    }
+
+    // GROUPING with one or more arguments
+    return null;
+  }
+
+  @Override public <T extends Object> @Nullable T unwrap(Class<T> clazz) {
+    if (clazz.isInstance(STATIC)) {
+      return clazz.cast(STATIC);
+    }
+    return super.unwrap(clazz);
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlInternalOperators.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlInternalOperators.java
index 492ac12..37a56ed 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlInternalOperators.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlInternalOperators.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.sql.fun;
 
 import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlInternalOperator;
 import org.apache.calcite.sql.SqlKind;
@@ -137,6 +138,10 @@
       SqlBasicOperator.create("OFFSET")
           .withPrecedence(SqlStdOperatorTable.UNION.getLeftPrec() - 2, true);
 
+  /** Aggregate function that always returns a given literal. */
+  public static final SqlAggFunction LITERAL_AGG =
+      SqlLiteralAggFunction.INSTANCE;
+
   /** Subject to change. */
   private static class SqlBasicOperator extends SqlOperator {
     @Override public SqlSyntax getSyntax() {
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralAggFunction.java
new file mode 100644
index 0000000..007f708
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLiteralAggFunction.java
@@ -0,0 +1,68 @@
+/*
+ * 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.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.SqlStaticAggFunction;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * The {@code LITERAL_AGG} aggregate function.
+ *
+ * <p>It accepts zero regular aggregate arguments (the kind that reference
+ * columns of the input group) but one argument of type
+ * {@link org.apache.calcite.rex.RexLiteral}, and its return is that literal.
+ */
+public class SqlLiteralAggFunction {
+  /** This is utility class. It is never instantiated. */
+  private SqlLiteralAggFunction() {
+  }
+
+  public static final SqlBasicAggFunction INSTANCE =
+      SqlBasicAggFunction.create(SqlKind.LITERAL_AGG,
+              SqlLiteralAggFunction::inferReturnType, OperandTypes.NILADIC)
+          .withStatic(SqlLiteralAggFunction::constant);
+
+  /** Implements {@link org.apache.calcite.sql.type.SqlReturnTypeInference}. */
+  private static RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+    // LITERAL_AGG takes one pre-operand and zero (post-)operands.
+    if (opBinding.getPreOperandCount() != 1
+        || opBinding.getOperandCount() != 1) {
+      throw new AssertionError();
+    }
+    return opBinding.getOperandType(0);
+  }
+
+  /** Implements {@link SqlStaticAggFunction}. */
+  private static @Nullable RexNode constant(RexBuilder rexBuilder,
+      ImmutableBitSet groupSet, ImmutableList<ImmutableBitSet> groupSets,
+      AggregateCall aggregateCall) {
+    // LITERAL_AGG[literal]() evaluates to "literal".
+    return Iterables.getOnlyElement(aggregateCall.rexList);
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
index 3c61d81..5750b2b 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
@@ -146,7 +146,7 @@
   }
 
   @Override public <T extends Object> @Nullable T unwrap(Class<T> clazz) {
-    if (clazz == SqlSplittableAggFunction.class) {
+    if (clazz.isInstance(SqlSplittableAggFunction.SelfSplitter.INSTANCE)) {
       return clazz.cast(SqlSplittableAggFunction.SelfSplitter.INSTANCE);
     }
     return super.unwrap(clazz);
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
index a186fe4..18973d2 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSingleValueAggFunction.java
@@ -82,7 +82,7 @@
   }
 
   @Override public <T extends Object> @Nullable T unwrap(Class<T> clazz) {
-    if (clazz == SqlSplittableAggFunction.class) {
+    if (clazz.isInstance(SqlSplittableAggFunction.SelfSplitter.INSTANCE)) {
       return clazz.cast(SqlSplittableAggFunction.SelfSplitter.INSTANCE);
     }
     return super.unwrap(clazz);
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
index 5c5c53c..ba77d81 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumAggFunction.java
@@ -80,7 +80,7 @@
   }
 
   @Override public <T extends Object> @Nullable T unwrap(Class<T> clazz) {
-    if (clazz == SqlSplittableAggFunction.class) {
+    if (clazz.isInstance(SqlSplittableAggFunction.SumSplitter.INSTANCE)) {
       return clazz.cast(SqlSplittableAggFunction.SumSplitter.INSTANCE);
     }
     return super.unwrap(clazz);
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
index 089d91e..f3dbf54 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSumEmptyIsZeroAggFunction.java
@@ -71,7 +71,7 @@
   }
 
   @Override public <T extends Object> @Nullable T unwrap(Class<T> clazz) {
-    if (clazz == SqlSplittableAggFunction.class) {
+    if (clazz.isInstance(SqlSplittableAggFunction.Sum0Splitter.INSTANCE)) {
       return clazz.cast(SqlSplittableAggFunction.Sum0Splitter.INSTANCE);
     }
     return super.unwrap(clazz);
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 63b78f9..f136050 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -217,6 +217,7 @@
 
 import static org.apache.calcite.linq4j.Nullness.castNonNull;
 import static org.apache.calcite.runtime.FlatLists.append;
+import static org.apache.calcite.sql.SqlUtil.containsIn;
 import static org.apache.calcite.sql.SqlUtil.stripAs;
 
 import static java.util.Objects.requireNonNull;
@@ -959,31 +960,6 @@
   }
 
   /**
-   * Returns whether a given node contains a {@link SqlInOperator}.
-   *
-   * @param node a RexNode tree
-   */
-  private static boolean containsInOperator(
-      SqlNode node) {
-    try {
-      SqlVisitor<Void> visitor =
-          new SqlBasicVisitor<Void>() {
-            @Override public Void visit(SqlCall call) {
-              if (call.getOperator() instanceof SqlInOperator) {
-                throw new Util.FoundOne(call);
-              }
-              return super.visit(call);
-            }
-          };
-      node.accept(visitor);
-      return false;
-    } catch (Util.FoundOne e) {
-      Util.swallow(e, null);
-      return true;
-    }
-  }
-
-  /**
    * Push down all the NOT logical operators into any IN/NOT IN operators.
    *
    * @param scope Scope where {@code sqlNode} occurs
@@ -992,7 +968,7 @@
    */
   private static SqlNode pushDownNotForIn(SqlValidatorScope scope,
       SqlNode sqlNode) {
-    if (!(sqlNode instanceof SqlCall) || !containsInOperator(sqlNode)) {
+    if (!(sqlNode instanceof SqlCall) || !containsIn(sqlNode)) {
       return sqlNode;
     }
     final SqlCall sqlCall = (SqlCall) sqlNode;
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 47a35b8..f5fa384 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -62,6 +62,7 @@
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.metadata.RelColumnMapping;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.rules.AggregateRemoveRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -88,9 +89,11 @@
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlStaticAggFunction;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlWindow;
 import org.apache.calcite.sql.fun.SqlCountAggFunction;
+import org.apache.calcite.sql.fun.SqlInternalOperators;
 import org.apache.calcite.sql.fun.SqlLikeOperator;
 import org.apache.calcite.sql.fun.SqlQuantifyOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
@@ -152,6 +155,7 @@
 import java.util.stream.StreamSupport;
 
 import static org.apache.calcite.linq4j.Nullness.castNonNull;
+import static org.apache.calcite.rel.rules.AggregateRemoveRule.canFlattenStatic;
 import static org.apache.calcite.sql.SqlKind.UNION;
 import static org.apache.calcite.util.Static.RESOURCE;
 
@@ -1183,6 +1187,10 @@
     }
   }
 
+  private RexNode aliasMaybe(RexNode node, @Nullable String name) {
+    return name == null ? node : alias(node, name);
+  }
+
   /** Converts a sort expression to descending. */
   public RexNode desc(RexNode node) {
     return call(SqlStdOperatorTable.DESC, node);
@@ -1533,6 +1541,12 @@
         ImmutableList.of(operand));
   }
 
+  /** Creates a call to the {@code LITERAL_AGG} aggregate function. */
+  public AggCall literalAgg(@Nullable Object value) {
+    return aggregateCall(SqlInternalOperators.LITERAL_AGG)
+        .preOperands(literal(value));
+  }
+
   // Methods for patterns
 
   /**
@@ -2023,9 +2037,7 @@
 
     // Simplify expressions.
     if (config.simplify()) {
-      for (int i = 0; i < nodeList.size(); i++) {
-        nodeList.set(i, simplifier.simplifyPreservingType(nodeList.get(i)));
-      }
+      nodeList.replaceAll(e -> simplifier.simplifyPreservingType(e));
     }
 
     // Replace null names with generated aliases.
@@ -2286,69 +2298,78 @@
   /** Creates an {@link Aggregate} that makes the
    * relational expression distinct on all fields. */
   public RelBuilder distinct() {
-    return aggregate(groupKey(fields()));
+    return aggregate_((GroupKeyImpl) groupKey(fields()), ImmutableList.of());
   }
 
   /** Creates an {@link Aggregate} with an array of
    * calls. */
+  @SuppressWarnings({"unchecked", "rawtypes"})
   public RelBuilder aggregate(GroupKey groupKey, AggCall... aggCalls) {
-    return aggregate(groupKey, ImmutableList.copyOf(aggCalls));
+    return aggregate_((GroupKeyImpl) groupKey,
+        (ImmutableList) ImmutableList.copyOf(aggCalls));
   }
 
   /** Creates an {@link Aggregate} with an array of
    * {@link AggregateCall}s. */
-  public RelBuilder aggregate(GroupKey groupKey, List<AggregateCall> aggregateCalls) {
-    return aggregate(groupKey,
+  public RelBuilder aggregate(GroupKey groupKey,
+      List<AggregateCall> aggregateCalls) {
+    return aggregate_((GroupKeyImpl) groupKey,
         aggregateCalls.stream()
             .map(aggregateCall ->
                 new AggCallImpl2(aggregateCall,
                     aggregateCall.getArgList().stream()
                         .map(this::field)
                         .collect(Util.toImmutableList())))
-            .collect(Collectors.toList()));
+            .collect(Util.toImmutableList()));
   }
 
   /** Creates an {@link Aggregate} with multiple calls. */
-  public RelBuilder aggregate(GroupKey groupKey, Iterable<AggCall> aggCalls) {
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  public RelBuilder aggregate(GroupKey groupKey,
+      Iterable<? extends AggCall> aggCalls) {
+    return aggregate_((GroupKeyImpl) groupKey,
+        ImmutableList.<AggCallPlus>copyOf((Iterable) aggCalls));
+  }
+
+  /** Creates an {@link Aggregate} with multiple calls. */
+  private RelBuilder aggregate_(GroupKeyImpl groupKey,
+      final ImmutableList<AggCallPlus> aggCalls) {
+    if (groupKey.nodes.isEmpty()
+        && aggCalls.isEmpty()
+        && config.pruneInputOfAggregate()) {
+      // Query is "SELECT /* no fields */ FROM t GROUP BY ()", which always
+      // returns one row with zero columns.
+      if (config.preventEmptyFieldList()) {
+        // Convert to "VALUES ROW(true)".
+        return values(new String[] {"dummy"}, true);
+      } else {
+        // Convert to "VALUES ROW()".
+        return values(ImmutableList.of(ImmutableList.of()),
+            getTypeFactory().builder().build());
+      }
+    }
     final Registrar registrar =
         new Registrar(fields(), peek().getRowType().getFieldNames());
-    final GroupKeyImpl groupKey_ = (GroupKeyImpl) groupKey;
-    ImmutableBitSet groupSet =
-        ImmutableBitSet.of(registrar.registerExpressions(groupKey_.nodes));
-  label:
-    if (Iterables.isEmpty(aggCalls)) {
-      final RelMetadataQuery mq = peek().getCluster().getMetadataQuery();
-      if (groupSet.isEmpty()) {
-        final Double minRowCount = mq.getMinRowCount(peek());
-        if (minRowCount == null || minRowCount < 1D) {
-          // We can't remove "GROUP BY ()" if there's a chance the rel could be
-          // empty.
-          break label;
-        }
-      }
-      if (registrar.extraNodes.size() == fields().size()) {
-        final Boolean unique = mq.areColumnsUnique(peek(), groupSet);
-        if (unique != null && unique
-            && !config.aggregateUnique()
-            && groupKey_.isSimple()) {
-          // Rel is already unique.
-          return project(fields(groupSet));
-        }
-      }
-      final Double maxRowCount = mq.getMaxRowCount(peek());
-      if (maxRowCount != null && maxRowCount <= 1D
-          && !config.aggregateUnique()
-          && groupKey_.isSimple()) {
-        // If there is at most one row, rel is already unique.
-        return project(fields(groupSet));
-      }
+    final ImmutableBitSet groupSet =
+        ImmutableBitSet.of(registrar.registerExpressions(groupKey.nodes));
+    if (alreadyUnique(aggCalls, groupKey, groupSet, registrar.extraNodes)) {
+      final List<RexNode> nodes = new ArrayList<>(fields(groupSet));
+      aggCalls.forEach(c -> {
+        final AggregateCall call = c.aggregateCall();
+        final SqlStaticAggFunction staticFun =
+            call.getAggregation().unwrapOrThrow(SqlStaticAggFunction.class);
+        final RexNode node =
+            staticFun.constant(getRexBuilder(), groupSet, ImmutableList.of(), call);
+        nodes.add(aliasMaybe(requireNonNull(node, "node"), call.getName()));
+      });
+      return project(nodes);
     }
 
     ImmutableList<ImmutableBitSet> groupSets;
-    if (groupKey_.nodeLists != null) {
+    if (groupKey.nodeLists != null) {
       final int sizeBefore = registrar.extraNodes.size();
       final List<ImmutableBitSet> groupSetList = new ArrayList<>();
-      for (ImmutableList<RexNode> nodeList : groupKey_.nodeLists) {
+      for (ImmutableList<RexNode> nodeList : groupKey.nodeLists) {
         final ImmutableBitSet groupSet2 =
             ImmutableBitSet.of(registrar.registerExpressions(nodeList));
         if (!groupSet.contains(groupSet2)) {
@@ -2360,10 +2381,10 @@
       final ImmutableSortedMultiset<ImmutableBitSet> groupSetMultiset =
           ImmutableSortedMultiset.copyOf(ImmutableBitSet.COMPARATOR,
               groupSetList);
-      if (Iterables.any(aggCalls, RelBuilder::isGroupId)
+      if (aggCalls.stream().anyMatch(RelBuilder::isGroupId)
           || !ImmutableBitSet.ORDERING.isStrictlyOrdered(groupSetMultiset)) {
         return rewriteAggregateWithDuplicateGroupSets(groupSet, groupSetMultiset,
-            ImmutableList.copyOf(aggCalls));
+            aggCalls);
       }
       groupSets = ImmutableList.copyOf(groupSetMultiset.elementSet());
       if (registrar.extraNodes.size() > sizeBefore) {
@@ -2375,17 +2396,14 @@
       groupSets = ImmutableList.of(groupSet);
     }
 
-    for (AggCall aggCall : aggCalls) {
-      ((AggCallPlus) aggCall).register(registrar);
-    }
+    aggCalls.forEach(aggCall -> aggCall.register(registrar));
     project(registrar.extraNodes);
     rename(registrar.names);
     final Frame frame = stack.pop();
     RelNode r = frame.rel;
     final List<AggregateCall> aggregateCalls = new ArrayList<>();
-    for (AggCall aggCall : aggCalls) {
-      aggregateCalls.add(
-          ((AggCallPlus) aggCall).aggregateCall(registrar, groupSet, r));
+    for (AggCallPlus aggCall : aggCalls) {
+      aggregateCalls.add(aggCall.aggregateCall(registrar, groupSet, r));
     }
 
     assert ImmutableBitSet.ORDERING.isStrictlyOrdered(groupSets) : groupSets;
@@ -2394,6 +2412,8 @@
     }
 
     List<Field> inFields = frame.fields;
+    final ImmutableBitSet groupSet2;
+    final ImmutableList<ImmutableBitSet> groupSets2;
     if (config.pruneInputOfAggregate()
         && r instanceof Project) {
       final Set<Integer> fieldsUsed =
@@ -2402,6 +2422,8 @@
       // pretend that one field is used.
       if (fieldsUsed.isEmpty()) {
         r = ((Project) r).getInput();
+        groupSet2 = groupSet;
+        groupSets2 = groupSets;
       } else if (fieldsUsed.size() < r.getRowType().getFieldCount()) {
         // Some fields are computed but not used. Prune them.
         final Map<Integer, Integer> map = new HashMap<>();
@@ -2409,8 +2431,8 @@
           map.put(source, map.size());
         }
 
-        groupSet = groupSet.permute(map);
-        groupSets =
+        groupSet2 = groupSet.permute(map);
+        groupSets2 =
             ImmutableBitSet.ORDERING.immutableSortedCopy(
                 ImmutableBitSet.permute(groupSets, map));
 
@@ -2433,12 +2455,20 @@
           newProjects.add(project.getProjects().get(i));
           builder.add(project.getRowType().getFieldList().get(i));
         }
-        r = project.copy(cluster.traitSet(), project.getInput(), newProjects, builder.build());
+        r =
+            project.copy(cluster.traitSet(), project.getInput(), newProjects,
+                builder.build());
+      } else {
+        groupSet2 = groupSet;
+        groupSets2 = groupSets;
       }
+    } else {
+      groupSet2 = groupSet;
+      groupSets2 = groupSets;
     }
 
     if (!config.dedupAggregateCalls() || Util.isDistinct(aggregateCalls)) {
-      return aggregate_(groupSet, groupSets, r, aggregateCalls,
+      return aggregate_(groupSet2, groupSets2, r, aggregateCalls,
           registrar.extraNodes, inFields);
     }
 
@@ -2463,12 +2493,52 @@
     aggregate_(groupSet, groupSets, r, distinctAggregateCalls,
         registrar.extraNodes, inFields);
     final List<RexNode> fields = projects.stream()
-        .map(p -> p.right == null ? field(p.left)
-            : alias(field(p.left), p.right))
+        .map(p -> aliasMaybe(field(p.left), p.right))
         .collect(Collectors.toList());
     return project(fields);
   }
 
+  /** Returns whether an input is already unique, and therefore a Project
+   * can be created instead of an Aggregate.
+   *
+   * <p>{@link AggregateRemoveRule} does something similar, but also handles
+   * {@link org.apache.calcite.sql.SqlSingletonAggFunction} calls. */
+  private boolean alreadyUnique(List<AggCallPlus> aggCallList,
+      GroupKeyImpl groupKey, ImmutableBitSet groupSet,
+      List<RexNode> extraNodes) {
+    final RelMetadataQuery mq = peek().getCluster().getMetadataQuery();
+    if (aggCallList.isEmpty() && groupSet.isEmpty()) {
+      final Double minRowCount = mq.getMinRowCount(peek());
+      if (minRowCount == null || minRowCount < 1d) {
+        // We can't remove "GROUP BY ()" if there's a chance the rel could be
+        // empty.
+        return false;
+      }
+    }
+
+    // If there are aggregate functions, we must be able to flatten them
+    if (!aggCallList.stream()
+        .allMatch(c -> canFlattenStatic(c.aggregateCall()))) {
+      return false;
+    }
+
+    if (extraNodes.size() == fields().size()) {
+      final Boolean unique = mq.areColumnsUnique(peek(), groupSet);
+      if (unique != null && unique
+          && !config.aggregateUnique()
+          && groupKey.isSimple()) {
+        // Rel is already unique.
+        return true;
+      }
+    }
+
+    // If there is at most one row, rel is already unique.
+    final Double maxRowCount = mq.getMaxRowCount(peek());
+    return maxRowCount != null && maxRowCount <= 1D
+        && !config.aggregateUnique()
+        && groupKey.isSimple();
+  }
+
   /** Finishes the implementation of {@link #aggregate} by creating an
    * {@link Aggregate} and pushing it onto the stack. */
   private RelBuilder aggregate_(ImmutableBitSet groupSet,
@@ -2540,11 +2610,11 @@
   private RelBuilder rewriteAggregateWithDuplicateGroupSets(
       ImmutableBitSet groupSet,
       ImmutableSortedMultiset<ImmutableBitSet> groupSets,
-      List<AggCall> aggregateCalls) {
+      List<AggCallPlus> aggregateCalls) {
     final List<String> fieldNamesIfNoRewrite =
         Aggregate.deriveRowType(getTypeFactory(), peek().getRowType(), false,
             groupSet, groupSets.asList(),
-            aggregateCalls.stream().map(c -> ((AggCallPlus) c).aggregateCall())
+            aggregateCalls.stream().map(AggCallPlus::aggregateCall)
                 .collect(Util.toImmutableList())).getFieldNames();
 
     // If n duplicates exist for a particular grouping, the {@code GROUP_ID()}
@@ -3061,7 +3131,10 @@
    * {@code fieldNames}, or an integer multiple if you wish to create multiple
    * rows.
    *
-   * <p>If there are zero rows, or if all values of a any column are
+   * <p>The {@code fieldNames} array must not be null or empty, but may contain
+   * null values.
+   *
+   * <p>If there are zero rows, or if all values of any column are
    * null, this method cannot deduce the type of columns. For these cases,
    * call {@link #values(Iterable, RelDataType)}.
    *
@@ -3069,8 +3142,8 @@
    * @param values Values
    */
   public RelBuilder values(@Nullable String[] fieldNames, @Nullable Object... values) {
-    if (fieldNames == null
-        || fieldNames.length == 0
+    requireNonNull(fieldNames, "fieldNames");
+    if (fieldNames.length == 0
         || values.length % fieldNames.length != 0
         || values.length < fieldNames.length) {
       throw new IllegalArgumentException(
@@ -4051,7 +4124,7 @@
       final RelDataType type =
           getTypeFactory().createSqlType(SqlTypeName.BOOLEAN);
       return AggregateCall.create(aggFunction, distinct, approximate,
-          ignoreNulls, ImmutableList.of(), ImmutableList.of(), -1,
+          ignoreNulls, preOperands, ImmutableList.of(), -1,
           null, collation, type, alias);
     }
 
@@ -4489,7 +4562,7 @@
           .makeOver(type, op, operands, partitionKeys, sortKeys,
               lowerBound, upperBound, rows, allowPartial, nullWhenCountZero,
               distinct, ignoreNulls);
-      return alias == null ? over : alias(over, alias);
+      return aliasMaybe(over, alias);
     }
   }
 
@@ -4723,6 +4796,15 @@
     /** Sets {@link #pruneInputOfAggregate}. */
     Config withPruneInputOfAggregate(boolean pruneInputOfAggregate);
 
+    /** Whether to ensure that relational operators always have at least one
+     * column. */
+    @Value.Default default boolean preventEmptyFieldList() {
+      return true;
+    }
+
+    /** Sets {@link #preventEmptyFieldList()}. */
+    Config withPreventEmptyFieldList(boolean preventEmptyFieldList);
+
     /** Whether to push down join conditions; default false (but
      * {@link SqlToRelConverter#config()} by default sets this to true). */
     @Value.Default default boolean pushJoinCondition() {
diff --git a/core/src/main/java/org/apache/calcite/util/mapping/MappingType.java b/core/src/main/java/org/apache/calcite/util/mapping/MappingType.java
index 3235192..a20608b 100644
--- a/core/src/main/java/org/apache/calcite/util/mapping/MappingType.java
+++ b/core/src/main/java/org/apache/calcite/util/mapping/MappingType.java
@@ -92,7 +92,7 @@
    * An inverse function has a source for every target, but a source might
    * have 0, 1 or more targets.
    *
-   * <p>Obeys the constaints {@link MappingType#isMandatorySource()},
+   * <p>Obeys the constraints {@link MappingType#isMandatorySource()},
    * {@link MappingType#isSingleSource()}.
    *
    * <p>Similar types:
diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
index d0a55e0..3c2d2ad 100644
--- a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
@@ -462,19 +462,31 @@
 
   @Test void testSelectQueryWithGroupByEmpty2() {
     final String query = "select 42 as c from \"product\" group by ()";
-    final String expected = "SELECT 42 AS \"C\"\n"
-        + "FROM \"foodmart\".\"product\"\n"
-        + "GROUP BY ()";
-    final String expectedMysql = "SELECT 42 AS `C`\n"
-        + "FROM `foodmart`.`product`\n"
-        + "GROUP BY ()";
-    final String expectedPresto = "SELECT 42 AS \"C\"\n"
-        + "FROM \"foodmart\".\"product\"\n"
-        + "GROUP BY ()";
+    final String expected = "SELECT *\n"
+        + "FROM (VALUES (42)) AS \"t\" (\"C\")";
+    final String expectedMysql = "SELECT 42 AS `C`";
     sql(query)
         .ok(expected)
         .withMysql().ok(expectedMysql)
-        .withPresto().ok(expectedPresto);
+        .withPresto().ok(expected);
+  }
+
+  @Test void testSelectLiteralAgg() {
+    final Function<RelBuilder, RelNode> relFn = b -> b
+        .scan("EMP")
+        .aggregate(b.groupKey("DEPTNO"),
+            b.literalAgg(2).as("two"))
+        .build();
+    final String expected = "SELECT \"DEPTNO\", 2 AS \"two\"\n"
+        + "FROM \"scott\".\"EMP\"\n"
+        + "GROUP BY \"DEPTNO\"";
+    final String expectedMysql = "SELECT `DEPTNO`, 2 AS `two`\n"
+        + "FROM `scott`.`EMP`\n"
+        + "GROUP BY `DEPTNO`";
+    relFn(relFn)
+        .ok(expected)
+        .withMysql().ok(expectedMysql)
+        .withPresto().ok(expected);
   }
 
   /** Test case for
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 31c3afe..727fe5e 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -124,6 +124,7 @@
 import static org.hamcrest.CoreMatchers.allOf;
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.notNullValue;
 import static org.hamcrest.CoreMatchers.nullValue;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertNull;
@@ -1963,19 +1964,31 @@
             .project()
             .distinct()
             .build();
-    final String expected = "LogicalAggregate(group=[{}])\n"
-        + "  LogicalFilter(condition=[IS NULL($6)])\n"
-        + "    LogicalTableScan(table=[[scott, EMP]])\n";
-    assertThat(f.apply(createBuilder()), hasTree(expected));
+    final String expected = ""
+        + "LogicalValues(tuples=[[{ true }]])\n";
+    final RelNode r = f.apply(createBuilder());
+    assertThat(r, hasTree(expected));
 
-    // now without pruning
+    // Now without adding extra fields
+    final String expected2 = ""
+        + "LogicalValues(tuples=[[{  }]])\n";
+    final RelNode r2 =
+        f.apply(createBuilder(c -> c.withPreventEmptyFieldList(false)));
+    assertThat(r2, hasTree(expected2));
+
+    // Now without pruning
     // (The empty LogicalProject is dubious, but it's what we've always done)
-    final String expected2 = "LogicalAggregate(group=[{}])\n"
+    final String expected3 = ""
+        + "LogicalAggregate(group=[{}])\n"
         + "  LogicalProject\n"
         + "    LogicalFilter(condition=[IS NULL($6)])\n"
         + "      LogicalTableScan(table=[[scott, EMP]])\n";
-    assertThat(f.apply(createBuilder(c -> c.withPruneInputOfAggregate(false))),
-        hasTree(expected2));
+    final RelNode r3 =
+        f.apply(
+            createBuilder(c ->
+                c.withPruneInputOfAggregate(false)
+                    .withPreventEmptyFieldList(false)));
+    assertThat(r3, hasTree(expected3));
   }
 
   @Test void testUnion() {
@@ -3355,14 +3368,21 @@
   }
 
   @Test void testValuesBadNullFieldNames() {
-    try {
-      final RelBuilder builder = RelBuilder.create(config().build());
-      RelBuilder root = builder.values((String[]) null, "a", "b");
-      fail("expected error, got " + root);
-    } catch (IllegalArgumentException e) {
-      assertThat(e.getMessage(),
-          is("Value count must be a positive multiple of field count"));
-    }
+    final RelBuilder builder = RelBuilder.create(config().build());
+    assertThrows(NullPointerException.class,
+        () -> builder.values((String[]) null, "a", "b"),
+        "fieldNames");
+
+    final String[] f1 = {"x"};
+    assertThat(builder.values(f1, "a", "b", "c", "d"), notNullValue());
+
+    final String[] f2 = {"x", "y"};
+    assertThat(builder.values(f2, "a", "b", "c", "d"), notNullValue());
+
+    final String[] f3 = {"x", "y", "z"};
+    assertThrows(IllegalArgumentException.class,
+        () -> builder.values(f3, "a", "b", "c", "d"),
+        "Value count must be a positive multiple of field count");
   }
 
   @Test void testValuesBadNoFields() {
diff --git a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
index eb2ef3b..3d71c63 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -101,7 +101,6 @@
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.test.catalog.MockCatalogReaderSimple;
-import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.Holder;
@@ -128,6 +127,7 @@
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.locks.ReentrantLock;
 
@@ -146,6 +146,7 @@
 import static org.hamcrest.CoreMatchers.nullValue;
 import static org.hamcrest.CoreMatchers.startsWith;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.object.HasToString.hasToString;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
@@ -1462,7 +1463,7 @@
     assertThat(colType(mq, input, 0), equalTo("DEPTNO-agg"));
 
     // There is no caching. Another request causes another call to the provider.
-    assertThat(buf.toString(), equalTo("[DEPTNO-rel, EXPR$1-rel, DEPTNO-agg]"));
+    assertThat(buf, hasToString("[DEPTNO-rel, EXPR$1-rel, DEPTNO-agg]"));
     assertThat(buf.size(), equalTo(3));
     assertThat(colType(mq, input, 0), equalTo("DEPTNO-agg"));
     assertThat(buf.size(), equalTo(4));
@@ -3229,8 +3230,8 @@
   /** Tests calling {@link RelMetadataQuery#getTableOrigin} for
    * an aggregate with no columns. Previously threw. */
   @Test void testEmptyAggregateTableOrigin() {
-    final FrameworkConfig config = RelBuilderTest.config().build();
-    final RelBuilder builder = RelBuilder.create(config);
+    final RelBuilder builder =
+        RelBuilderTest.createBuilder(b -> b.withPreventEmptyFieldList(false));
     RelMetadataQuery mq = builder.getCluster().getMetadataQuery();
     RelNode agg = builder
         .scan("EMP")
@@ -3241,8 +3242,7 @@
   }
 
   @Test void testGetPredicatesForJoin() {
-    final FrameworkConfig config = RelBuilderTest.config().build();
-    final RelBuilder builder = RelBuilder.create(config);
+    final RelBuilder builder = RelBuilderTest.createBuilder();
     RelNode join = builder
         .scan("EMP")
         .scan("DEPT")
@@ -3266,9 +3266,8 @@
         is("=($0, $8)"));
   }
 
-  @Test void testGetPredicatesForFilter() throws Exception {
-    final FrameworkConfig config = RelBuilderTest.config().build();
-    final RelBuilder builder = RelBuilder.create(config);
+  @Test void testGetPredicatesForFilter() {
+    final RelBuilder builder = RelBuilderTest.createBuilder();
     RelNode filter = builder
         .scan("EMP")
         .filter(builder.call(NONDETERMINISTIC_OP))
@@ -3290,6 +3289,28 @@
         is("=($0, $1)"));
   }
 
+  @Test void testGetPredicatesForLiteralAgg() {
+    final RelBuilder b = RelBuilderTest.createBuilder();
+    RelNode r = b
+        .scan("EMP")
+        .aggregate(b.groupKey("DEPTNO"),
+            b.literalAgg(42),
+            b.literalAgg(null))
+        .build();
+    RelMetadataQuery mq = r.getCluster().getMetadataQuery();
+    final RelOptPredicateList predicateList = mq.getPulledUpPredicates(r);
+    assertThat(predicateList.pulledUpPredicates,
+        hasToString("[=($1, 42), IS NULL($2)]"));
+    assertThat(toSortedStringList(predicateList.constantMap),
+        hasToString("[$1=42, $2=null:NULL]"));
+  }
+
+  /** Converts a Map to a sorted list of its entries. */
+  static <K, V> List<String> toSortedStringList(Map<K, V> map) {
+    return map.entrySet().stream().map(Object::toString)
+        .sorted().collect(Util.toImmutableList());
+  }
+
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-4315">[CALCITE-4315]
    * NPE in RelMdUtil#checkInputForCollationAndLimit</a>. */
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 98f73d5..6bbedf8 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -5644,9 +5644,11 @@
    * empno is unique and all aggregate functions are splittable.
    */
   @Test void testAggregateRemove1() {
-    final String sql = "select empno, sum(sal), min(sal), max(sal), "
-        + "bit_and(distinct sal), bit_or(sal), count(distinct sal) "
-        + "from sales.emp group by empno, deptno\n";
+    final String sql = "select empno, sum(sal), min(sal), max(sal),\n"
+        + " bit_and(distinct sal), bit_or(sal), count(distinct sal),\n"
+        + " grouping(deptno)\n"
+        + "from sales.emp\n"
+        + "group by empno, deptno";
     sql(sql)
         .withRule(CoreRules.AGGREGATE_REMOVE,
             CoreRules.PROJECT_MERGE)
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 546b222..95b6274 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -426,6 +426,11 @@
     sql("select sum(deptno) from emp group by ()").ok();
   }
 
+  @Test void testGroupEmptyYieldLiteral() {
+    // Expected plan is "VALUES 42". The result is one row even if EMP is empty.
+    sql("select 42 from emp group by ()").ok();
+  }
+
   // Same effect as writing "GROUP BY deptno"
   @Test void testSingletonGroupingSet() {
     sql("select sum(sal) from emp group by grouping sets (deptno)").ok();
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 a8335f3..291b9ce 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -882,20 +882,23 @@
   </TestCase>
   <TestCase name="testAggregateRemove1">
     <Resource name="sql">
-      <![CDATA[select empno, sum(sal), min(sal), max(sal), bit_and(distinct sal), bit_or(sal), count(distinct sal) from sales.emp group by empno, deptno
-]]>
+      <![CDATA[select empno, sum(sal), min(sal), max(sal),
+ bit_and(distinct sal), bit_or(sal), count(distinct sal),
+ grouping(deptno)
+from sales.emp
+group by empno, deptno]]>
     </Resource>
     <Resource name="planBefore">
       <![CDATA[
-LogicalProject(EMPNO=[$0], EXPR$1=[$2], EXPR$2=[$3], EXPR$3=[$4], EXPR$4=[$5], EXPR$5=[$6], EXPR$6=[$7])
-  LogicalAggregate(group=[{0, 1}], EXPR$1=[SUM($2)], EXPR$2=[MIN($2)], EXPR$3=[MAX($2)], EXPR$4=[BIT_AND($2)], EXPR$5=[BIT_OR($2)], EXPR$6=[COUNT(DISTINCT $2)])
+LogicalProject(EMPNO=[$0], EXPR$1=[$2], EXPR$2=[$3], EXPR$3=[$4], EXPR$4=[$5], EXPR$5=[$6], EXPR$6=[$7], EXPR$7=[$8])
+  LogicalAggregate(group=[{0, 1}], EXPR$1=[SUM($2)], EXPR$2=[MIN($2)], EXPR$3=[MAX($2)], EXPR$4=[BIT_AND($2)], EXPR$5=[BIT_OR($2)], EXPR$6=[COUNT(DISTINCT $2)], EXPR$7=[GROUPING($1)])
     LogicalProject(EMPNO=[$0], DEPTNO=[$7], SAL=[$5])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-LogicalProject(EMPNO=[$0], EXPR$1=[$5], EXPR$2=[$5], EXPR$3=[$5], EXPR$4=[$5], EXPR$5=[$5], EXPR$6=[1:BIGINT])
+LogicalProject(EMPNO=[$0], EXPR$1=[$5], EXPR$2=[$5], EXPR$3=[$5], EXPR$4=[$5], EXPR$5=[$5], EXPR$6=[1:BIGINT], EXPR$7=[0:BIGINT])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
     </Resource>
@@ -1326,8 +1329,8 @@
       LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)])
         LogicalProject(MGR=[$3])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-    LogicalAggregate(group=[{0, 1}])
-      LogicalProject(MGR=[$3], i=[true])
+    LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)])
+      LogicalProject(MGR=[$3])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
     </Resource>
@@ -1340,8 +1343,8 @@
       LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)])
         LogicalProject(MGR=[$3])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-    LogicalAggregate(group=[{0, 1}])
-      LogicalProject(MGR=[$3], i=[true])
+    LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)])
+      LogicalProject(MGR=[$3])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
     </Resource>
@@ -3533,8 +3536,8 @@
     LogicalFilter(condition=[OR(IS NOT NULL($10), <($5, 100))])
       LogicalJoin(condition=[=($7, $9)], joinType=[left])
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-        LogicalAggregate(group=[{0, 1}])
-          LogicalProject(DEPTNO=[$7], i=[true])
+        LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)])
+          LogicalProject(DEPTNO=[$7])
             LogicalFilter(condition=[<($0, 20)])
               LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -3579,8 +3582,8 @@
             LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null:INTEGER)])
               LogicalFilter(condition=[<($0, 20)])
                 LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-        LogicalAggregate(group=[{0, 1}])
-          LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null:INTEGER)], i=[true])
+        LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)])
+          LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null:INTEGER)])
             LogicalFilter(condition=[<($0, 20)])
               LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -3957,8 +3960,8 @@
 LogicalProject(EMPNO=[$0], D=[CASE(IS NOT NULL($10), true, false)])
   LogicalJoin(condition=[=($7, $9)], joinType=[left])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-    LogicalAggregate(group=[{0, 1}])
-      LogicalProject(DEPTNO=[$7], i=[true])
+    LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)])
+      LogicalProject(DEPTNO=[$7])
         LogicalFilter(condition=[<($0, 20)])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
@@ -4022,8 +4025,8 @@
           LogicalFilter(condition=[<($0, 20)])
             LogicalProject(EMPNO=[$0], DEPTNO=[CASE(true, CAST($7):INTEGER, null:INTEGER)])
               LogicalTableScan(table=[[CATALOG, SALES, EMP]])
-    LogicalAggregate(group=[{0, 1}])
-      LogicalProject(DEPTNO=[$1], i=[true])
+    LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)])
+      LogicalProject(DEPTNO=[$1])
         LogicalFilter(condition=[<($0, 20)])
           LogicalProject(EMPNO=[$0], DEPTNO=[CASE(true, CAST($7):INTEGER, null:INTEGER)])
             LogicalTableScan(table=[[CATALOG, SALES, EMP]])
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 68aeb48..6902fe5 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -2344,6 +2344,19 @@
 ]]>
     </Resource>
   </TestCase>
+  <TestCase name="testGroupEmptyYieldLiteral">
+    <Resource name="sql">
+      <![CDATA[select 42 from emp group by ()]]>
+    </Resource>
+    <Resource name="plan">
+      <![CDATA[
+LogicalProject(EXPR$0=[42])
+  LogicalAggregate(group=[{}])
+    LogicalProject($f0=[0])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+    </Resource>
+  </TestCase>
   <TestCase name="testGroupExpressionsInsideAndOut">
     <Resource name="plan">
       <![CDATA[
diff --git a/core/src/test/resources/sql/agg.iq b/core/src/test/resources/sql/agg.iq
index f359d06..822a3c4 100644
--- a/core/src/test/resources/sql/agg.iq
+++ b/core/src/test/resources/sql/agg.iq
@@ -996,6 +996,78 @@
 
 !ok
 
+# Degenerate case: GROUP_ID() without GROUPING SETS
+select group_id() as g
+from "scott".emp
+group by ();
++---+
+| G |
++---+
+| 0 |
++---+
+(1 row)
+
+!ok
+
+# GROUP_ID() does not make a query into an aggregate query
+# (maybe it should)
+select group_id() as g
+from "scott".emp;
+GROUP_ID operator may only occur in an aggregate query
+!error
+
+# GROUP_ID() does not make a query into an aggregate query
+select group_id() as g, sum(3) as s3
+from "scott".emp;
++---+----+
+| G | S3 |
++---+----+
+| 0 | 42 |
++---+----+
+(1 row)
+
+!ok
+
+# Extremely degenerate case: GROUP_ID on an empty table
+select group_id() as g, sum(3) as s3
+from "scott".emp
+where empno < 0;
++---+----+
+| G | S3 |
++---+----+
+| 0 |    |
++---+----+
+(1 row)
+
+!ok
+
+# As above, explicit empty GROUP BY
+select group_id() as g
+from "scott".emp
+where empno < 0
+group by ();
++---+
+| G |
++---+
+| 0 |
++---+
+(1 row)
+
+!ok
+
+# As above, non-empty GROUP BY
+select group_id() as g
+from "scott".emp
+where empno < 0
+group by deptno;
++---+
+| G |
++---+
++---+
+(0 rows)
+
+!ok
+
 # From http://rwijk.blogspot.com/2008/12/groupid.html
 select deptno
        , job
@@ -2349,6 +2421,49 @@
 !plan
 !}
 
+# Aggregate query that uses no columns throws AssertionError in
+# RelFieldTrimmer.trimFields
+select 2 as two
+from emp
+group by ();
++-----+
+| TWO |
++-----+
+|   2 |
++-----+
+(1 row)
+
+!ok
+
+# As previous, as a scalar sub-query
+select deptno,
+  (select 2 as two from emp group by ()) as two
+from emp
+group by deptno;
++--------+-----+
+| DEPTNO | TWO |
++--------+-----+
+|     10 |   2 |
+|     20 |   2 |
+|     30 |   2 |
++--------+-----+
+(3 rows)
+
+!ok
+
+# As previous, grand total
+select (select 2 from emp group by ()) as two
+from emp
+group by ();
++-----+
+| TWO |
++-----+
+|   2 |
++-----+
+(1 row)
+
+!ok
+
 !use orinoco
 
 # FLOOR to achieve a 2-hour window
diff --git a/core/src/test/resources/sql/blank.iq b/core/src/test/resources/sql/blank.iq
index b8b125c..c8151c1 100644
--- a/core/src/test/resources/sql/blank.iq
+++ b/core/src/test/resources/sql/blank.iq
@@ -100,10 +100,9 @@
             EnumerableCalc(expr#0..1=[{inputs}], expr#2=[IS NOT NULL($t1)], proj#0..1=[{exprs}], $condition=[$t2])
               EnumerableTableScan(table=[[BLANK, TABLE2]])
     EnumerableSort(sort0=[$1], sort1=[$0], dir0=[ASC], dir1=[ASC])
-      EnumerableCalc(expr#0..1=[{inputs}], expr#2=[true], proj#0..2=[{exprs}])
-        EnumerableAggregate(group=[{0, 1}])
-          EnumerableCalc(expr#0..1=[{inputs}], expr#2=[IS NOT NULL($t1)], expr#3=[IS NOT NULL($t0)], expr#4=[AND($t2, $t3)], proj#0..1=[{exprs}], $condition=[$t4])
-            EnumerableTableScan(table=[[BLANK, TABLE2]])
+      EnumerableAggregate(group=[{0, 1}], i=[LITERAL_AGG(true)])
+        EnumerableCalc(expr#0..1=[{inputs}], expr#2=[IS NOT NULL($t1)], expr#3=[IS NOT NULL($t0)], expr#4=[AND($t2, $t3)], proj#0..1=[{exprs}], $condition=[$t4])
+          EnumerableTableScan(table=[[BLANK, TABLE2]])
 !plan
 +---+---+
 | I | J |
diff --git a/core/src/test/resources/sql/sub-query.iq b/core/src/test/resources/sql/sub-query.iq
index 007e52c..3cabe1b 100644
--- a/core/src/test/resources/sql/sub-query.iq
+++ b/core/src/test/resources/sql/sub-query.iq
@@ -388,8 +388,8 @@
                 EnumerableCalc(expr#0..2=[{inputs}], DEPTNO=[$t0])
                   EnumerableTableScan(table=[[scott, DEPT]])
     EnumerableSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC])
-      EnumerableCalc(expr#0..1=[{inputs}], expr#2=[true], expr#3=[IS NOT NULL($t0)], proj#0..2=[{exprs}], $condition=[$t3])
-        EnumerableAggregate(group=[{1, 3}])
+      EnumerableCalc(expr#0..2=[{inputs}], expr#3=[IS NOT NULL($t0)], proj#0..2=[{exprs}], $condition=[$t3])
+        EnumerableAggregate(group=[{1, 3}], i=[LITERAL_AGG(true)])
           EnumerableNestedLoopJoin(condition=[>($2, $3)], joinType=[inner])
             EnumerableCalc(expr#0..7=[{inputs}], EMPNO=[$t0], JOB=[$t2], DEPTNO=[$t7])
               EnumerableTableScan(table=[[scott, EMP]])
diff --git a/site/_docs/algebra.md b/site/_docs/algebra.md
index d11c69a..552a139 100644
--- a/site/_docs/algebra.md
+++ b/site/_docs/algebra.md
@@ -506,9 +506,10 @@
 | `aggregateCall(op, expr...)`<br/>`aggregateCall(op, exprList)` | Creates a call to a given aggregate function
 | `count([ distinct, alias, ] expr...)`<br/>`count([ distinct, alias, ] exprList)` | Creates a call to the `COUNT` aggregate function
 | `countStar(alias)` | Creates a call to the `COUNT(*)` aggregate function
-| `sum([ distinct, alias, ] expr)` | Creates a call to the `SUM` aggregate function
-| `min([ alias, ] expr)` | Creates a call to the `MIN` aggregate function
+| `literalAgg(value)` | Creates a call to an aggregate function that always evaluates to *value*
 | `max([ alias, ] expr)` | Creates a call to the `MAX` aggregate function
+| `min([ alias, ] expr)` | Creates a call to the `MIN` aggregate function
+| `sum([ distinct, alias, ] expr)` | Creates a call to the `SUM` aggregate function
 
 To further modify the `AggCall`, call its methods: