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 2017/08/11 16:36:21 UTC

[1/2] calcite git commit: [CALCITE-1936] Allow ROUND() and TRUNCATE() to take one operand, defaulting scale to 0

Repository: calcite
Updated Branches:
  refs/heads/master 54ed57f82 -> 7321c8708


[CALCITE-1936] Allow ROUND() and TRUNCATE() to take one operand, defaulting scale to 0

Also, make ROUND and TRUNCATE nullable if their 2nd operand is nullable.

Close apache/calcite#513


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/bfaea7cc
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/bfaea7cc
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/bfaea7cc

Branch: refs/heads/master
Commit: bfaea7ccfd33e9a4b5fee1617f56f2ccdd78db08
Parents: 54ed57f
Author: Minji Kim <mi...@dremio.com>
Authored: Thu Jan 12 09:03:24 2017 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Aug 11 08:02:24 2017 -0700

----------------------------------------------------------------------
 .../apache/calcite/runtime/SqlFunctions.java    | 38 +++++++++++++++++++-
 .../calcite/sql/fun/SqlStdOperatorTable.java    |  8 ++---
 .../apache/calcite/sql/type/OperandTypes.java   | 11 ++++++
 .../calcite/sql/test/SqlOperatorBaseTest.java   | 38 ++++++++++++++++++++
 site/_docs/reference.md                         |  4 +--
 5 files changed, 92 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/bfaea7cc/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index fec73c6..c52abfa 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -1197,45 +1197,81 @@ public class SqlFunctions {
   }
 
   // SQL ROUND
-  /** SQL <code>ROUND</code> operator applied to long values. */
+  /** SQL <code>ROUND</code> operator applied to int values. */
+  public static int sround(int b0) {
+    return sround(b0, 0);
+  }
+
+  /** SQL <code>ROUND</code> operator applied to int values. */
   public static int sround(int b0, int b1) {
     return sround(BigDecimal.valueOf(b0), b1).intValue();
   }
 
   /** SQL <code>ROUND</code> operator applied to long values. */
+  public static long sround(long b0) {
+    return sround(b0, 0);
+  }
+
+  /** SQL <code>ROUND</code> operator applied to long values. */
   public static long sround(long b0, int b1) {
     return sround(BigDecimal.valueOf(b0), b1).longValue();
   }
 
   /** SQL <code>ROUND</code> operator applied to BigDecimal values. */
+  public static BigDecimal sround(BigDecimal b0) {
+    return sround(b0, 0);
+  }
+
+  /** SQL <code>ROUND</code> operator applied to BigDecimal values. */
   public static BigDecimal sround(BigDecimal b0, int b1) {
     return b0.movePointRight(b1)
         .setScale(0, RoundingMode.HALF_UP).movePointLeft(b1);
   }
 
   /** SQL <code>ROUND</code> operator applied to double values. */
+  public static double sround(double b0) {
+    return sround(b0, 0);
+  }
+
+  /** SQL <code>ROUND</code> operator applied to double values. */
   public static double sround(double b0, int b1) {
     return sround(BigDecimal.valueOf(b0), b1).doubleValue();
   }
 
   // SQL TRUNCATE
   /** SQL <code>TRUNCATE</code> operator applied to int values. */
+  public static int struncate(int b0) {
+    return struncate(b0, 0);
+  }
+
   public static int struncate(int b0, int b1) {
     return struncate(BigDecimal.valueOf(b0), b1).intValue();
   }
 
   /** SQL <code>TRUNCATE</code> operator applied to long values. */
+  public static long struncate(long b0) {
+    return struncate(b0, 0);
+  }
+
   public static long struncate(long b0, int b1) {
     return struncate(BigDecimal.valueOf(b0), b1).longValue();
   }
 
   /** SQL <code>TRUNCATE</code> operator applied to BigDecimal values. */
+  public static BigDecimal struncate(BigDecimal b0) {
+    return struncate(b0, 0);
+  }
+
   public static BigDecimal struncate(BigDecimal b0, int b1) {
     return b0.movePointRight(b1)
         .setScale(0, RoundingMode.DOWN).movePointLeft(b1);
   }
 
   /** SQL <code>TRUNCATE</code> operator applied to double values. */
+  public static double struncate(double b0) {
+    return struncate(b0, 0);
+  }
+
   public static double struncate(double b0, int b1) {
     return struncate(BigDecimal.valueOf(b0), b1).doubleValue();
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/bfaea7cc/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
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 8300add..e4227f7 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
@@ -1442,9 +1442,9 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
       new SqlFunction(
           "ROUND",
           SqlKind.OTHER_FUNCTION,
-          ReturnTypes.ARG0,
+          ReturnTypes.ARG0_NULLABLE,
           null,
-          OperandTypes.NUMERIC_INTEGER,
+          OperandTypes.NUMERIC_OPTIONAL_INTEGER,
           SqlFunctionCategory.NUMERIC);
 
   public static final SqlFunction SIGN =
@@ -1479,9 +1479,9 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
       new SqlFunction(
           "TRUNCATE",
           SqlKind.OTHER_FUNCTION,
-          ReturnTypes.ARG0,
+          ReturnTypes.ARG0_NULLABLE,
           null,
-          OperandTypes.NUMERIC_INTEGER,
+          OperandTypes.NUMERIC_OPTIONAL_INTEGER,
           SqlFunctionCategory.NUMERIC);
 
   public static final SqlFunction PI =

http://git-wip-us.apache.org/repos/asf/calcite/blob/bfaea7cc/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
index 15d3b70..75366d9 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
@@ -18,6 +18,7 @@ package org.apache.calcite.sql.type;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeComparability;
+import org.apache.calcite.runtime.PredicateImpl;
 import org.apache.calcite.sql.SqlCallBinding;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
@@ -200,6 +201,16 @@ public abstract class OperandTypes {
   public static final SqlSingleOperandTypeChecker NUMERIC =
       family(SqlTypeFamily.NUMERIC);
 
+
+  public static final SqlSingleOperandTypeChecker NUMERIC_OPTIONAL_INTEGER =
+      family(ImmutableList.of(SqlTypeFamily.NUMERIC, SqlTypeFamily.INTEGER),
+          // Second operand optional (operand index 0, 1)
+          new PredicateImpl<Integer>() {
+            public boolean test(Integer number) {
+              return number == 1;
+            }
+          });
+
   public static final SqlSingleOperandTypeChecker NUMERIC_INTEGER =
       family(SqlTypeFamily.NUMERIC, SqlTypeFamily.INTEGER);
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/bfaea7cc/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
----------------------------------------------------------------------
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 7925bf6..a54e2d0 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
@@ -1640,6 +1640,8 @@ public abstract class SqlOperatorBaseTest {
     tester.checkScalarApprox("{fn RADIANS(90)}", "DOUBLE NOT NULL", 1.57080, 0.001);
     tester.checkScalarApprox("{fn RAND(42)}", "DOUBLE NOT NULL", 0.63708, 0.001);
     tester.checkScalar("{fn ROUND(1251, -2)}", 1300, "INTEGER NOT NULL");
+    tester.checkFails("^{fn ROUND(1251)}^", "Cannot apply '\\{fn ROUND\\}' to "
+        + "arguments of type '\\{fn ROUND\\}\\(<INTEGER>\\)'.*", false);
     tester.checkScalar("{fn SIGN(-1)}", -1, "INTEGER NOT NULL");
     tester.checkScalarApprox("{fn SIN(0.2)}", "DOUBLE NOT NULL", 0.19867, 0.001);
     tester.checkScalarApprox("{fn SQRT(4.2)}", "DOUBLE NOT NULL", 2.04939, 0.001);
@@ -4388,9 +4390,28 @@ public abstract class SqlOperatorBaseTest {
         "round(cast(42.346 as decimal(2, 3)), 2)",
         BigDecimal.valueOf(4235, 2),
         "DECIMAL(2, 3) NOT NULL");
+    tester.checkScalar(
+        "round(cast(-42.346 as decimal(2, 3)), 2)",
+        BigDecimal.valueOf(-4235, 2),
+        "DECIMAL(2, 3) NOT NULL");
     tester.checkNull("round(cast(null as integer), 1)");
     tester.checkNull("round(cast(null as double), 1)");
+    tester.checkNull("round(43.21, cast(null as integer))");
+
+    tester.checkNull("round(cast(null as double))");
+    tester.checkScalar("round(42)", 42, "INTEGER NOT NULL");
+    tester.checkScalar(
+        "round(cast(42.346 as decimal(2, 3)))",
+        BigDecimal.valueOf(42, 0),
+        "DECIMAL(2, 3) NOT NULL");
+    tester.checkScalar("round(42.324)",
+        BigDecimal.valueOf(42, 0),
+        "DECIMAL(5, 3) NOT NULL");
+    tester.checkScalar("round(42.724)",
+        BigDecimal.valueOf(43, 0),
+        "DECIMAL(5, 3) NOT NULL");
   }
+
   @Test public void testSignFunc() {
     tester.setFor(
         SqlStdOperatorTable.SIGN);
@@ -4487,8 +4508,25 @@ public abstract class SqlOperatorBaseTest {
         "truncate(cast(42.345 as decimal(2, 3)), 2)",
         BigDecimal.valueOf(4234, 2),
         "DECIMAL(2, 3) NOT NULL");
+    tester.checkScalar(
+        "truncate(cast(-42.345 as decimal(2, 3)), 2)",
+        BigDecimal.valueOf(-4234, 2),
+        "DECIMAL(2, 3) NOT NULL");
     tester.checkNull("truncate(cast(null as integer), 1)");
     tester.checkNull("truncate(cast(null as double), 1)");
+    tester.checkNull("truncate(43.21, cast(null as integer))");
+
+    tester.checkScalar("truncate(42)", 42, "INTEGER NOT NULL");
+    tester.checkScalar("truncate(42.324)",
+        BigDecimal.valueOf(42, 0),
+        "DECIMAL(5, 3) NOT NULL");
+    tester.checkScalar("truncate(cast(42.324 as float))", 42F, "FLOAT NOT NULL");
+    tester.checkScalar(
+        "truncate(cast(42.345 as decimal(2, 3)))",
+        BigDecimal.valueOf(42, 0),
+        "DECIMAL(2, 3) NOT NULL");
+    tester.checkNull("truncate(cast(null as integer))");
+    tester.checkNull("truncate(cast(null as double))");
   }
 
   @Test public void testNullifFunc() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/bfaea7cc/site/_docs/reference.md
----------------------------------------------------------------------
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index e911765..dc4f3db 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -1108,11 +1108,11 @@ comp:
 | DEGREES(numeric)          | Converts *numeric* from radians to degrees
 | PI()                      | Returns a value that is closer than any other value to *pi*
 | RADIANS(numeric)          | Converts *numeric* from degrees to radians
-| ROUND(numeric1, numeric2) | Rounds *numeric1* to *numeric2* places right to the decimal point
+| ROUND(numeric1 [, numeric2]) | Rounds *numeric1* to optionally *numeric2* (if not specified 0) places right to the decimal point
 | SIGN(numeric)             | Returns the signum of *numeric*
 | SIN(numeric)              | Returns the sine of *numeric*
 | TAN(numeric)              | Returns the tangent of *numeric*
-| TRUNCATE(numeric1, numeric2) | Truncates *numeric1* to *numeric2* places right to the decimal point
+| TRUNCATE(numeric1 [, numeric2]) | Truncates *numeric1* to optionally *numeric2* (if not specified 0) places right to the decimal point.
 
 ### Character string operators and functions
 


[2/2] calcite git commit: [CALCITE-1930] Fix AggregateExpandDistinctAggregatesRule when there are multiple AggregateCalls referring to the same input

Posted by jh...@apache.org.
[CALCITE-1930] Fix AggregateExpandDistinctAggregatesRule when there are multiple AggregateCalls referring to the same input

Close apache/calcite#510


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/7321c870
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/7321c870
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/7321c870

Branch: refs/heads/master
Commit: 7321c8708bd1273b8cd4aab818bf3abb55afb2a4
Parents: bfaea7c
Author: Minji Kim <mi...@dremio.com>
Authored: Fri Jan 13 17:42:15 2017 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Aug 11 08:03:19 2017 -0700

----------------------------------------------------------------------
 .../AggregateExpandDistinctAggregatesRule.java  | 234 +++++++------------
 .../apache/calcite/test/RelOptRulesTest.java    |  51 ++++
 .../org/apache/calcite/test/RelOptRulesTest.xml | 125 +++++++++-
 core/src/test/resources/sql/agg.iq              |  85 +++++++
 4 files changed, 349 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/7321c870/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
index 7d3abf6..dee822d 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
@@ -33,10 +33,8 @@ import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlAggFunction;
-import org.apache.calcite.sql.fun.SqlCountAggFunction;
-import org.apache.calcite.sql.fun.SqlMinMaxAggFunction;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.fun.SqlSumAggFunction;
 import org.apache.calcite.sql.fun.SqlSumEmptyIsZeroAggFunction;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.tools.RelBuilder;
@@ -54,6 +52,7 @@ import com.google.common.collect.Lists;
 import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -128,34 +127,41 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
       return;
     }
 
-    // Find all of the agg expressions. We use a LinkedHashSet to ensure
-    // determinism.
-    int nonDistinctCount = 0;
-    int distinctCount = 0;
+    // Find all of the agg expressions. We use a LinkedHashSet to ensure determinism.
+    int nonDistinctAggCallCount = 0;  // find all aggregate calls without distinct
     int filterCount = 0;
-    int unsupportedAggCount = 0;
+    int unsupportedNonDistinctAggCallCount = 0;
     final Set<Pair<List<Integer>, Integer>> argLists = new LinkedHashSet<>();
     for (AggregateCall aggCall : aggregate.getAggCallList()) {
       if (aggCall.filterArg >= 0) {
         ++filterCount;
       }
       if (!aggCall.isDistinct()) {
-        ++nonDistinctCount;
-        if (!(aggCall.getAggregation() instanceof SqlCountAggFunction
-              || aggCall.getAggregation() instanceof SqlSumAggFunction
-              || aggCall.getAggregation() instanceof SqlMinMaxAggFunction)) {
-          ++unsupportedAggCount;
+        ++nonDistinctAggCallCount;
+        final SqlKind aggCallKind = aggCall.getAggregation().getKind();
+        // We only support COUNT/SUM/MIN/MAX for the "single" count distinct optimization
+        switch (aggCallKind) {
+        case COUNT:
+        case SUM:
+        case SUM0:
+        case MIN:
+        case MAX:
+          break;
+        default:
+          ++unsupportedNonDistinctAggCallCount;
         }
-        continue;
+      } else {
+        argLists.add(Pair.of(aggCall.getArgList(), aggCall.filterArg));
       }
-      ++distinctCount;
-      argLists.add(Pair.of(aggCall.getArgList(), aggCall.filterArg));
     }
+
+    final int distinctAggCallCount =
+        aggregate.getAggCallList().size() - nonDistinctAggCallCount;
     Preconditions.checkState(argLists.size() > 0, "containsDistinctCall lied");
 
     // If all of the agg expressions are distinct and have the same
     // arguments then we can use a more efficient form.
-    if (nonDistinctCount == 0 && argLists.size() == 1) {
+    if (nonDistinctAggCallCount == 0 && argLists.size() == 1) {
       final Pair<List<Integer>, Integer> pair =
           Iterables.getOnlyElement(argLists);
       final RelBuilder relBuilder = call.builder();
@@ -171,10 +177,10 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
 
     // If only one distinct aggregate and one or more non-distinct aggregates,
     // we can generate multi-phase aggregates
-    if (distinctCount == 1 // one distinct aggregate
+    if (distinctAggCallCount == 1 // one distinct aggregate
         && filterCount == 0 // no filter
-        && unsupportedAggCount == 0 // sum/min/max/count in non-distinct aggregate
-        && nonDistinctCount > 0) { // one or more non-distinct aggregates
+        && unsupportedNonDistinctAggCallCount == 0 // sum/min/max/count in non-distinct aggregate
+        && nonDistinctAggCallCount > 0) { // one or more non-distinct aggregates
       final RelBuilder relBuilder = call.builder();
       convertSingletonDistinct(relBuilder, aggregate, argLists);
       call.transformTo(relBuilder.build());
@@ -245,6 +251,11 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
    */
   private RelBuilder convertSingletonDistinct(RelBuilder relBuilder,
       Aggregate aggregate, Set<Pair<List<Integer>, Integer>> argLists) {
+
+    // In this case, we are assuming that there is a single distinct function.
+    // So make sure that argLists is of size one.
+    Preconditions.checkArgument(argLists.size() == 1);
+
     // For example,
     //    SELECT deptno, COUNT(*), SUM(bonus), MIN(DISTINCT sal)
     //    FROM emp
@@ -259,168 +270,103 @@ public final class AggregateExpandDistinctAggregatesRule extends RelOptRule {
     //          GROUP BY deptno, sal)            // Aggregate B
     //    GROUP BY deptno                        // Aggregate A
     relBuilder.push(aggregate.getInput());
-    final List<Pair<RexNode, String>> projects = new ArrayList<>();
-    final Map<Integer, Integer> sourceOf = new HashMap<>();
-    SortedSet<Integer> newGroupSet = new TreeSet<>();
-    final List<RelDataTypeField> childFields =
-        relBuilder.peek().getRowType().getFieldList();
-    final boolean hasGroupBy = aggregate.getGroupSet().size() > 0;
 
-    final Set<Integer> groupSet = aggregate.getGroupSet().asSet();
+    final List<AggregateCall> originalAggCalls = aggregate.getAggCallList();
+    final ImmutableBitSet originalGroupSet = aggregate.getGroupSet();
 
     // Add the distinct aggregate column(s) to the group-by columns,
     // if not already a part of the group-by
-    newGroupSet.addAll(aggregate.getGroupSet().asList());
-    for (Pair<List<Integer>, Integer> argList : argLists) {
-      newGroupSet.addAll(argList.getKey());
-    }
-
-    // Re-map the arguments to the aggregate A. These arguments will get
-    // remapped because of the intermediate aggregate B generated as part of the
-    // transformation.
-    for (int arg : newGroupSet) {
-      sourceOf.put(arg, projects.size());
-      projects.add(RexInputRef.of2(arg, childFields));
-    }
-    // Generate the intermediate aggregate B
-    final List<AggregateCall> aggCalls = aggregate.getAggCallList();
-    final List<AggregateCall> newAggCalls = new ArrayList<>();
-    final List<Integer> fakeArgs = new ArrayList<>();
-    final Map<AggregateCall, Integer> callArgMap = new HashMap<>();
-    // First identify the real arguments, then use the rest for fake arguments
-    // e.g. if real arguments are 0, 1, 3. Then the fake arguments will be 2, 4
-    for (final AggregateCall aggCall : aggCalls) {
-      if (!aggCall.isDistinct()) {
-        for (int arg : aggCall.getArgList()) {
-          if (!groupSet.contains(arg)) {
-            sourceOf.put(arg, projects.size());
-          }
-        }
-      }
-    }
-    int fakeArg0 = 0;
-    for (final AggregateCall aggCall : aggCalls) {
-      // We will deal with non-distinct aggregates below
-      if (!aggCall.isDistinct()
-          && (aggCall.getArgList().isEmpty()
-              || Util.intersects(groupSet, aggCall.getArgList()))) {
-        while (sourceOf.get(fakeArg0) != null) {
-          ++fakeArg0;
-        }
-        fakeArgs.add(fakeArg0);
-        ++fakeArg0;
-      }
-    }
-    for (final AggregateCall aggCall : aggCalls) {
-      if (!aggCall.isDistinct()) {
-        for (int arg : aggCall.getArgList()) {
-          if (!groupSet.contains(arg)) {
-            sourceOf.remove(arg);
-          }
-        }
+    final SortedSet<Integer> bottomGroupSet = new TreeSet<>();
+    bottomGroupSet.addAll(aggregate.getGroupSet().asList());
+    for (AggregateCall aggCall : originalAggCalls) {
+      if (aggCall.isDistinct()) {
+        bottomGroupSet.addAll(aggCall.getArgList());
+        break;  // since we only have single distinct call
       }
     }
-    // Compute the remapped arguments using fake arguments for non-distinct
-    // aggregates with no arguments e.g. count(*).
-    int fakeArgIdx = 0;
-    for (final AggregateCall aggCall : aggCalls) {
+
+    // Generate the intermediate aggregate B, the one on the bottom that converts
+    // a distinct call to group by call.
+    // Bottom aggregate is the same as the original aggregate, except that
+    // the bottom aggregate has converted the DISTINCT aggregate to a group by clause.
+    final List<AggregateCall> bottomAggregateCalls = new ArrayList<>();
+    for (AggregateCall aggCall : originalAggCalls) {
       // Project the column corresponding to the distinct aggregate. Project
       // as-is all the non-distinct aggregates
       if (!aggCall.isDistinct()) {
         final AggregateCall newCall =
             AggregateCall.create(aggCall.getAggregation(), false,
                 aggCall.getArgList(), -1,
-                ImmutableBitSet.of(newGroupSet).cardinality(),
+                ImmutableBitSet.of(bottomGroupSet).cardinality(),
                 relBuilder.peek(), null, aggCall.name);
-        newAggCalls.add(newCall);
-        if (newCall.getArgList().size() == 0) {
-          int fakeArg = fakeArgs.get(fakeArgIdx);
-          callArgMap.put(newCall, fakeArg);
-          sourceOf.put(fakeArg, projects.size());
-          projects.add(
-              Pair.of((RexNode) new RexInputRef(fakeArg, newCall.getType()),
-                  newCall.getName()));
-          ++fakeArgIdx;
-        } else {
-          for (int arg : newCall.getArgList()) {
-            if (groupSet.contains(arg)) {
-              arg = fakeArgs.get(fakeArgIdx++);
-              callArgMap.put(newCall, arg);
-            }
-            sourceOf.put(arg, projects.size());
-            projects.add(
-                Pair.of((RexNode) new RexInputRef(arg, newCall.getType()),
-                    newCall.getName()));
-          }
-        }
+        bottomAggregateCalls.add(newCall);
       }
     }
     // Generate the aggregate B (see the reference example above)
     relBuilder.push(
         aggregate.copy(
             aggregate.getTraitSet(), relBuilder.build(),
-            false, ImmutableBitSet.of(newGroupSet), null, newAggCalls));
-    // Convert the existing aggregate to aggregate A (see the reference example above)
-    final List<AggregateCall> newTopAggCalls =
-            Lists.newArrayList(aggregate.getAggCallList());
+            false, ImmutableBitSet.of(bottomGroupSet), null, bottomAggregateCalls));
+
+    // Add aggregate A (see the reference example above), the top aggregate
+    // to handle the rest of the aggregation that the bottom aggregate hasn't handled
+    final List<AggregateCall> topAggregateCalls = Lists.newArrayList();
     // Use the remapped arguments for the (non)distinct aggregate calls
-    for (int i = 0; i < newTopAggCalls.size(); i++) {
-      // Re-map arguments.
-      final AggregateCall aggCall = newTopAggCalls.get(i);
-      final int argCount = aggCall.getArgList().size();
-      final List<Integer> newArgs = new ArrayList<>(argCount);
+    int nonDistinctAggCallProcessedSoFar = 0;
+    for (AggregateCall aggCall : originalAggCalls) {
       final AggregateCall newCall;
-      for (int j = 0; j < argCount; j++) {
-        final Integer arg = aggCall.getArgList().get(j);
-        if (callArgMap.containsKey(aggCall)) {
-          newArgs.add(sourceOf.get(callArgMap.get(aggCall)));
-        } else {
-          newArgs.add(sourceOf.get(arg));
-        }
-      }
       if (aggCall.isDistinct()) {
+        List<Integer> newArgList = new ArrayList<>();
+        for (int arg : aggCall.getArgList()) {
+          newArgList.add(bottomGroupSet.headSet(arg).size());
+        }
         newCall =
-            AggregateCall.create(aggCall.getAggregation(), false, newArgs,
-                -1, aggregate.getGroupSet().cardinality(), relBuilder.peek(),
-                aggCall.getType(), aggCall.name);
+            AggregateCall.create(
+                aggCall.getAggregation(),
+                false,
+                newArgList,
+                -1,
+                originalGroupSet.cardinality(),
+                relBuilder.peek(),
+                aggCall.getType(),
+                aggCall.name);
       } else {
         // If aggregate B had a COUNT aggregate call the corresponding aggregate at
         // aggregate A must be SUM. For other aggregates, it remains the same.
-        if (aggCall.getAggregation() instanceof SqlCountAggFunction) {
-          if (aggCall.getArgList().size() == 0) {
-            newArgs.add(sourceOf.get(callArgMap.get(aggCall)));
-          }
-          if (hasGroupBy) {
-            SqlSumAggFunction sumAgg = new SqlSumAggFunction(null);
-            newCall =
-                AggregateCall.create(sumAgg, false, newArgs, -1,
-                    aggregate.getGroupSet().cardinality(), relBuilder.peek(),
-                    aggCall.getType(), aggCall.getName());
-          } else {
-            SqlSumEmptyIsZeroAggFunction sumAgg = new SqlSumEmptyIsZeroAggFunction();
-            newCall =
-                AggregateCall.create(sumAgg, false, newArgs, -1,
-                    aggregate.getGroupSet().cardinality(), relBuilder.peek(),
-                    aggCall.getType(), aggCall.getName());
-          }
+        final List<Integer> newArgs =
+            Lists.newArrayList(bottomGroupSet.size() + nonDistinctAggCallProcessedSoFar);
+        if (aggCall.getAggregation().getKind() == SqlKind.COUNT) {
+          newCall =
+              AggregateCall.create(new SqlSumEmptyIsZeroAggFunction(), false, newArgs, -1,
+                  originalGroupSet.cardinality(), relBuilder.peek(),
+                  aggCall.getType(), aggCall.getName());
         } else {
           newCall =
               AggregateCall.create(aggCall.getAggregation(), false, newArgs, -1,
-                  aggregate.getGroupSet().cardinality(),
+                  originalGroupSet.cardinality(),
                   relBuilder.peek(), aggCall.getType(), aggCall.name);
         }
+        nonDistinctAggCallProcessedSoFar++;
       }
-      newTopAggCalls.set(i, newCall);
+
+      topAggregateCalls.add(newCall);
     }
+
     // Populate the group-by keys with the remapped arguments for aggregate A
-    newGroupSet.clear();
-    for (int arg : aggregate.getGroupSet()) {
-      newGroupSet.add(sourceOf.get(arg));
+    // The top groupset is basically an identity (first X fields of aggregate B's
+    // output), minus the distinct aggCall's input.
+    final Set<Integer> topGroupSet = new HashSet<>();
+    int groupSetToAdd = 0;
+    for (int bottomGroup : bottomGroupSet) {
+      if (originalGroupSet.get(bottomGroup)) {
+        topGroupSet.add(groupSetToAdd);
+      }
+      groupSetToAdd++;
     }
     relBuilder.push(
         aggregate.copy(aggregate.getTraitSet(),
             relBuilder.build(), aggregate.indicator,
-            ImmutableBitSet.of(newGroupSet), null, newTopAggCalls));
+            ImmutableBitSet.of(topGroupSet), null, topAggregateCalls));
     return relBuilder;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/7321c870/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
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 8f5e27d..3e19616 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -911,6 +911,57 @@ public class RelOptRulesTest extends RelOptTestBase {
     sql(sql).with(program).check();
   }
 
+  @Test public void testDistinctNonDistinctTwoAggregatesWithGrouping() {
+    final String sql = "SELECT deptno, SUM(comm), MIN(comm), SUM(DISTINCT sal)\n"
+        + "FROM emp\n"
+        + "GROUP BY deptno";
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(AggregateExpandDistinctAggregatesRule.JOIN)
+        .build();
+    sql(sql).with(program).check();
+  }
+
+  @Test public void testDistinctWithGrouping() {
+    final String sql = "SELECT sal, SUM(comm), MIN(comm), SUM(DISTINCT sal)\n"
+        + "FROM emp\n"
+        + "GROUP BY sal";
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(AggregateExpandDistinctAggregatesRule.JOIN)
+        .build();
+    sql(sql).with(program).check();
+  }
+
+
+  @Test public void testMultipleDistinctWithGrouping() {
+    final String sql = "SELECT sal, SUM(comm), MIN(DISTINCT comm), SUM(DISTINCT sal)\n"
+        + "FROM emp\n"
+        + "GROUP BY sal";
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(AggregateExpandDistinctAggregatesRule.JOIN)
+        .build();
+    sql(sql).with(program).check();
+  }
+
+  @Test public void testDistinctWithMultipleInputs() {
+    final String sql = "SELECT deptno, SUM(comm), MIN(comm), COUNT(DISTINCT sal, comm)\n"
+        + "FROM emp\n"
+        + "GROUP BY deptno";
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(AggregateExpandDistinctAggregatesRule.JOIN)
+        .build();
+    sql(sql).with(program).check();
+  }
+
+  @Test public void testDistinctWithMultipleInputsAndGroupby() {
+    final String sql = "SELECT deptno, SUM(comm), MIN(comm), COUNT(DISTINCT sal, deptno, comm)\n"
+        + "FROM emp\n"
+        + "GROUP BY deptno";
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(AggregateExpandDistinctAggregatesRule.JOIN)
+        .build();
+    sql(sql).with(program).check();
+  }
+
   @Test public void testPushProjectPastFilter() {
     checkPlanning(ProjectFilterTransposeRule.INSTANCE,
         "select empno + deptno from emp where sal = 10 * comm "

http://git-wip-us.apache.org/repos/asf/calcite/blob/7321c870/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
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 e208866..28c3b9d 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -620,13 +620,134 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT()], EXPR$2=[SUM(DISTINCT $1)])
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalAggregate(group=[{0}], EXPR$1=[SUM($2)], EXPR$2=[SUM($1)])
+LogicalAggregate(group=[{0}], EXPR$1=[$SUM0($2)], EXPR$2=[SUM($1)])
   LogicalAggregate(group=[{0, 1}], EXPR$1=[COUNT()])
     LogicalProject(DEPTNO=[$7], SAL=[$5])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testDistinctNonDistinctTwoAggregatesWithGrouping">
+        <Resource name="sql">
+            <![CDATA[SELECT deptno, sum(comm), min(comm), SUM(DISTINCT sal)
+FROM emp
+GROUP BY deptno]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[MIN($1)], EXPR$3=[SUM(DISTINCT $2)])
+  LogicalProject(DEPTNO=[$7], COMM=[$6], SAL=[$5])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($2)], EXPR$2=[MIN($3)], EXPR$3=[SUM($1)])
+  LogicalAggregate(group=[{0, 2}], EXPR$1=[SUM($1)], EXPR$2=[MIN($1)])
+    LogicalProject(DEPTNO=[$7], COMM=[$6], SAL=[$5])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testDistinctWithGrouping">
+        <Resource name="sql">
+            <![CDATA[SELECT sal, sum(comm), min(comm), SUM(DISTINCT sal)
+FROM emp
+GROUP BY sal]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[MIN($1)], EXPR$3=[SUM(DISTINCT $0)])
+  LogicalProject(SAL=[$5], COMM=[$6])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[MIN($2)], EXPR$3=[SUM($0)])
+  LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[MIN($1)])
+    LogicalProject(SAL=[$5], COMM=[$6])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testMultipleDistinctWithGrouping">
+        <Resource name="sql">
+            <![CDATA[SELECT sal, SUM(comm), MIN(DISTINCT comm), SUM(DISTINCT sal)
+FROM emp
+GROUP BY sal]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[MIN(DISTINCT $1)], EXPR$3=[SUM(DISTINCT $0)])
+  LogicalProject(SAL=[$5], COMM=[$6])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(SAL=[$0], EXPR$1=[$1], EXPR$2=[$3], EXPR$3=[$5])
+  LogicalJoin(condition=[IS NOT DISTINCT FROM($0, $4)], joinType=[inner])
+    LogicalJoin(condition=[IS NOT DISTINCT FROM($0, $2)], joinType=[inner])
+      LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
+        LogicalProject(SAL=[$5], COMM=[$6])
+          LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+      LogicalAggregate(group=[{0}], EXPR$2=[MIN($1)])
+        LogicalAggregate(group=[{0, 1}])
+          LogicalProject(SAL=[$5], COMM=[$6])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+    LogicalAggregate(group=[{0}], EXPR$3=[SUM($0)])
+      LogicalAggregate(group=[{0}])
+        LogicalProject(SAL=[$0])
+          LogicalProject(SAL=[$5], COMM=[$6])
+            LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testDistinctWithMultipleInputs">
+        <Resource name="sql">
+            <![CDATA[SELECT deptno, SUM(comm), MIN(comm), COUNT(DISTINCT sal, comm)
+FROM emp
+GROUP BY deptno]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[MIN($1)], EXPR$3=[COUNT(DISTINCT $2, $1)])
+  LogicalProject(DEPTNO=[$7], COMM=[$6], SAL=[$5])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($3)], EXPR$2=[MIN($4)], EXPR$3=[COUNT($2, $1)])
+  LogicalAggregate(group=[{0, 1, 2}], EXPR$1=[SUM($1)], EXPR$2=[MIN($1)])
+    LogicalProject(DEPTNO=[$7], COMM=[$6], SAL=[$5])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testDistinctWithMultipleInputsAndGroupby">
+        <Resource name="sql">
+            <![CDATA[SELECT deptno, SUM(comm), MIN(comm), COUNT(DISTINCT sal, deptno, comm)
+FROM emp
+GROUP BY deptno]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[MIN($1)], EXPR$3=[COUNT(DISTINCT $2, $0, $1)])
+  LogicalProject(DEPTNO=[$7], COMM=[$6], SAL=[$5])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($3)], EXPR$2=[MIN($4)], EXPR$3=[COUNT($2, $0, $1)])
+  LogicalAggregate(group=[{0, 1, 2}], EXPR$1=[SUM($1)], EXPR$2=[MIN($1)])
+    LogicalProject(DEPTNO=[$7], COMM=[$6], SAL=[$5])
+      LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testEmptyAggregate">
         <Resource name="sql">
             <![CDATA[select sum(empno) from emp where false group by deptno]]>
@@ -7314,7 +7435,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT()], EXPR$2=[AVG(DISTINCT $1)])
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalAggregate(group=[{0}], EXPR$1=[SUM($2)], EXPR$2=[AVG($1)])
+LogicalAggregate(group=[{0}], EXPR$1=[$SUM0($2)], EXPR$2=[AVG($1)])
   LogicalAggregate(group=[{0, 1}], EXPR$1=[COUNT()])
     LogicalProject(EMPNO=[$0], DEPTNO0=[$9])
       LogicalJoin(condition=[=($7, $9)], joinType=[inner])

http://git-wip-us.apache.org/repos/asf/calcite/blob/7321c870/core/src/test/resources/sql/agg.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/agg.iq b/core/src/test/resources/sql/agg.iq
index 7382eb7..e4ec228 100755
--- a/core/src/test/resources/sql/agg.iq
+++ b/core/src/test/resources/sql/agg.iq
@@ -1808,4 +1808,89 @@ EnumerableCalc(expr#0=[{inputs}], expr#1=[0], expr#2=[=($t0, $t1)], expr#3=[null
         EnumerableValues(tuples=[[{ 1, 2 }, { 3, 4 }]])
 !plan
 
+!use scott
+
+# [CALCITE-1930] AggregateExpandDistinctAggregateRules should handle multiple aggregate calls with same input ref
+select count(distinct EMPNO), COUNT(SAL), MIN(SAL), MAX(SAL) from "scott".emp;
++--------+--------+--------+---------+
+| EXPR$0 | EXPR$1 | EXPR$2 | EXPR$3  |
++--------+--------+--------+---------+
+|     14 |     14 | 800.00 | 5000.00 |
++--------+--------+--------+---------+
+(1 row)
+
+!ok
+
+EnumerableCalc(expr#0..3=[{inputs}], expr#4=[CAST($t1):BIGINT NOT NULL], EXPR$0=[$t0], EXPR$1=[$t4], EXPR$2=[$t2], EXPR$3=[$t3])
+  EnumerableAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $5], EXPR$1=[MIN($2) FILTER $6], EXPR$2=[MIN($3) FILTER $6], EXPR$3=[MIN($4) FILTER $6])
+    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[1], expr#7=[CASE($t1, $t5, $t6)], expr#8=[=($t7, $t6)], expr#9=[=($t7, $t5)], proj#0..4=[{exprs}], $i0=[$t8], $=[$t9])
+      EnumerableAggregate(group=[{0}], groups=[[{0}, {}]], indicator=[true], EXPR$1=[COUNT($5)], EXPR$2=[MIN($5)], EXPR$3=[MAX($5)])
+        EnumerableTableScan(table=[[scott, EMP]])
+!plan
+
+# [CALCITE-1930] AggregateExpandDistinctAggregateRules should handle multiple aggregate calls with same input ref
+select count(distinct DEPTNO), COUNT(JOB), MIN(SAL), MAX(SAL) from "scott".emp;
++--------+--------+--------+---------+
+| EXPR$0 | EXPR$1 | EXPR$2 | EXPR$3  |
++--------+--------+--------+---------+
+|      3 |     14 | 800.00 | 5000.00 |
++--------+--------+--------+---------+
+(1 row)
+
+!ok
+
+EnumerableCalc(expr#0..3=[{inputs}], expr#4=[CAST($t1):BIGINT NOT NULL], EXPR$0=[$t0], EXPR$1=[$t4], EXPR$2=[$t2], EXPR$3=[$t3])
+  EnumerableAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $5], EXPR$1=[MIN($2) FILTER $6], EXPR$2=[MIN($3) FILTER $6], EXPR$3=[MIN($4) FILTER $6])
+    EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[1], expr#7=[CASE($t1, $t5, $t6)], expr#8=[=($t7, $t6)], expr#9=[=($t7, $t5)], proj#0..4=[{exprs}], $i0=[$t8], $=[$t9])
+      EnumerableAggregate(group=[{7}], groups=[[{7}, {}]], indicator=[true], EXPR$1=[COUNT($2)], EXPR$2=[MIN($5)], EXPR$3=[MAX($5)])
+        EnumerableTableScan(table=[[scott, EMP]])
+!plan
+
+# [CALCITE-1930] AggregateExpandDistinctAggregateRules should handle multiple aggregate calls with same input ref
+select MGR, count(distinct DEPTNO), COUNT(JOB), MIN(SAL), MAX(SAL) from "scott".emp group by MGR;
++------+--------+--------+---------+---------+
+| MGR  | EXPR$1 | EXPR$2 | EXPR$3  | EXPR$4  |
++------+--------+--------+---------+---------+
+| 7566 |      1 |      2 | 3000.00 | 3000.00 |
+| 7698 |      1 |      5 |  950.00 | 1600.00 |
+| 7782 |      1 |      1 | 1300.00 | 1300.00 |
+| 7788 |      1 |      1 | 1100.00 | 1100.00 |
+| 7839 |      3 |      3 | 2450.00 | 2975.00 |
+| 7902 |      1 |      1 |  800.00 |  800.00 |
+|      |      1 |      1 | 5000.00 | 5000.00 |
++------+--------+--------+---------+---------+
+(7 rows)
+
+!ok
+
+EnumerableCalc(expr#0..4=[{inputs}], expr#5=[CAST($t2):BIGINT NOT NULL], proj#0..1=[{exprs}], EXPR$2=[$t5], EXPR$3=[$t3], EXPR$4=[$t4])
+  EnumerableAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $7], EXPR$2=[MIN($4) FILTER $8], EXPR$3=[MIN($5) FILTER $8], EXPR$4=[MIN($6) FILTER $8])
+    EnumerableCalc(expr#0..6=[{inputs}], expr#7=[0], expr#8=[1], expr#9=[CASE($t2, $t7, $t8)], expr#10=[2], expr#11=[CASE($t3, $t7, $t10)], expr#12=[+($t9, $t11)], expr#13=[3], expr#14=[=($t12, $t13)], expr#15=[=($t12, $t8)], proj#0..6=[{exprs}], $i0_1=[$t14], $i0=[$t15])
+      EnumerableAggregate(group=[{3, 7}], groups=[[{3, 7}, {3}]], indicator=[true], EXPR$2=[COUNT($2)], EXPR$3=[MIN($5)], EXPR$4=[MAX($5)])
+        EnumerableTableScan(table=[[scott, EMP]])
+!plan
+
+# [CALCITE-1930] AggregateExpandDistinctAggregateRules should handle multiple aggregate calls with same input ref
+select MGR, count(distinct DEPTNO, JOB), MIN(SAL), MAX(SAL) from "scott".emp group by MGR;
++------+--------+---------+---------+
+| MGR  | EXPR$1 | EXPR$2  | EXPR$3  |
++------+--------+---------+---------+
+| 7566 |      1 | 3000.00 | 3000.00 |
+| 7698 |      2 |  950.00 | 1600.00 |
+| 7782 |      1 | 1300.00 | 1300.00 |
+| 7788 |      1 | 1100.00 | 1100.00 |
+| 7839 |      3 | 2450.00 | 2975.00 |
+| 7902 |      1 |  800.00 |  800.00 |
+|      |      1 | 5000.00 | 5000.00 |
++------+--------+---------+---------+
+(7 rows)
+
+!ok
+
+EnumerableAggregate(group=[{1}], EXPR$1=[COUNT($2, $0) FILTER $8], EXPR$2=[MIN($6) FILTER $9], EXPR$3=[MIN($7) FILTER $9])
+  EnumerableCalc(expr#0..7=[{inputs}], expr#8=[0], expr#9=[1], expr#10=[CASE($t3, $t8, $t9)], expr#11=[2], expr#12=[CASE($t4, $t8, $t11)], expr#13=[+($t10, $t12)], expr#14=[4], expr#15=[CASE($t5, $t8, $t14)], expr#16=[+($t13, $t15)], expr#17=[7], expr#18=[=($t16, $t17)], expr#19=[=($t16, $t11)], proj#0..7=[{exprs}], $i2_3_7=[$t18], $i3=[$t19])
+    EnumerableAggregate(group=[{2, 3, 7}], groups=[[{2, 3, 7}, {3}]], indicator=[true], EXPR$2=[MIN($5)], EXPR$3=[MAX($5)])
+      EnumerableTableScan(table=[[scott, EMP]])
+!plan
+
 # End agg.iq