You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ku...@apache.org on 2019/05/30 07:37:14 UTC

[flink] branch master updated: [FLINK-12610][table-planner-blink] Introduce aggregate related planner rules, which includes:

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 370e0cb  [FLINK-12610][table-planner-blink] Introduce aggregate related planner rules, which includes:
370e0cb is described below

commit 370e0cb427c2908677d9f32a1fcfd84cd77bb445
Author: godfrey he <go...@163.com>
AuthorDate: Thu May 30 15:36:51 2019 +0800

    [FLINK-12610][table-planner-blink] Introduce aggregate related planner rules, which includes:
    
    1. AggregateCalcMergeRule, that recognizes Aggregate on top of a Calc and if possible aggregate through the calc or removes the calc
    
    2. AggregateReduceGroupingRule, that reduces unless grouping columns
    
    3. PruneAggregateCallRule, that removes unreferenced AggregateCall from Aggregate
    
    4. FlinkAggregateRemoveRule, that is copied from Calcite's AggregateRemoveRule, and supports SUM, MIN, MAX, AUXILIARY_GROUP functions in non-empty group aggregate
    
    5. FlinkAggregateJoinTransposeRule, that is copied from Calcite's AggregateJoinTransposeRule, and supports Left/Right outer join and aggregate with AUXILIARY_GROUP
---
 .../calcite/sql/SqlSplittableAggFunction.java      | 374 ++++++++
 .../logical/FlinkAggregateJoinTransposeRule.java   | 593 +++++++++++++
 .../rules/logical/FlinkAggregateRemoveRule.java    | 131 +++
 .../physical/batch/BatchExecHashAggregate.scala    |   8 +-
 .../batch/BatchExecHashWindowAggregateBase.scala   |   2 +-
 .../physical/batch/BatchExecSortMergeJoin.scala    |   2 +-
 .../table/plan/rules/FlinkBatchRuleSets.scala      |  20 +-
 .../table/plan/rules/FlinkStreamRuleSets.scala     |  20 +-
 .../logical/AggregateReduceGroupingRule.scala      | 124 +++
 .../logical/FlinkAggregateJoinTransposeRule.scala  |  73 --
 .../rules/logical/PruneAggregateCallRule.scala     | 201 +++++
 .../batch/sql/agg/AggregateReduceGroupingTest.xml  | 974 +++++++++++++++++++++
 .../sql/join/BroadcastHashSemiAntiJoinTest.xml     |   4 +-
 .../table/plan/batch/sql/join/LookupJoinTest.xml   |  28 +-
 .../batch/sql/join/NestedLoopSemiAntiJoinTest.xml  |  30 +-
 .../table/plan/batch/sql/join/SemiAntiJoinTest.xml |  30 +-
 .../sql/join/ShuffledHashSemiAntiJoinTest.xml      |   4 +-
 .../batch/sql/join/SortMergeSemiAntiJoinTest.xml   |   4 +-
 .../logical/AggregateReduceGroupingRuleTest.xml    | 909 +++++++++++++++++++
 .../logical/CalcPruneAggregateCallRuleTest.xml     | 377 ++++++++
 .../FlinkAggregateInnerJoinTransposeRuleTest.xml   | 257 ++++++
 .../FlinkAggregateOuterJoinTransposeRuleTest.xml   | 267 ++++++
 .../rules/logical/FlinkAggregateRemoveRuleTest.xml | 528 +++++++++++
 .../logical/ProjectPruneAggregateCallRuleTest.xml  | 379 ++++++++
 .../flink/table/plan/stream/sql/join/JoinTest.xml  |  99 +--
 .../plan/stream/sql/join/SemiAntiJoinTest.xml      |  24 +-
 .../table/plan/batch/sql/RemoveCollationTest.scala |  20 +-
 .../table/plan/batch/sql/RemoveShuffleTest.scala   |   6 +-
 .../sql/agg/AggregateReduceGroupingTest.scala      |  24 +
 .../common/AggregateReduceGroupingTestBase.scala   | 318 +++++++
 .../logical/AggregateReduceGroupingRuleTest.scala  |  48 +
 .../logical/CalcPruneAggregateCallRuleTest.scala   |  55 ++
 .../FlinkAggregateInnerJoinTransposeRuleTest.scala | 150 ++++
 .../FlinkAggregateOuterJoinTransposeRuleTest.scala | 124 +++
 .../logical/FlinkAggregateRemoveRuleTest.scala     | 237 +++++
 .../logical/JoinDeriveNullFilterRuleTest.scala     |  10 +-
 .../ProjectPruneAggregateCallRuleTest.scala        |  51 ++
 .../logical/PruneAggregateCallRuleTestBase.scala   | 175 ++++
 .../flink/table/runtime/batch/sql/CalcITCase.scala |  13 +-
 .../table/runtime/batch/sql/LimitITCase.scala      |   2 +-
 .../table/runtime/batch/sql/OverWindowITCase.scala |  12 +-
 .../flink/table/runtime/batch/sql/RankITCase.scala |   4 +-
 .../table/runtime/batch/sql/UnionITCase.scala      |   7 +-
 .../sql/agg/AggregateJoinTransposeITCase.scala     | 205 +++++
 .../sql/agg/AggregateReduceGroupingITCase.scala    | 405 +++++++++
 .../batch/sql/agg/AggregateRemoveITCase.scala      | 214 +++++
 .../batch/sql/agg/PruneAggregateCallITCase.scala   | 132 +++
 .../runtime/batch/sql/join/InnerJoinITCase.scala   |   8 +-
 ...e.scala => JoinConditionTypeCoerceITCase.scala} |  10 +-
 .../table/runtime/batch/sql/join/JoinITCase.scala  |  18 +-
 .../runtime/batch/sql/join/OuterJoinITCase.scala   |   6 +-
 .../table/runtime/stream/sql/AggregateITCase.scala |   9 +-
 .../runtime/stream/sql/AggregateRemoveITCase.scala | 254 ++++++
 .../table/runtime/stream/sql/CalcITCase.scala      |  10 +-
 .../runtime/stream/sql/DeduplicateITCase.scala     |   6 +-
 .../runtime/stream/sql/OverWindowITCase.scala      |  17 +-
 .../stream/sql/PruneAggregateCallITCase.scala      | 130 +++
 .../runtime/stream/sql/TemporalSortITCase.scala    |   2 +-
 .../runtime/utils/BatchScalaTableEnvUtil.scala     |  20 +-
 .../table/runtime/utils/BatchTableEnvUtil.scala    |  31 +-
 .../flink/table/runtime/utils/BatchTestBase.scala  |  20 +-
 .../table/runtime/utils/StreamTableEnvUtil.scala   |  52 ++
 .../flink/table/runtime/utils/StreamTestData.scala | 109 ---
 .../flink/table/runtime/utils/TestData.scala       | 146 +--
 .../apache/flink/table/util/TableTestBase.scala    |  20 +-
 65 files changed, 8024 insertions(+), 518 deletions(-)

diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/SqlSplittableAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/SqlSplittableAggFunction.java
new file mode 100644
index 0000000..a69a82d
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/SqlSplittableAggFunction.java
@@ -0,0 +1,374 @@
+/*
+ * 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 com.google.common.collect.ImmutableList;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.mapping.Mappings;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This file is copied from Calcite and made the following changes:
+ * 1. makeProperRexNodeForOuterJoin function added for CountSplitter and AbstractSumSplitter.
+ * 2. If the join type is left or right outer join then make the proper rexNode, or follow the previous logic.
+ *
+ * This copy can be removed once [CALCITE-2378] is fixed.
+ */
+
+/**
+ * Aggregate function that can be split into partial aggregates.
+ *
+ * <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 {
+	AggregateCall split(AggregateCall aggregateCall,
+			Mappings.TargetMapping mapping);
+
+	/** Called to generate an aggregate for the other side of the join
+	 * than the side aggregate call's arguments come from. Returns null if
+	 * no aggregate is required. */
+	AggregateCall other(RelDataTypeFactory typeFactory, AggregateCall e);
+
+	/** Generates an aggregate call to merge sub-totals.
+	 *
+	 * <p>Most implementations will add a single aggregate call to
+	 * {@code aggCalls}, and return a {@link RexInputRef} that points to it.
+	 *
+	 * @param rexBuilder Rex builder
+	 * @param extra Place to define extra input expressions
+	 * @param offset Offset due to grouping columns (and indicator columns if
+	 *     applicable)
+	 * @param inputRowType Input row type
+	 * @param aggregateCall Source aggregate call
+	 * @param leftSubTotal Ordinal of the sub-total coming from the left side of
+	 *     the join, or -1 if there is no such sub-total
+	 * @param rightSubTotal Ordinal of the sub-total coming from the right side
+	 *     of the join, or -1 if there is no such sub-total
+	 * @param joinRelType the join type
+	 *
+	 * @return Aggregate call
+	 */
+	AggregateCall topSplit(RexBuilder rexBuilder, Registry<RexNode> extra,
+			int offset, RelDataType inputRowType, AggregateCall aggregateCall,
+			int leftSubTotal, int rightSubTotal, JoinRelType joinRelType);
+
+	/** 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);
+
+	/** Collection in which one can register an element. Registering may return
+	 * a reference to an existing element.
+	 *
+	 * @param <E> element type */
+	interface Registry<E> {
+		int register(E e);
+	}
+
+	/** Splitting strategy for {@code COUNT}.
+	 *
+	 * <p>COUNT splits into itself followed by SUM. (Actually
+	 * SUM0, because the total needs to be 0, not null, if there are 0 rows.)
+	 * This rule works for any number of arguments to COUNT, including COUNT(*).
+	 */
+	class CountSplitter implements SqlSplittableAggFunction {
+		public static final CountSplitter INSTANCE = new CountSplitter();
+
+		public AggregateCall split(AggregateCall aggregateCall,
+								   Mappings.TargetMapping mapping) {
+			return aggregateCall.transform(mapping);
+		}
+
+		public AggregateCall other(RelDataTypeFactory typeFactory, AggregateCall e) {
+			return AggregateCall.create(SqlStdOperatorTable.COUNT, false, false,
+				ImmutableIntList.of(), -1,
+				typeFactory.createSqlType(SqlTypeName.BIGINT), null);
+		}
+
+		/**
+		 * This new function create a proper RexNode for {@coide COUNT} Agg with OuterJoin Condition.
+		 */
+		private RexNode makeProperRexNodeForOuterJoin(RexBuilder rexBuilder,
+													  RelDataType inputRowType,
+													  AggregateCall aggregateCall,
+													  int index) {
+			RexInputRef inputRef = rexBuilder.makeInputRef(inputRowType.getFieldList().get(index).getType(), index);
+			RexLiteral literal;
+			boolean isCountStar = aggregateCall.getArgList() == null || aggregateCall.getArgList().isEmpty();
+			if (isCountStar) {
+				literal = rexBuilder.makeExactLiteral(BigDecimal.ONE);
+			} else {
+				literal = rexBuilder.makeExactLiteral(BigDecimal.ZERO);
+			}
+			RexNode predicate = rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, inputRef);
+			return rexBuilder.makeCall(SqlStdOperatorTable.CASE,
+				predicate,
+				literal,
+				rexBuilder.makeCast(aggregateCall.type, inputRef)
+			);
+		}
+
+		public AggregateCall topSplit(RexBuilder rexBuilder,
+									  Registry<RexNode> extra, int offset, RelDataType inputRowType,
+									  AggregateCall aggregateCall, int leftSubTotal, int rightSubTotal,
+									  JoinRelType joinRelType) {
+			final List<RexNode> merges = new ArrayList<>();
+			if (leftSubTotal >= 0) {
+				// add support for right outer join
+				if (joinRelType == JoinRelType.RIGHT) {
+					merges.add(
+						makeProperRexNodeForOuterJoin(rexBuilder, inputRowType, aggregateCall, leftSubTotal)
+					);
+				} else {
+					// if it's a inner join, then do the previous logic
+					merges.add(
+						rexBuilder.makeInputRef(aggregateCall.type, leftSubTotal));
+				}
+			}
+			if (rightSubTotal >= 0) {
+				// add support for left outer join
+				if (joinRelType == JoinRelType.LEFT) {
+					merges.add(
+						makeProperRexNodeForOuterJoin(rexBuilder, inputRowType, aggregateCall, rightSubTotal)
+					);
+				} else {
+					// if it's a inner join, then do the previous logic
+					merges.add(
+						rexBuilder.makeInputRef(aggregateCall.type, rightSubTotal));
+				}
+			}
+			RexNode node;
+			switch (merges.size()) {
+				case 1:
+					node = merges.get(0);
+					break;
+				case 2:
+					node = rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, merges);
+					break;
+				default:
+					throw new AssertionError("unexpected count " + merges);
+			}
+			int ordinal = extra.register(node);
+			return AggregateCall.create(SqlStdOperatorTable.SUM0, false, false,
+				ImmutableList.of(ordinal), -1, aggregateCall.type,
+				aggregateCall.name);
+		}
+
+		/**
+		 * {@inheritDoc}
+		 *
+		 * <p>{@code COUNT(*)}, and {@code COUNT} applied to all NOT NULL arguments,
+		 * become {@code 1}; otherwise
+		 * {@code CASE WHEN arg0 IS NOT NULL THEN 1 ELSE 0 END}.
+		 */
+		public RexNode singleton(RexBuilder rexBuilder, RelDataType inputRowType,
+								 AggregateCall aggregateCall) {
+			final List<RexNode> predicates = new ArrayList<>();
+			for (Integer arg : aggregateCall.getArgList()) {
+				final RelDataType type = inputRowType.getFieldList().get(arg).getType();
+				if (type.isNullable()) {
+					predicates.add(
+						rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL,
+							rexBuilder.makeInputRef(type, arg)));
+				}
+			}
+			final RexNode predicate =
+				RexUtil.composeConjunction(rexBuilder, predicates, true);
+			if (predicate == null) {
+				return rexBuilder.makeExactLiteral(BigDecimal.ONE);
+			} else {
+				return rexBuilder.makeCall(SqlStdOperatorTable.CASE, predicate,
+					rexBuilder.makeExactLiteral(BigDecimal.ONE),
+					rexBuilder.makeExactLiteral(BigDecimal.ZERO));
+			}
+		}
+	}
+
+	/** Aggregate function that splits into two applications of itself.
+	 *
+	 * <p>Examples are MIN and MAX. */
+	class SelfSplitter implements SqlSplittableAggFunction {
+		public static final SelfSplitter INSTANCE = new SelfSplitter();
+
+		public RexNode singleton(RexBuilder rexBuilder,
+								 RelDataType inputRowType, AggregateCall aggregateCall) {
+			final int arg = aggregateCall.getArgList().get(0);
+			final RelDataTypeField field = inputRowType.getFieldList().get(arg);
+			return rexBuilder.makeInputRef(field.getType(), arg);
+		}
+
+		public AggregateCall split(AggregateCall aggregateCall,
+								   Mappings.TargetMapping mapping) {
+			return aggregateCall.transform(mapping);
+		}
+
+		public AggregateCall other(RelDataTypeFactory typeFactory, AggregateCall e) {
+			return null; // no aggregate function required on other side
+		}
+
+		public AggregateCall topSplit(RexBuilder rexBuilder,
+									  Registry<RexNode> extra, int offset, RelDataType inputRowType,
+									  AggregateCall aggregateCall, int leftSubTotal, int rightSubTotal,
+									  JoinRelType joinRelType) {
+			assert (leftSubTotal >= 0) != (rightSubTotal >= 0);
+			final int arg = leftSubTotal >= 0 ? leftSubTotal : rightSubTotal;
+			return aggregateCall.copy(ImmutableIntList.of(arg), -1);
+		}
+	}
+
+	/** Common Splitting strategy for {@coide SUM} and {@coide SUM0}. */
+	abstract class AbstractSumSplitter implements SqlSplittableAggFunction {
+
+		public RexNode singleton(RexBuilder rexBuilder,
+								 RelDataType inputRowType, AggregateCall aggregateCall) {
+			final int arg = aggregateCall.getArgList().get(0);
+			final RelDataTypeField field = inputRowType.getFieldList().get(arg);
+			return rexBuilder.makeInputRef(field.getType(), arg);
+		}
+
+		public AggregateCall split(AggregateCall aggregateCall,
+								   Mappings.TargetMapping mapping) {
+			return aggregateCall.transform(mapping);
+		}
+
+		public AggregateCall other(RelDataTypeFactory typeFactory, AggregateCall e) {
+			return AggregateCall.create(SqlStdOperatorTable.COUNT, false, false,
+				ImmutableIntList.of(), -1,
+				typeFactory.createSqlType(SqlTypeName.BIGINT), null);
+		}
+
+		/**
+		 * This new function create a proper RexNode for {@coide SUM} Agg with OuterJoin Condition.
+		 */
+		private RexNode makeProperRexNodeForOuterJoin(RexBuilder rexBuilder,
+													  RelDataType inputRowType,
+													  AggregateCall aggregateCall,
+													  int index) {
+			RexInputRef inputRef = rexBuilder.makeInputRef(inputRowType.getFieldList().get(index).getType(), index);
+			RexLiteral literal = rexBuilder.makeExactLiteral(BigDecimal.ZERO);
+			RexNode predicate = rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, inputRef);
+			return rexBuilder.makeCall(SqlStdOperatorTable.CASE,
+				predicate,
+				literal,
+				rexBuilder.makeCast(aggregateCall.type, inputRef)
+			);
+		}
+
+		public AggregateCall topSplit(RexBuilder rexBuilder,
+									  Registry<RexNode> extra, int offset, RelDataType inputRowType,
+									  AggregateCall aggregateCall, int leftSubTotal, int rightSubTotal,
+									  JoinRelType joinRelType) {
+			final List<RexNode> merges = new ArrayList<>();
+			final List<RelDataTypeField> fieldList = inputRowType.getFieldList();
+			if (leftSubTotal >= 0) {
+				// add support for left outer join
+				if (joinRelType == JoinRelType.RIGHT && getMergeAggFunctionOfTopSplit() == SqlStdOperatorTable.SUM0) {
+					merges.add(makeProperRexNodeForOuterJoin(rexBuilder, inputRowType, aggregateCall, leftSubTotal));
+				} else {
+					// if it's a inner join, then do the previous logic
+					final RelDataType type = fieldList.get(leftSubTotal).getType();
+					merges.add(rexBuilder.makeInputRef(type, leftSubTotal));
+				}
+			}
+			if (rightSubTotal >= 0) {
+				// add support for right outer join
+				if (joinRelType == JoinRelType.LEFT && getMergeAggFunctionOfTopSplit() == SqlStdOperatorTable.SUM0) {
+					merges.add(makeProperRexNodeForOuterJoin(rexBuilder, inputRowType, aggregateCall, offset + rightSubTotal));
+				} else {
+					// if it's a inner join, then do the previous logic
+					final RelDataType type = fieldList.get(rightSubTotal).getType();
+					merges.add(rexBuilder.makeInputRef(type, rightSubTotal));
+				}
+			}
+			RexNode node;
+			switch (merges.size()) {
+				case 1:
+					node = merges.get(0);
+					break;
+				case 2:
+					node = rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, merges);
+					node = rexBuilder.makeAbstractCast(aggregateCall.type, node);
+					break;
+				default:
+					throw new AssertionError("unexpected count " + merges);
+			}
+			int ordinal = extra.register(node);
+			return AggregateCall.create(getMergeAggFunctionOfTopSplit(), false, false,
+				ImmutableList.of(ordinal), -1, aggregateCall.type,
+				aggregateCall.name);
+		}
+
+		protected abstract SqlAggFunction getMergeAggFunctionOfTopSplit();
+
+	}
+
+	/** Splitting strategy for {@coide SUM}. */
+	class SumSplitter extends AbstractSumSplitter {
+
+		public static final SumSplitter INSTANCE = new SumSplitter();
+
+		@Override public SqlAggFunction getMergeAggFunctionOfTopSplit() {
+			return SqlStdOperatorTable.SUM;
+		}
+
+	}
+
+	/** Splitting strategy for {@code SUM0}. */
+	class Sum0Splitter extends AbstractSumSplitter {
+
+		public static final Sum0Splitter INSTANCE = new Sum0Splitter();
+
+		@Override public SqlAggFunction getMergeAggFunctionOfTopSplit() {
+			return SqlStdOperatorTable.SUM0;
+		}
+	}
+}
+
+// End SqlSplittableAggFunction.java
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkAggregateJoinTransposeRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkAggregateJoinTransposeRule.java
new file mode 100644
index 0000000..10c0b94
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkAggregateJoinTransposeRule.java
@@ -0,0 +1,593 @@
+/*
+ * 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.flink.table.plan.rules.logical;
+
+import org.apache.flink.table.plan.util.AggregateUtil;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalSnapshot;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlSplittableAggFunction;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.Bug;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.Mapping;
+import org.apache.calcite.util.mapping.Mappings;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import scala.Tuple2;
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+/**
+ * This rule is copied from Calcite's {@link org.apache.calcite.rel.rules.AggregateJoinTransposeRule}.
+ * Modification:
+ * - Do not match temporal join since lookup table source doesn't support aggregate.
+ * - Support Left/Right Outer Join
+ * - Fix type mismatch error
+ * - Support aggregate with AUXILIARY_GROUP
+ */
+
+/**
+ * Planner rule that pushes an
+ * {@link org.apache.calcite.rel.core.Aggregate}
+ * past a {@link org.apache.calcite.rel.core.Join}.
+ */
+public class FlinkAggregateJoinTransposeRule extends RelOptRule {
+	public static final FlinkAggregateJoinTransposeRule INSTANCE =
+			new FlinkAggregateJoinTransposeRule(LogicalAggregate.class, LogicalJoin.class,
+					RelFactories.LOGICAL_BUILDER, false, false);
+
+	/** Extended instance of the rule that can push down aggregate functions. */
+	public static final FlinkAggregateJoinTransposeRule EXTENDED =
+			new FlinkAggregateJoinTransposeRule(LogicalAggregate.class, LogicalJoin.class,
+					RelFactories.LOGICAL_BUILDER, true, false);
+
+	public static final FlinkAggregateJoinTransposeRule LEFT_RIGHT_OUTER_JOIN_EXTENDED =
+			new FlinkAggregateJoinTransposeRule(LogicalAggregate.class, LogicalJoin.class,
+					RelFactories.LOGICAL_BUILDER, true, true);
+
+	private final boolean allowFunctions;
+
+	private final boolean allowLeftOrRightOuterJoin;
+
+	/** Creates an FlinkAggregateJoinTransposeRule. */
+	public FlinkAggregateJoinTransposeRule(Class<? extends Aggregate> aggregateClass,
+			Class<? extends Join> joinClass, RelBuilderFactory relBuilderFactory,
+			boolean allowFunctions, boolean allowLeftOrRightOuterJoin) {
+		super(
+				operandJ(aggregateClass, null,
+						aggregate -> aggregate.getGroupType() == Aggregate.Group.SIMPLE,
+						operand(joinClass, any())),
+				relBuilderFactory, null);
+
+		this.allowFunctions = allowFunctions;
+		this.allowLeftOrRightOuterJoin = allowLeftOrRightOuterJoin;
+	}
+
+	@Deprecated // to be removed before 2.0
+	public FlinkAggregateJoinTransposeRule(Class<? extends Aggregate> aggregateClass,
+			RelFactories.AggregateFactory aggregateFactory,
+			Class<? extends Join> joinClass,
+			RelFactories.JoinFactory joinFactory) {
+		this(aggregateClass, joinClass,
+				RelBuilder.proto(aggregateFactory, joinFactory), false, false);
+	}
+
+	@Deprecated // to be removed before 2.0
+	public FlinkAggregateJoinTransposeRule(Class<? extends Aggregate> aggregateClass,
+			RelFactories.AggregateFactory aggregateFactory,
+			Class<? extends Join> joinClass,
+			RelFactories.JoinFactory joinFactory,
+			boolean allowFunctions) {
+		this(aggregateClass, joinClass,
+				RelBuilder.proto(aggregateFactory, joinFactory), allowFunctions, false);
+	}
+
+	@Deprecated // to be removed before 2.0
+	public FlinkAggregateJoinTransposeRule(Class<? extends Aggregate> aggregateClass,
+			RelFactories.AggregateFactory aggregateFactory,
+			Class<? extends Join> joinClass,
+			RelFactories.JoinFactory joinFactory,
+			RelFactories.ProjectFactory projectFactory) {
+		this(aggregateClass, joinClass,
+				RelBuilder.proto(aggregateFactory, joinFactory, projectFactory), false, false);
+	}
+
+	@Deprecated // to be removed before 2.0
+	public FlinkAggregateJoinTransposeRule(Class<? extends Aggregate> aggregateClass,
+			RelFactories.AggregateFactory aggregateFactory,
+			Class<? extends Join> joinClass,
+			RelFactories.JoinFactory joinFactory,
+			RelFactories.ProjectFactory projectFactory,
+			boolean allowFunctions) {
+		this(aggregateClass, joinClass,
+				RelBuilder.proto(aggregateFactory, joinFactory, projectFactory),
+				allowFunctions, false);
+	}
+
+	private boolean containsSnapshot(RelNode relNode) {
+		RelNode original = null;
+		if (relNode instanceof RelSubset) {
+			original = ((RelSubset) relNode).getOriginal();
+		} else if (relNode instanceof HepRelVertex) {
+			original = ((HepRelVertex) relNode).getCurrentRel();
+		} else {
+			original = relNode;
+		}
+		if (original instanceof LogicalSnapshot) {
+			return true;
+		} else if (original instanceof SingleRel) {
+			return containsSnapshot(((SingleRel) original).getInput());
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public boolean matches(RelOptRuleCall call) {
+		// avoid push aggregates through dim join
+		Join join = call.rel(1);
+		RelNode right = join.getRight();
+		// right tree should not contain temporal table
+		return !containsSnapshot(right);
+	}
+
+	public void onMatch(RelOptRuleCall call) {
+		final Aggregate origAgg = call.rel(0);
+		final Join join = call.rel(1);
+		final RexBuilder rexBuilder = origAgg.getCluster().getRexBuilder();
+		final RelBuilder relBuilder = call.builder();
+
+		boolean isLeftOrRightOuterJoin =
+				join.getJoinType() == JoinRelType.LEFT || join.getJoinType() == JoinRelType.RIGHT;
+
+		if (join.getJoinType() != JoinRelType.INNER && !(allowLeftOrRightOuterJoin && isLeftOrRightOuterJoin)) {
+			return;
+		}
+
+		// converts an aggregate with AUXILIARY_GROUP to a regular aggregate.
+		// if the converted aggregate can be push down,
+		// AggregateReduceGroupingRule will try reduce grouping of new aggregates created by this rule
+		final Pair<Aggregate, List<RexNode>> newAggAndProject = toRegularAggregate(origAgg);
+		final Aggregate aggregate = newAggAndProject.left;
+		final List<RexNode> projectAfterAgg = newAggAndProject.right;
+
+		// If any aggregate functions do not support splitting, bail out
+		// If any aggregate call has a filter or distinct, bail out
+		for (AggregateCall aggregateCall : aggregate.getAggCallList()) {
+			if (aggregateCall.getAggregation().unwrap(SqlSplittableAggFunction.class)
+					== null) {
+				return;
+			}
+			if (allowLeftOrRightOuterJoin && isLeftOrRightOuterJoin) {
+				// todo do not support max/min agg until we've built the proper model
+				if (aggregateCall.getAggregation().kind == SqlKind.MAX ||
+						aggregateCall.getAggregation().kind == SqlKind.MIN) {
+					return;
+				}
+			}
+			if (aggregateCall.filterArg >= 0 || aggregateCall.isDistinct()) {
+				return;
+			}
+		}
+
+		if (!allowFunctions && !aggregate.getAggCallList().isEmpty()) {
+			return;
+		}
+
+		// Do the columns used by the join appear in the output of the aggregate?
+		final ImmutableBitSet aggregateColumns = aggregate.getGroupSet();
+		final RelMetadataQuery mq = call.getMetadataQuery();
+		ImmutableBitSet keyColumns;
+		if (!isLeftOrRightOuterJoin) {
+			keyColumns = keyColumns(aggregateColumns,
+					mq.getPulledUpPredicates(join).pulledUpPredicates);
+		} else {
+			// this is an incomplete implementation
+			if (isAggregateKeyApplicable(aggregateColumns, join)) {
+				keyColumns = keyColumns(aggregateColumns,
+						com.google.common.collect.ImmutableList.copyOf(RelOptUtil.conjunctions(join.getCondition())));
+			} else {
+				keyColumns = aggregateColumns;
+			}
+		}
+		final ImmutableBitSet joinColumns =
+				RelOptUtil.InputFinder.bits(join.getCondition());
+		final boolean allColumnsInAggregate =
+				keyColumns.contains(joinColumns);
+		final ImmutableBitSet belowAggregateColumns =
+				aggregateColumns.union(joinColumns);
+
+		// Split join condition
+		final List<Integer> leftKeys = com.google.common.collect.Lists.newArrayList();
+		final List<Integer> rightKeys = com.google.common.collect.Lists.newArrayList();
+		final List<Boolean> filterNulls = com.google.common.collect.Lists.newArrayList();
+		RexNode nonEquiConj =
+				RelOptUtil.splitJoinCondition(join.getLeft(), join.getRight(),
+						join.getCondition(), leftKeys, rightKeys, filterNulls);
+		// If it contains non-equi join conditions, we bail out
+		if (!nonEquiConj.isAlwaysTrue()) {
+			return;
+		}
+
+		// Push each aggregate function down to each side that contains all of its
+		// arguments. Note that COUNT(*), because it has no arguments, can go to
+		// both sides.
+		final Map<Integer, Integer> map = new HashMap<>();
+		final List<Side> sides = new ArrayList<>();
+		int uniqueCount = 0;
+		int offset = 0;
+		int belowOffset = 0;
+		for (int s = 0; s < 2; s++) {
+			final Side side = new Side();
+			final RelNode joinInput = join.getInput(s);
+			int fieldCount = joinInput.getRowType().getFieldCount();
+			final ImmutableBitSet fieldSet =
+					ImmutableBitSet.range(offset, offset + fieldCount);
+			final ImmutableBitSet belowAggregateKeyNotShifted =
+					belowAggregateColumns.intersect(fieldSet);
+			for (Ord<Integer> c : Ord.zip(belowAggregateKeyNotShifted)) {
+				map.put(c.e, belowOffset + c.i);
+			}
+			final Mappings.TargetMapping mapping =
+					s == 0
+							? Mappings.createIdentity(fieldCount)
+							: Mappings.createShiftMapping(fieldCount + offset, 0, offset,
+							fieldCount);
+
+			final ImmutableBitSet belowAggregateKey =
+					belowAggregateKeyNotShifted.shift(-offset);
+			final boolean unique;
+			if (!allowFunctions) {
+				assert aggregate.getAggCallList().isEmpty();
+				// If there are no functions, it doesn't matter as much whether we
+				// aggregate the inputs before the join, because there will not be
+				// any functions experiencing a cartesian product effect.
+				//
+				// But finding out whether the input is already unique requires a call
+				// to areColumnsUnique that currently (until [CALCITE-1048] "Make
+				// metadata more robust" is fixed) places a heavy load on
+				// the metadata system.
+				//
+				// So we choose to imagine the the input is already unique, which is
+				// untrue but harmless.
+				//
+				Util.discard(Bug.CALCITE_1048_FIXED);
+				unique = true;
+			} else {
+				final Boolean unique0 =
+						mq.areColumnsUnique(joinInput, belowAggregateKey);
+				unique = unique0 != null && unique0;
+			}
+			if (unique) {
+				++uniqueCount;
+				side.aggregate = false;
+				relBuilder.push(joinInput);
+				final Map<Integer, Integer> belowAggregateKeyToNewProjectMap = new HashMap<>();
+				final List<RexNode> projects = new ArrayList<>();
+				for (Integer i : belowAggregateKey) {
+					belowAggregateKeyToNewProjectMap.put(i, projects.size());
+					projects.add(relBuilder.field(i));
+				}
+				for (Ord<AggregateCall> aggCall : Ord.zip(aggregate.getAggCallList())) {
+					final SqlAggFunction aggregation = aggCall.e.getAggregation();
+					final SqlSplittableAggFunction splitter =
+							Preconditions.checkNotNull(
+									aggregation.unwrap(SqlSplittableAggFunction.class));
+					if (!aggCall.e.getArgList().isEmpty()
+							&& fieldSet.contains(ImmutableBitSet.of(aggCall.e.getArgList()))) {
+						final RexNode singleton = splitter.singleton(rexBuilder,
+								joinInput.getRowType(), aggCall.e.transform(mapping));
+						final RexNode targetSingleton = rexBuilder.ensureType(aggCall.e.type, singleton, false);
+
+						if (targetSingleton instanceof RexInputRef) {
+							final int index = ((RexInputRef) targetSingleton).getIndex();
+							if (!belowAggregateKey.get(index)) {
+								projects.add(targetSingleton);
+								side.split.put(aggCall.i, projects.size() - 1);
+							} else {
+								side.split.put(aggCall.i, belowAggregateKeyToNewProjectMap.get(index));
+							}
+						} else {
+							projects.add(targetSingleton);
+							side.split.put(aggCall.i, projects.size() - 1);
+						}
+					}
+				}
+				relBuilder.project(projects);
+				side.newInput = relBuilder.build();
+			} else {
+				side.aggregate = true;
+				List<AggregateCall> belowAggCalls = new ArrayList<>();
+				final SqlSplittableAggFunction.Registry<AggregateCall>
+						belowAggCallRegistry = registry(belowAggCalls);
+				final int oldGroupKeyCount = aggregate.getGroupCount();
+				final int newGroupKeyCount = belowAggregateKey.cardinality();
+				for (Ord<AggregateCall> aggCall : Ord.zip(aggregate.getAggCallList())) {
+					final SqlAggFunction aggregation = aggCall.e.getAggregation();
+					final SqlSplittableAggFunction splitter =
+							Preconditions.checkNotNull(
+									aggregation.unwrap(SqlSplittableAggFunction.class));
+					final AggregateCall call1;
+					if (fieldSet.contains(ImmutableBitSet.of(aggCall.e.getArgList()))) {
+						final AggregateCall splitCall = splitter.split(aggCall.e, mapping);
+						call1 = splitCall.adaptTo(
+								joinInput, splitCall.getArgList(), splitCall.filterArg,
+								oldGroupKeyCount, newGroupKeyCount);
+					} else {
+						call1 = splitter.other(rexBuilder.getTypeFactory(), aggCall.e);
+					}
+					if (call1 != null) {
+						side.split.put(aggCall.i,
+								belowAggregateKey.cardinality()
+										+ belowAggCallRegistry.register(call1));
+					}
+				}
+				side.newInput = relBuilder.push(joinInput)
+						.aggregate(relBuilder.groupKey(belowAggregateKey, null),
+								belowAggCalls)
+						.build();
+			}
+			offset += fieldCount;
+			belowOffset += side.newInput.getRowType().getFieldCount();
+			sides.add(side);
+		}
+
+		if (uniqueCount == 2) {
+			// Both inputs to the join are unique. There is nothing to be gained by
+			// this rule. In fact, this aggregate+join may be the result of a previous
+			// invocation of this rule; if we continue we might loop forever.
+			return;
+		}
+
+		// Update condition
+		final Mapping mapping = (Mapping) Mappings.target(
+				map::get,
+				join.getRowType().getFieldCount(),
+				belowOffset);
+		final RexNode newCondition =
+				RexUtil.apply(mapping, join.getCondition());
+
+		// Create new join
+		relBuilder.push(sides.get(0).newInput)
+				.push(sides.get(1).newInput)
+				.join(join.getJoinType(), newCondition);
+
+		// Aggregate above to sum up the sub-totals
+		final List<AggregateCall> newAggCalls = new ArrayList<>();
+		final int groupIndicatorCount =
+				aggregate.getGroupCount() + aggregate.getIndicatorCount();
+		final int newLeftWidth = sides.get(0).newInput.getRowType().getFieldCount();
+		final List<RexNode> projects =
+				new ArrayList<>(
+						rexBuilder.identityProjects(relBuilder.peek().getRowType()));
+		for (Ord<AggregateCall> aggCall : Ord.zip(aggregate.getAggCallList())) {
+			final SqlAggFunction aggregation = aggCall.e.getAggregation();
+			final SqlSplittableAggFunction splitter =
+					Preconditions.checkNotNull(
+							aggregation.unwrap(SqlSplittableAggFunction.class));
+			final Integer leftSubTotal = sides.get(0).split.get(aggCall.i);
+			final Integer rightSubTotal = sides.get(1).split.get(aggCall.i);
+			newAggCalls.add(
+					splitter.topSplit(rexBuilder, registry(projects),
+							groupIndicatorCount, relBuilder.peek().getRowType(), aggCall.e,
+							leftSubTotal == null ? -1 : leftSubTotal,
+							rightSubTotal == null ? -1 : rightSubTotal + newLeftWidth, join.getJoinType()));
+		}
+
+		relBuilder.project(projects);
+
+		boolean aggConvertedToProjects = false;
+		if (allColumnsInAggregate) {
+			// let's see if we can convert aggregate into projects
+			List<RexNode> projects2 = new ArrayList<>();
+			for (int key : Mappings.apply(mapping, aggregate.getGroupSet())) {
+				projects2.add(relBuilder.field(key));
+			}
+			int aggCallIdx = projects2.size();
+			for (AggregateCall newAggCall : newAggCalls) {
+				final SqlSplittableAggFunction splitter =
+						newAggCall.getAggregation().unwrap(SqlSplittableAggFunction.class);
+				if (splitter != null) {
+					final RelDataType rowType = relBuilder.peek().getRowType();
+					final RexNode singleton = splitter.singleton(rexBuilder, rowType, newAggCall);
+					final RelDataType originalAggCallType =
+							aggregate.getRowType().getFieldList().get(aggCallIdx).getType();
+					final RexNode targetSingleton = rexBuilder.ensureType(originalAggCallType, singleton, false);
+					projects2.add(targetSingleton);
+				}
+				aggCallIdx += 1;
+			}
+			if (projects2.size()
+					== aggregate.getGroupSet().cardinality() + newAggCalls.size()) {
+				// We successfully converted agg calls into projects.
+				relBuilder.project(projects2);
+				aggConvertedToProjects = true;
+			}
+		}
+
+		if (!aggConvertedToProjects) {
+			relBuilder.aggregate(
+					relBuilder.groupKey(Mappings.apply(mapping, aggregate.getGroupSet()),
+							Mappings.apply2(mapping, aggregate.getGroupSets())),
+					newAggCalls);
+		}
+		if (projectAfterAgg != null) {
+			relBuilder.project(projectAfterAgg, origAgg.getRowType().getFieldNames());
+		}
+
+		call.transformTo(relBuilder.build());
+	}
+
+	/**
+	 * Convert aggregate with AUXILIARY_GROUP to regular aggregate.
+	 * Return original aggregate and null project if the given aggregate does not contain AUXILIARY_GROUP,
+	 * else new aggregate without AUXILIARY_GROUP and a project to permute output columns if needed.
+	 */
+	private Pair<Aggregate, List<RexNode>> toRegularAggregate(Aggregate aggregate) {
+		Tuple2<int[], Seq<AggregateCall>> auxGroupAndRegularAggCalls = AggregateUtil.checkAndSplitAggCalls(aggregate);
+		final int[] auxGroup = auxGroupAndRegularAggCalls._1;
+		final Seq<AggregateCall> regularAggCalls = auxGroupAndRegularAggCalls._2;
+		if (auxGroup.length != 0) {
+			int[] fullGroupSet = AggregateUtil.checkAndGetFullGroupSet(aggregate);
+			ImmutableBitSet newGroupSet = ImmutableBitSet.of(fullGroupSet);
+			List<AggregateCall> aggCalls = JavaConverters.seqAsJavaListConverter(regularAggCalls).asJava();
+			final Aggregate newAgg = aggregate.copy(
+					aggregate.getTraitSet(),
+					aggregate.getInput(),
+					aggregate.indicator,
+					newGroupSet,
+					com.google.common.collect.ImmutableList.of(newGroupSet),
+					aggCalls);
+			final List<RelDataTypeField> aggFields = aggregate.getRowType().getFieldList();
+			final List<RexNode> projectAfterAgg = new ArrayList<>();
+			for (int i = 0; i < fullGroupSet.length; ++i) {
+				int group = fullGroupSet[i];
+				int index = newGroupSet.indexOf(group);
+				projectAfterAgg.add(new RexInputRef(index, aggFields.get(i).getType()));
+			}
+			int fieldCntOfAgg = aggFields.size();
+			for (int i = fullGroupSet.length; i < fieldCntOfAgg; ++i) {
+				projectAfterAgg.add(new RexInputRef(i, aggFields.get(i).getType()));
+			}
+			Preconditions.checkArgument(projectAfterAgg.size() == fieldCntOfAgg);
+			return new Pair<>(newAgg, projectAfterAgg);
+		} else {
+			return new Pair<>(aggregate, null);
+		}
+	}
+
+	/**
+	 * Computes the closure of a set of columns according to a given list of
+	 * constraints. Each 'x = y' constraint causes bit y to be set if bit x is
+	 * set, and vice versa.
+	 */
+	private static ImmutableBitSet keyColumns(ImmutableBitSet aggregateColumns,
+			com.google.common.collect.ImmutableList<RexNode> predicates) {
+		SortedMap<Integer, BitSet> equivalence = new TreeMap<>();
+		for (RexNode predicate : predicates) {
+			populateEquivalences(equivalence, predicate);
+		}
+		ImmutableBitSet keyColumns = aggregateColumns;
+		for (Integer aggregateColumn : aggregateColumns) {
+			final BitSet bitSet = equivalence.get(aggregateColumn);
+			if (bitSet != null) {
+				keyColumns = keyColumns.union(bitSet);
+			}
+		}
+		return keyColumns;
+	}
+
+	private static void populateEquivalences(Map<Integer, BitSet> equivalence,
+			RexNode predicate) {
+		switch (predicate.getKind()) {
+			case EQUALS:
+				RexCall call = (RexCall) predicate;
+				final List<RexNode> operands = call.getOperands();
+				if (operands.get(0) instanceof RexInputRef) {
+					final RexInputRef ref0 = (RexInputRef) operands.get(0);
+					if (operands.get(1) instanceof RexInputRef) {
+						final RexInputRef ref1 = (RexInputRef) operands.get(1);
+						populateEquivalence(equivalence, ref0.getIndex(), ref1.getIndex());
+						populateEquivalence(equivalence, ref1.getIndex(), ref0.getIndex());
+					}
+				}
+		}
+	}
+
+	private static boolean isAggregateKeyApplicable(ImmutableBitSet aggregateKeys, Join join) {
+		JoinInfo joinInfo = join.analyzeCondition();
+		return (join.getJoinType() == JoinRelType.LEFT && joinInfo.leftSet().contains(aggregateKeys)) ||
+				(join.getJoinType() == JoinRelType.RIGHT &&
+						joinInfo.rightSet().shift(join.getInput(0).getRowType().getFieldCount())
+								.contains(aggregateKeys));
+	}
+
+	private static void populateEquivalence(Map<Integer, BitSet> equivalence,
+			int i0, int i1) {
+		BitSet bitSet = equivalence.get(i0);
+		if (bitSet == null) {
+			bitSet = new BitSet();
+			equivalence.put(i0, bitSet);
+		}
+		bitSet.set(i1);
+	}
+
+	/**
+	 * Creates a {@link org.apache.calcite.sql.SqlSplittableAggFunction.Registry}
+	 * that is a view of a list.
+	 */
+	private static <E> SqlSplittableAggFunction.Registry<E> registry(
+			final List<E> list) {
+		return new SqlSplittableAggFunction.Registry<E>() {
+			public int register(E e) {
+				int i = list.indexOf(e);
+				if (i < 0) {
+					i = list.size();
+					list.add(e);
+				}
+				return i;
+			}
+		};
+	}
+
+	/** Work space for an input to a join. */
+	private static class Side {
+		final Map<Integer, Integer> split = new HashMap<>();
+		RelNode newInput;
+		boolean aggregate;
+	}
+}
+
+// End FlinkAggregateJoinTransposeRule.java
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRule.java
new file mode 100644
index 0000000..7394a09
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRule.java
@@ -0,0 +1,131 @@
+/*
+ * 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.flink.table.plan.rules.logical;
+
+import org.apache.flink.table.functions.sql.internal.SqlAuxiliaryGroupAggFunction;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+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.RelFactories;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This rule is copied from Calcite's {@link org.apache.calcite.rel.rules.AggregateRemoveRule}.
+ * Modification:
+ * - only matches aggregate with with SIMPLE group and non-empty group
+ * - supports SUM, MIN, MAX, AUXILIARY_GROUP aggregate functions with no filterArgs
+ */
+
+/**
+ * Planner rule that removes
+ * a {@link org.apache.calcite.rel.core.Aggregate}
+ * if its aggregate functions are SUM, MIN, MAX, AUXILIARY_GROUP with no filterArgs,
+ * and the underlying relational expression is already distinct.
+ */
+public class FlinkAggregateRemoveRule extends RelOptRule {
+	public static final FlinkAggregateRemoveRule INSTANCE =
+			new FlinkAggregateRemoveRule(LogicalAggregate.class,
+					RelFactories.LOGICAL_BUILDER);
+
+	//~ Constructors -----------------------------------------------------------
+
+	@Deprecated // to be removed before 2.0
+	public FlinkAggregateRemoveRule(Class<? extends Aggregate> aggregateClass) {
+		this(aggregateClass, RelFactories.LOGICAL_BUILDER);
+	}
+
+	/**
+	 * Creates an FlinkAggregateRemoveRule.
+	 */
+	public FlinkAggregateRemoveRule(Class<? extends Aggregate> aggregateClass,
+			RelBuilderFactory relBuilderFactory) {
+		// REVIEW jvs 14-Mar-2006: We have to explicitly mention the child here
+		// to make sure the rule re-fires after the child changes (e.g. via
+		// ProjectRemoveRule), since that may change our information
+		// about whether the child is distinct.  If we clean up the inference of
+		// distinct to make it correct up-front, we can get rid of the reference
+		// to the child here.
+		super(
+				operand(aggregateClass,
+						operand(RelNode.class, any())),
+				relBuilderFactory, null);
+	}
+
+	@Override
+	public boolean matches(RelOptRuleCall call) {
+		final Aggregate aggregate = call.rel(0);
+		final RelNode input = call.rel(1);
+		if (aggregate.getGroupCount() == 0 || aggregate.indicator ||
+				aggregate.getGroupType() != Aggregate.Group.SIMPLE) {
+			return false;
+		}
+		for (AggregateCall aggCall : aggregate.getAggCallList()) {
+			SqlKind aggCallKind = aggCall.getAggregation().getKind();
+			// TODO supports more AggregateCalls
+			boolean isAllowAggCall = aggCallKind == SqlKind.SUM ||
+					aggCallKind == SqlKind.MIN ||
+					aggCallKind == SqlKind.MAX ||
+					aggCall.getAggregation() instanceof SqlAuxiliaryGroupAggFunction;
+			if (!isAllowAggCall || aggCall.filterArg >= 0 || aggCall.getArgList().size() != 1) {
+				return false;
+			}
+		}
+
+		final RelMetadataQuery mq = call.getMetadataQuery();
+		return SqlFunctions.isTrue(mq.areColumnsUnique(input, aggregate.getGroupSet()));
+	}
+
+	//~ Methods ----------------------------------------------------------------
+
+	public void onMatch(RelOptRuleCall call) {
+		final Aggregate aggregate = call.rel(0);
+		final RelNode input = call.rel(1);
+
+		// Distinct is "GROUP BY c1, c2" (where c1, c2 are a set of columns on
+		// which the input is unique, i.e. contain a key) and has no aggregate
+		// functions or the functions we enumerated. It can be removed.
+		final RelNode newInput = convert(input, aggregate.getTraitSet().simplify());
+
+		// If aggregate was projecting a subset of columns, add a project for the
+		// same effect.
+		final RelBuilder relBuilder = call.builder();
+		relBuilder.push(newInput);
+		List<Integer> projectIndices = new ArrayList<>(aggregate.getGroupSet().asList());
+		for (AggregateCall aggCall : aggregate.getAggCallList()) {
+			projectIndices.addAll(aggCall.getArgList());
+		}
+		relBuilder.project(relBuilder.fields(projectIndices));
+		// Create a project if some of the columns have become
+		// NOT NULL due to aggregate functions are removed
+		relBuilder.convert(aggregate.getRowType(), true);
+		call.transformTo(relBuilder.build());
+	}
+}
+
+// End FlinkAggregateRemoveRule.java
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala
index 2827987..3793ed3 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala
@@ -19,7 +19,7 @@ package org.apache.flink.table.plan.nodes.physical.batch
 
 import org.apache.flink.runtime.operators.DamBehavior
 import org.apache.flink.streaming.api.transformations.StreamTransformation
-import org.apache.flink.table.api.{PlannerConfigOptions, TableConfig}
+import org.apache.flink.table.api.{PlannerConfigOptions, TableConfig, TableConfigOptions}
 import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.functions.UserDefinedFunction
 import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef}
@@ -156,6 +156,10 @@ class BatchExecHashAggregate(
   }
 
   override def getParallelism(input: StreamTransformation[BaseRow], conf: TableConfig): Int = {
-    if (isFinal && grouping.length == 0) 1 else input.getParallelism
+    if (isFinal && grouping.length == 0) {
+      1
+    } else {
+      conf.getConf.getInteger(TableConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM)
+    }
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
index dcd9026..e450dff 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
@@ -147,6 +147,6 @@ abstract class BatchExecHashWindowAggregateBase(
       getOperatorName,
       operator,
       outputType.toTypeInfo,
-      input.getParallelism)
+      tableEnv.getConfig.getConf.getInteger(TableConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM))
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
index 42d248b..506e3be 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
@@ -271,7 +271,7 @@ class BatchExecSortMergeJoin(
       getOperatorName,
       operator,
       FlinkTypeFactory.toInternalRowType(getRowType).toTypeInfo,
-      leftInput.getParallelism)
+      tableEnv.getConfig.getConf.getInteger(TableConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM))
   }
 
   private def estimateOutputSize(relNode: RelNode): Double = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkBatchRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkBatchRuleSets.scala
index 1401605..9852dbd 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkBatchRuleSets.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkBatchRuleSets.scala
@@ -213,11 +213,10 @@ object FlinkBatchRuleSets {
     * This RuleSet is a sub-set of [[LOGICAL_OPT_RULES]].
     */
   private val LOGICAL_RULES: RuleSet = RuleSets.ofList(
-    // aggregation and projection rules
-    AggregateProjectMergeRule.INSTANCE,
-    AggregateProjectPullUpConstantsRule.INSTANCE,
     // reorder sort and projection
     SortProjectTransposeRule.INSTANCE,
+    // remove unnecessary sort rule
+    SortRemoveRule.INSTANCE,
 
     // join rules
     FlinkJoinPushExpressionsRule.INSTANCE,
@@ -227,8 +226,12 @@ object FlinkBatchRuleSets {
     // convert non-all union into all-union + distinct
     UnionToDistinctRule.INSTANCE,
 
+    // aggregation and projection rules
+    AggregateProjectMergeRule.INSTANCE,
+    AggregateProjectPullUpConstantsRule.INSTANCE,
+
     // remove aggregation if it does not aggregate and input is already distinct
-    AggregateRemoveRule.INSTANCE,
+    FlinkAggregateRemoveRule.INSTANCE,
     // push aggregate through join
     FlinkAggregateJoinTransposeRule.EXTENDED,
     // aggregate union rule
@@ -240,12 +243,15 @@ object FlinkBatchRuleSets {
     AggregateReduceFunctionsRule.INSTANCE,
     WindowAggregateReduceFunctionsRule.INSTANCE,
 
+    // reduce group by columns
+    AggregateReduceGroupingRule.INSTANCE,
+    // reduce useless aggCall
+    PruneAggregateCallRule.PROJECT_ON_AGGREGATE,
+    PruneAggregateCallRule.CALC_ON_AGGREGATE,
+
     // expand grouping sets
     DecomposeGroupingSetsRule.INSTANCE,
 
-    // remove unnecessary sort rule
-    SortRemoveRule.INSTANCE,
-
     // rank rules
     FlinkLogicalRankRule.CONSTANT_RANGE_INSTANCE,
     // transpose calc past rank to reduce rank input fields
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkStreamRuleSets.scala
index 419306e..eae74a1 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkStreamRuleSets.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkStreamRuleSets.scala
@@ -195,11 +195,10 @@ object FlinkStreamRuleSets {
     * This RuleSet is a sub-set of [[LOGICAL_OPT_RULES]].
     */
   private val LOGICAL_RULES: RuleSet = RuleSets.ofList(
-    // aggregation and projection rules
-    AggregateProjectMergeRule.INSTANCE,
-    AggregateProjectPullUpConstantsRule.INSTANCE,
     // reorder sort and projection
     SortProjectTransposeRule.INSTANCE,
+    // remove unnecessary sort rule
+    SortRemoveRule.INSTANCE,
 
     // join rules
     FlinkJoinPushExpressionsRule.INSTANCE,
@@ -209,8 +208,14 @@ object FlinkStreamRuleSets {
     // convert non-all union into all-union + distinct
     UnionToDistinctRule.INSTANCE,
 
+    // aggregation and projection rules
+    AggregateProjectMergeRule.INSTANCE,
+    AggregateProjectPullUpConstantsRule.INSTANCE,
+
     // remove aggregation if it does not aggregate and input is already distinct
-    AggregateRemoveRule.INSTANCE,
+    FlinkAggregateRemoveRule.INSTANCE,
+    // push aggregate through join
+    FlinkAggregateJoinTransposeRule.LEFT_RIGHT_OUTER_JOIN_EXTENDED,
     // using variants of aggregate union rule
     AggregateUnionAggregateRule.AGG_ON_FIRST_INPUT,
     AggregateUnionAggregateRule.AGG_ON_SECOND_INPUT,
@@ -219,12 +224,13 @@ object FlinkStreamRuleSets {
     AggregateReduceFunctionsRule.INSTANCE,
     WindowAggregateReduceFunctionsRule.INSTANCE,
 
+    // reduce useless aggCall
+    PruneAggregateCallRule.PROJECT_ON_AGGREGATE,
+    PruneAggregateCallRule.CALC_ON_AGGREGATE,
+
     // expand grouping sets
     DecomposeGroupingSetsRule.INSTANCE,
 
-    // remove unnecessary sort rule
-    SortRemoveRule.INSTANCE,
-
     // calc rules
     FilterCalcMergeRule.INSTANCE,
     ProjectCalcMergeRule.INSTANCE,
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRule.scala
new file mode 100644
index 0000000..e0fc406
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRule.scala
@@ -0,0 +1,124 @@
+/*
+ * 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.flink.table.plan.rules.logical
+
+import org.apache.flink.table.functions.sql.FlinkSqlOperatorTable
+import org.apache.flink.table.plan.metadata.FlinkRelMetadataQuery
+
+import com.google.common.collect.ImmutableList
+import org.apache.calcite.plan.RelOptRule.{any, operand}
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rel.core.Aggregate.Group
+import org.apache.calcite.rel.core.{Aggregate, AggregateCall, RelFactories}
+import org.apache.calcite.tools.RelBuilderFactory
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable
+
+/**
+  * Planner rule that reduces unless grouping columns.
+  *
+  * Find (minimum) unique group for the grouping columns, and use it as new grouping columns.
+  */
+class AggregateReduceGroupingRule(relBuilderFactory: RelBuilderFactory) extends RelOptRule(
+  operand(classOf[Aggregate], any),
+  relBuilderFactory,
+  "AggregateReduceGroupingRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val agg: Aggregate = call.rel(0)
+    agg.getGroupCount > 1 && agg.getGroupType == Group.SIMPLE && !agg.indicator
+  }
+
+  override def onMatch(call: RelOptRuleCall): Unit = {
+    val agg: Aggregate = call.rel(0)
+    val aggRowType = agg.getRowType
+    val input = agg.getInput
+    val inputRowType = input.getRowType
+    val originalGrouping = agg.getGroupSet
+    val fmq = FlinkRelMetadataQuery.reuseOrCreate(call.getMetadataQuery)
+    val newGrouping = fmq.getUniqueGroups(input, originalGrouping)
+    val uselessGrouping = originalGrouping.except(newGrouping)
+    if (uselessGrouping.isEmpty) {
+      return
+    }
+
+    // new agg: new grouping + aggCalls for dropped grouping + original aggCalls
+    val indexOldToNewMap = new mutable.HashMap[Int, Int]()
+    val newGroupingList = newGrouping.toList
+    var idxOfNewGrouping = 0
+    var idxOfAggCallsForDroppedGrouping = newGroupingList.size()
+    originalGrouping.zipWithIndex.foreach {
+      case (column, oldIdx) =>
+        val newIdx = if (newGroupingList.contains(column)) {
+          val p = idxOfNewGrouping
+          idxOfNewGrouping += 1
+          p
+        } else {
+          val p = idxOfAggCallsForDroppedGrouping
+          idxOfAggCallsForDroppedGrouping += 1
+          p
+        }
+        indexOldToNewMap += (oldIdx -> newIdx)
+    }
+    require(indexOldToNewMap.size == originalGrouping.cardinality())
+
+    // the indices of aggCalls (or NamedProperties for WindowAggregate) do not change
+    (originalGrouping.cardinality() until aggRowType.getFieldCount).foreach {
+      index => indexOldToNewMap += (index -> index)
+    }
+
+    val aggCallsForDroppedGrouping = uselessGrouping.map { column =>
+      val fieldType = inputRowType.getFieldList.get(column).getType
+      val fieldName = inputRowType.getFieldNames.get(column)
+      AggregateCall.create(
+        FlinkSqlOperatorTable.AUXILIARY_GROUP,
+        false,
+        false,
+        ImmutableList.of(column),
+        -1,
+        fieldType,
+        fieldName)
+    }.toList
+
+    val newAggCalls = aggCallsForDroppedGrouping ++ agg.getAggCallList
+    val newAgg = agg.copy(
+      agg.getTraitSet,
+      input,
+      agg.indicator, // always false here
+      newGrouping,
+      ImmutableList.of(newGrouping),
+      newAggCalls
+    )
+    val builder = call.builder()
+    builder.push(newAgg)
+    val projects = (0 until aggRowType.getFieldCount).map {
+      index =>
+        val refIndex = indexOldToNewMap.getOrElse(index,
+          throw new IllegalArgumentException(s"Illegal index: $index"))
+        builder.field(refIndex)
+    }
+    builder.project(projects, aggRowType.getFieldNames)
+    call.transformTo(builder.build())
+  }
+}
+
+object AggregateReduceGroupingRule {
+  val INSTANCE = new AggregateReduceGroupingRule(RelFactories.LOGICAL_BUILDER)
+}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateJoinTransposeRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateJoinTransposeRule.scala
deleted file mode 100644
index 6de7a81..0000000
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateJoinTransposeRule.scala
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.plan.rules.logical
-
-import org.apache.calcite.plan.RelOptRuleCall
-import org.apache.calcite.plan.hep.HepRelVertex
-import org.apache.calcite.plan.volcano.RelSubset
-import org.apache.calcite.rel.{RelNode, SingleRel}
-import org.apache.calcite.rel.core.{Aggregate, Join, RelFactories}
-import org.apache.calcite.rel.logical.{LogicalAggregate, LogicalJoin, LogicalSnapshot}
-import org.apache.calcite.rel.rules.AggregateJoinTransposeRule
-import org.apache.calcite.tools.RelBuilderFactory
-
-/**
-  * Flink's [[AggregateJoinTransposeRule]] which does not match temporal join
-  * since lookup table source doesn't support aggregate.
-  */
-class FlinkAggregateJoinTransposeRule(
-    aggregateClass: Class[_ <: Aggregate],
-    joinClass: Class[_ <: Join],
-    factory: RelBuilderFactory,
-    allowFunctions: Boolean)
-  extends AggregateJoinTransposeRule(aggregateClass, joinClass, factory, allowFunctions) {
-
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val join: Join = call.rel(1)
-    if (containsSnapshot(join.getRight)) {
-      // avoid push aggregates through temporal join
-      false
-    } else {
-      super.matches(call)
-    }
-  }
-
-  private def containsSnapshot(relNode: RelNode): Boolean = {
-    val original = relNode match {
-      case r: RelSubset => r.getOriginal
-      case r: HepRelVertex => r.getCurrentRel
-      case _ => relNode
-    }
-    original match {
-      case _: LogicalSnapshot => true
-      case r: SingleRel => containsSnapshot(r.getInput)
-      case _ => false
-    }
-  }
-}
-
-object FlinkAggregateJoinTransposeRule {
-
-  /** Extended instance of the rule that can push down aggregate functions. */
-  val EXTENDED = new FlinkAggregateJoinTransposeRule(
-    classOf[LogicalAggregate],
-    classOf[LogicalJoin],
-    RelFactories.LOGICAL_BUILDER,
-    true)
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PruneAggregateCallRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PruneAggregateCallRule.scala
new file mode 100644
index 0000000..a0b67ef
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PruneAggregateCallRule.scala
@@ -0,0 +1,201 @@
+/*
+ * 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.flink.table.plan.rules.logical
+
+import com.google.common.collect.{ImmutableList, Maps}
+import org.apache.calcite.plan.RelOptRule.{any, operand}
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptUtil}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.core.Aggregate.Group
+import org.apache.calcite.rel.core.{Aggregate, AggregateCall, Calc, Project, RelFactories}
+import org.apache.calcite.rex.{RexInputRef, RexNode, RexProgram, RexUtil}
+import org.apache.calcite.runtime.Utilities
+import org.apache.calcite.util.ImmutableBitSet
+import org.apache.calcite.util.mapping.Mappings
+
+import java.util
+
+import scala.collection.JavaConversions._
+
+/**
+  * Planner rule that removes unreferenced AggregateCall from Aggregate
+  */
+abstract class PruneAggregateCallRule[T <: RelNode](topClass: Class[T]) extends RelOptRule(
+  operand(topClass,
+    operand(classOf[Aggregate], any)),
+  RelFactories.LOGICAL_BUILDER,
+  s"PruneAggregateCallRule_${topClass.getCanonicalName}") {
+
+  protected def getInputRefs(relOnAgg: T): ImmutableBitSet
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val relOnAgg: T = call.rel(0)
+    val agg: Aggregate = call.rel(1)
+    if (agg.indicator || agg.getGroupType != Group.SIMPLE || agg.getAggCallList.isEmpty ||
+      // at least output one column
+      (agg.getGroupCount == 0 && agg.getAggCallList.size() == 1)) {
+      return false
+    }
+    val inputRefs = getInputRefs(relOnAgg)
+    val unrefAggCallIndices = getUnrefAggCallIndices(inputRefs, agg)
+    unrefAggCallIndices.nonEmpty
+  }
+
+  private def getUnrefAggCallIndices(
+      inputRefs: ImmutableBitSet,
+      agg: Aggregate): Array[Int] = {
+    val groupCount = agg.getGroupCount
+    agg.getAggCallList.indices.flatMap { index =>
+      val aggCallOutputIndex = groupCount + index
+      if (inputRefs.get(aggCallOutputIndex)) {
+        Array.empty[Int]
+      } else {
+        Array(index)
+      }
+    }.toArray[Int]
+  }
+
+  override def onMatch(call: RelOptRuleCall): Unit = {
+    val relOnAgg: T = call.rel(0)
+    val agg: Aggregate = call.rel(1)
+    val inputRefs = getInputRefs(relOnAgg)
+    var unrefAggCallIndices = getUnrefAggCallIndices(inputRefs, agg)
+    require(unrefAggCallIndices.nonEmpty)
+
+    val newAggCalls: util.List[AggregateCall] = new util.ArrayList(agg.getAggCallList)
+    // remove unreferenced AggCall from original aggCalls
+    unrefAggCallIndices.sorted.reverse.foreach(i => newAggCalls.remove(i))
+
+    if (newAggCalls.isEmpty && agg.getGroupCount == 0) {
+      // at least output one column
+      newAggCalls.add(agg.getAggCallList.get(0))
+      unrefAggCallIndices = unrefAggCallIndices.slice(1, unrefAggCallIndices.length)
+    }
+
+    val newAgg = agg.copy(
+      agg.getTraitSet,
+      agg.getInput,
+      agg.indicator,
+      agg.getGroupSet,
+      ImmutableList.of(agg.getGroupSet),
+      newAggCalls
+    )
+
+    var newFieldIndex = 0
+    // map old agg output index to new agg output index
+    val mapOldToNew = Maps.newHashMap[Integer, Integer]()
+    val fieldCountOfOldAgg = agg.getRowType.getFieldCount
+    val unrefAggCallOutputIndices = unrefAggCallIndices.map(_ + agg.getGroupCount)
+    (0 until fieldCountOfOldAgg).foreach { i =>
+      if (!unrefAggCallOutputIndices.contains(i)) {
+        mapOldToNew.put(i, newFieldIndex)
+        newFieldIndex += 1
+      }
+    }
+    require(mapOldToNew.size() == newAgg.getRowType.getFieldCount)
+
+    val mapping = Mappings.target(mapOldToNew, fieldCountOfOldAgg, newAgg.getRowType.getFieldCount)
+    val newRelOnAgg = createNewRel(mapping, relOnAgg, newAgg)
+    call.transformTo(newRelOnAgg)
+  }
+
+  protected def createNewRel(
+      mapping: Mappings.TargetMapping,
+      project: T,
+      newAgg: RelNode): RelNode
+}
+
+class ProjectPruneAggregateCallRule extends PruneAggregateCallRule(classOf[Project]) {
+  override protected def getInputRefs(relOnAgg: Project): ImmutableBitSet = {
+    RelOptUtil.InputFinder.bits(relOnAgg.getProjects, null)
+  }
+
+  override protected def createNewRel(
+      mapping: Mappings.TargetMapping,
+      project: Project,
+      newAgg: RelNode): RelNode = {
+    val newProjects = RexUtil.apply(mapping, project.getProjects).toList
+    if (projectsOnlyIdentity(newProjects, newAgg.getRowType.getFieldCount) &&
+      Utilities.compare(project.getRowType.getFieldNames, newAgg.getRowType.getFieldNames) == 0) {
+      newAgg
+    } else {
+      project.copy(project.getTraitSet, newAgg, newProjects, project.getRowType)
+    }
+  }
+
+  private def projectsOnlyIdentity(
+      projects: util.List[RexNode],
+      inputFieldCount: Int): Boolean = {
+    if (projects.size != inputFieldCount) {
+      return false
+    }
+    projects.zipWithIndex.forall {
+      case (project, index) =>
+        project match {
+          case r: RexInputRef => r.getIndex == index
+          case _ => false
+        }
+    }
+  }
+}
+
+class CalcPruneAggregateCallRule extends PruneAggregateCallRule(classOf[Calc]) {
+  override protected def getInputRefs(relOnAgg: Calc): ImmutableBitSet = {
+    val program = relOnAgg.getProgram
+    val condition = if (program.getCondition != null) {
+      program.expandLocalRef(program.getCondition)
+    } else {
+      null
+    }
+    val projects = program.getProjectList.map(program.expandLocalRef)
+    RelOptUtil.InputFinder.bits(projects, condition)
+  }
+
+  override protected def createNewRel(
+      mapping: Mappings.TargetMapping,
+      calc: Calc,
+      newAgg: RelNode): RelNode = {
+    val program = calc.getProgram
+    val newCondition = if (program.getCondition != null) {
+      RexUtil.apply(mapping, program.expandLocalRef(program.getCondition))
+    } else {
+      null
+    }
+    val projects = program.getProjectList.map(program.expandLocalRef)
+    val newProjects = RexUtil.apply(mapping, projects).toList
+    val newProgram = RexProgram.create(
+      newAgg.getRowType,
+      newProjects,
+      newCondition,
+      program.getOutputRowType.getFieldNames,
+      calc.getCluster.getRexBuilder
+    )
+    if (newProgram.isTrivial &&
+      Utilities.compare(calc.getRowType.getFieldNames, newAgg.getRowType.getFieldNames) == 0) {
+      newAgg
+    } else {
+      calc.copy(calc.getTraitSet, newAgg, newProgram)
+    }
+  }
+}
+
+object PruneAggregateCallRule {
+  val PROJECT_ON_AGGREGATE = new ProjectPruneAggregateCallRule
+  val CALC_ON_AGGREGATE = new CalcPruneAggregateCallRule
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/agg/AggregateReduceGroupingTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/agg/AggregateReduceGroupingTest.xml
new file mode 100644
index 0000000..d815d86
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/agg/AggregateReduceGroupingTest.xml
@@ -0,0 +1,974 @@
+<?xml version="1.0" ?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<Root>
+  <TestCase name="testAggOnFullJoin1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a2, b2, COUNT(c1) FROM (SELECT * FROM T1 FULL OUTER JOIN T2 ON a1 = b2) t GROUP BY a1, b1, a2, b2]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a2=[$4], b2=[$5], c1=[$2])
+   +- LogicalJoin(condition=[=($0, $5)], joinType=[full])
+      :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+      +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, b1, a2, b2, EXPR$4])
++- HashAggregate(isMerge=[false], groupBy=[a1, b2], auxGrouping=[b1, a2], select=[a1, b2, b1, a2, COUNT(c1) AS EXPR$4])
+   +- Exchange(distribution=[hash[a1, b2]])
+      +- HashJoin(joinType=[FullOuterJoin], where=[=(a1, b2)], select=[a1, b1, c1, a2, b2], build=[right])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- Calc(select=[a1, b1, c1])
+         :     +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+         +- Exchange(distribution=[hash[b2]])
+            +- Calc(select=[a2, b2])
+               +- TableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnFullJoin2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a3, b3, COUNT(c1) FROM (SELECT * FROM T1 FULL OUTER JOIN T3 ON a1 = a3) t GROUP BY a1, b1, a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a3=[$4], b3=[$5], c1=[$2])
+   +- LogicalJoin(condition=[=($0, $4)], joinType=[full])
+      :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+      +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, b1, a3, b3, EXPR$4])
++- HashAggregate(isMerge=[true], groupBy=[a1, a3, b3], auxGrouping=[b1], select=[a1, a3, b3, b1, Final_COUNT(count$0) AS EXPR$4])
+   +- Exchange(distribution=[hash[a1, a3, b3]])
+      +- LocalHashAggregate(groupBy=[a1, a3, b3], auxGrouping=[b1], select=[a1, a3, b3, b1, Partial_COUNT(c1) AS count$0])
+         +- HashJoin(joinType=[FullOuterJoin], where=[=(a1, a3)], select=[a1, b1, c1, a3, b3], build=[right])
+            :- Exchange(distribution=[hash[a1]])
+            :  +- Calc(select=[a1, b1, c1])
+            :     +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+            +- Exchange(distribution=[hash[a3]])
+               +- Calc(select=[a3, b3])
+                  +- TableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnInnerJoin1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a2, b2, COUNT(c1) FROM (SELECT * FROM T1, T2 WHERE a1 = b2) t GROUP BY a1, b1, a2, b2]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a2=[$4], b2=[$5], c1=[$2])
+   +- LogicalFilter(condition=[=($0, $5)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+         +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1, a2, b2], select=[a1, b1, a2, b2, COUNT(c1) AS EXPR$4])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, b2)], select=[a1, b1, c1, a2, b2], build=[right])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- Calc(select=[a1, b1, c1])
+   :     +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+   +- Exchange(distribution=[hash[b2]])
+      +- Calc(select=[a2, b2])
+         +- TableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnInnerJoin2">
+    <Resource name="sql">
+      <![CDATA[SELECT a2, b2, a3, b3, COUNT(c2), AVG(d3) FROM (SELECT * FROM T2, T3 WHERE b2 = a3) t GROUP BY a2, b2, a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)], EXPR$5=[AVG($5)])
++- LogicalProject(a2=[$0], b2=[$1], a3=[$3], b3=[$4], c2=[$2], d3=[$6])
+   +- LogicalFilter(condition=[=($1, $3)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+         +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a2, b2, a3, b3, *($f2, $f20) AS EXPR$4, /(CAST(CASE(=($f4, 0), null:BIGINT, $f3)), $f4) AS EXPR$5])
++- HashJoin(joinType=[InnerJoin], where=[=(b2, a3)], select=[a2, b2, $f2, a3, b3, $f20, $f3, $f4], isBroadcast=[true], build=[right])
+   :- Calc(select=[a2, b2, CAST(CASE(IS NOT NULL(c2), 1, 0)) AS $f2])
+   :  +- TableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   +- Exchange(distribution=[broadcast])
+      +- HashAggregate(isMerge=[true], groupBy=[a3, b3], select=[a3, b3, Final_COUNT(count1$0) AS $f2, Final_$SUM0(sum$1) AS $f3, Final_COUNT(count$2) AS $f4])
+         +- Exchange(distribution=[hash[a3, b3]])
+            +- LocalHashAggregate(groupBy=[a3, b3], select=[a3, b3, Partial_COUNT(*) AS count1$0, Partial_$SUM0(d3) AS sum$1, Partial_COUNT(d3) AS count$2])
+               +- Calc(select=[a3, b3, d3])
+                  +- TableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnInnerJoin3">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a2, b2, a3, b3, COUNT(c1) FROM (SELECT * FROM T1, T2, T3 WHERE a1 = b2 AND a1 = a3) t GROUP BY a1, b1, a2, b2, a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3, 4, 5}], EXPR$6=[COUNT($6)])
++- LogicalProject(a1=[$0], b1=[$1], a2=[$4], b2=[$5], a3=[$7], b3=[$8], c1=[$2])
+   +- LogicalFilter(condition=[AND(=($0, $5), =($0, $7))])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalJoin(condition=[true], joinType=[inner])
+         :  :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+         :  +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+         +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, b1, a2, b2, a3, b3, *($f4, $f2) AS EXPR$6])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a1, b1, a2, b2, $f4, a3, b3, $f2], isBroadcast=[true], build=[right])
+   :- Calc(select=[a1, b1, a2, b2, CAST(CASE(IS NOT NULL(c1), 1, 0)) AS $f4])
+   :  +- HashJoin(joinType=[InnerJoin], where=[=(a1, b2)], select=[a1, b1, c1, a2, b2], build=[right])
+   :     :- Exchange(distribution=[hash[a1]])
+   :     :  +- Calc(select=[a1, b1, c1])
+   :     :     +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+   :     +- Exchange(distribution=[hash[b2]])
+   :        +- Calc(select=[a2, b2])
+   :           +- TableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   +- Exchange(distribution=[broadcast])
+      +- HashAggregate(isMerge=[true], groupBy=[a3, b3], select=[a3, b3, Final_COUNT(count1$0) AS $f2])
+         +- Exchange(distribution=[hash[a3, b3]])
+            +- LocalHashAggregate(groupBy=[a3, b3], select=[a3, b3, Partial_COUNT(*) AS count1$0])
+               +- Calc(select=[a3, b3])
+                  +- TableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnLeftJoin1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a2, b2, COUNT(c1) FROM (SELECT * FROM T1 LEFT JOIN T2 ON a1 = b2) t GROUP BY a1, b1, a2, b2]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a2=[$4], b2=[$5], c1=[$2])
+   +- LogicalJoin(condition=[=($0, $5)], joinType=[left])
+      :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+      +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1, a2, b2], select=[a1, b1, a2, b2, COUNT(c1) AS EXPR$4])
++- HashJoin(joinType=[LeftOuterJoin], where=[=(a1, b2)], select=[a1, b1, c1, a2, b2], build=[right])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- Calc(select=[a1, b1, c1])
+   :     +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+   +- Exchange(distribution=[hash[b2]])
+      +- Calc(select=[a2, b2])
+         +- TableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnLeftJoin2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a3, b3, COUNT(c1) FROM (SELECT * FROM T1 LEFT JOIN T3 ON a1 = a3) t GROUP BY a1, b1, a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a3=[$4], b3=[$5], c1=[$2])
+   +- LogicalJoin(condition=[=($0, $4)], joinType=[left])
+      :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+      +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, b1, a3, b3, EXPR$4])
++- HashAggregate(isMerge=[true], groupBy=[a1, a3, b3], auxGrouping=[b1], select=[a1, a3, b3, b1, Final_COUNT(count$0) AS EXPR$4])
+   +- Exchange(distribution=[hash[a1, a3, b3]])
+      +- LocalHashAggregate(groupBy=[a1, a3, b3], auxGrouping=[b1], select=[a1, a3, b3, b1, Partial_COUNT(c1) AS count$0])
+         +- HashJoin(joinType=[LeftOuterJoin], where=[=(a1, a3)], select=[a1, b1, c1, a3, b3], isBroadcast=[true], build=[right])
+            :- Calc(select=[a1, b1, c1])
+            :  +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+            +- Exchange(distribution=[broadcast])
+               +- Calc(select=[a3, b3])
+                  +- TableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnLeftJoin3">
+    <Resource name="sql">
+      <![CDATA[SELECT a3, b3, a1, b1, COUNT(c1) FROM (SELECT * FROM T3 LEFT JOIN T1 ON a1 = a3) t GROUP BY a3, b3, a1, b1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a3=[$0], b3=[$1], a1=[$4], b1=[$5], c1=[$6])
+   +- LogicalJoin(condition=[=($4, $0)], joinType=[left])
+      :- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a3, b3], auxGrouping=[a1, b1], select=[a3, b3, a1, b1, COUNT(c1) AS EXPR$4])
++- HashJoin(joinType=[LeftOuterJoin], where=[=(a1, a3)], select=[a3, b3, a1, b1, c1], build=[left])
+   :- Exchange(distribution=[hash[a3]])
+   :  +- Calc(select=[a3, b3])
+   :     +- TableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+   +- Exchange(distribution=[hash[a1]])
+      +- Calc(select=[a1, b1, c1])
+         +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnOver">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c, COUNT(d1) FROM (SELECT a1, b1, d1, COUNT(*) OVER (PARTITION BY c1) AS c FROM T1) t GROUP BY a1, b1, c]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT($3)])
++- LogicalProject(a1=[$0], b1=[$1], c=[COUNT() OVER (PARTITION BY $2 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)], d1=[$3])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, b1, c, EXPR$3])
++- HashAggregate(isMerge=[true], groupBy=[a1, c], auxGrouping=[b1], select=[a1, c, b1, Final_COUNT(count$0) AS EXPR$3])
+   +- Exchange(distribution=[hash[a1, c]])
+      +- LocalHashAggregate(groupBy=[a1, c], auxGrouping=[b1], select=[a1, c, b1, Partial_COUNT(d1) AS count$0])
+         +- Calc(select=[a1, b1, w0$o0 AS c, d1])
+            +- OverAggregate(partitionBy=[c1], window#0=[COUNT(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[a1, b1, c1, d1, w0$o0])
+               +- Sort(orderBy=[c1 ASC])
+                  +- Exchange(distribution=[hash[c1]])
+                     +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnRightJoin1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a2, b2, COUNT(c1) FROM (SELECT * FROM T1 RIGHT JOIN T2 ON a1 = b2) t GROUP BY a1, b1, a2, b2]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a2=[$4], b2=[$5], c1=[$2])
+   +- LogicalJoin(condition=[=($0, $5)], joinType=[right])
+      :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+      +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, b1, a2, b2, EXPR$4])
++- HashAggregate(isMerge=[false], groupBy=[b2], auxGrouping=[a1, b1, a2], select=[b2, a1, b1, a2, COUNT(c1) AS EXPR$4])
+   +- HashJoin(joinType=[RightOuterJoin], where=[=(a1, b2)], select=[a1, b1, c1, a2, b2], build=[right])
+      :- Exchange(distribution=[hash[a1]])
+      :  +- Calc(select=[a1, b1, c1])
+      :     +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- Exchange(distribution=[hash[b2]])
+         +- Calc(select=[a2, b2])
+            +- TableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnRightJoin2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a3, b3, COUNT(c1) FROM (SELECT * FROM T1 RIGHT JOIN T3 ON a1 = a3) t GROUP BY a1, b1, a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a3=[$4], b3=[$5], c1=[$2])
+   +- LogicalJoin(condition=[=($0, $4)], joinType=[right])
+      :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+      +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, b1, a3, b3, EXPR$4])
++- HashAggregate(isMerge=[false], groupBy=[a3, b3], auxGrouping=[a1, b1], select=[a3, b3, a1, b1, COUNT(c1) AS EXPR$4])
+   +- HashJoin(joinType=[RightOuterJoin], where=[=(a1, a3)], select=[a1, b1, c1, a3, b3], build=[right])
+      :- Exchange(distribution=[hash[a1]])
+      :  +- Calc(select=[a1, b1, c1])
+      :     +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- Exchange(distribution=[hash[a3]])
+         +- Calc(select=[a3, b3])
+            +- TableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnRightJoin3">
+    <Resource name="sql">
+      <![CDATA[SELECT a3, b3, a1, b1, COUNT(c1) FROM (SELECT * FROM T3 RIGHT JOIN T1 ON a1 = a3) t GROUP BY a3, b3, a1, b1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a3=[$0], b3=[$1], a1=[$4], b1=[$5], c1=[$6])
+   +- LogicalJoin(condition=[=($4, $0)], joinType=[right])
+      :- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[true], groupBy=[a3, b3, a1], auxGrouping=[b1], select=[a3, b3, a1, b1, Final_COUNT(count$0) AS EXPR$4])
++- Exchange(distribution=[hash[a3, b3, a1]])
+   +- LocalHashAggregate(groupBy=[a3, b3, a1], auxGrouping=[b1], select=[a3, b3, a1, b1, Partial_COUNT(c1) AS count$0])
+      +- HashJoin(joinType=[RightOuterJoin], where=[=(a1, a3)], select=[a3, b3, a1, b1, c1], isBroadcast=[true], build=[left])
+         :- Exchange(distribution=[broadcast])
+         :  +- Calc(select=[a3, b3])
+         :     +- TableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+         +- Calc(select=[a1, b1, c1])
+            +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnWindow1">
+    <Resource name="sql">
+      <![CDATA[SELECT a4, b4, COUNT(c4) FROM T4 GROUP BY a4, b4, TUMBLE(d4, INTERVAL '15' MINUTE)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a4=[$0], b4=[$1], EXPR$2=[$3])
++- LogicalAggregate(group=[{0, 1, 2}], EXPR$2=[COUNT($3)])
+   +- LogicalProject(a4=[$0], b4=[$1], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], c4=[$2])
+      +- LogicalTableScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashWindowAggregate(groupBy=[a4], auxGrouping=[b4], window=[TumblingGroupWindow], select=[a4, b4 AS EXPR$2, COUNT(c4) AS EXPR$2])
++- Exchange(distribution=[hash[a4]])
+   +- TableSourceScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnWindow2">
+    <Resource name="sql">
+      <![CDATA[SELECT a4, c4, COUNT(b4), AVG(b4) FROM T4 GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a4=[$0], c4=[$1], EXPR$2=[$3], EXPR$3=[$4])
++- LogicalAggregate(group=[{0, 1, 2}], EXPR$2=[COUNT($3)], EXPR$3=[AVG($3)])
+   +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1])
+      +- LogicalTableScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashWindowAggregate(groupBy=[a4], auxGrouping=[c4], window=[TumblingGroupWindow], select=[a4, c4 AS EXPR$2, COUNT(b4) AS EXPR$2, AVG(b4) AS EXPR$3])
++- Exchange(distribution=[hash[a4]])
+   +- TableSourceScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnWindow3">
+    <Resource name="sql">
+      <![CDATA[SELECT a4, c4, s, COUNT(b4) FROM (SELECT a4, c4, VAR_POP(b4) AS b4, TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, c4, s]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT($3)])
++- LogicalProject(a4=[$0], c4=[$1], s=[TUMBLE_START($2)], b4=[$3])
+   +- LogicalAggregate(group=[{0, 1, 2}], b4=[VAR_POP($3)])
+      +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1])
+         +- LogicalTableScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a4, c4, s, EXPR$3])
++- HashAggregate(isMerge=[true], groupBy=[a4, s], auxGrouping=[c4], select=[a4, s, c4, Final_COUNT(count$0) AS EXPR$3])
+   +- Exchange(distribution=[hash[a4, s]])
+      +- LocalHashAggregate(groupBy=[a4, s], auxGrouping=[c4], select=[a4, s, c4, Partial_COUNT(b4) AS count$0])
+         +- Calc(select=[a4, c4, w$start AS s, /(-($f2, /(*(CAST($f3), CAST($f3)), $f4)), $f4) AS b4])
+            +- HashWindowAggregate(groupBy=[a4], auxGrouping=[c4], window=[TumblingGroupWindow], properties=[w$start, w$end, w$rowtime], select=[a4, c4 AS $f2, SUM($f4) AS $f2, SUM(b4) AS $f3, COUNT(b4) AS $f4])
+               +- Calc(select=[a4, c4, d4, b4, *(CAST(b4), CAST(b4)) AS $f4])
+                  +- Exchange(distribution=[hash[a4]])
+                     +- TableSourceScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnWindow4">
+    <Resource name="sql">
+      <![CDATA[SELECT a4, c4, e, COUNT(b4) FROM (SELECT a4, c4, VAR_POP(b4) AS b4, TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, c4, e]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT($3)])
++- LogicalProject(a4=[$0], c4=[$1], e=[TUMBLE_END($2)], b4=[$3])
+   +- LogicalAggregate(group=[{0, 1, 2}], b4=[VAR_POP($3)])
+      +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1])
+         +- LogicalTableScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a4, c4, e, EXPR$3])
++- HashAggregate(isMerge=[true], groupBy=[a4, e], auxGrouping=[c4], select=[a4, e, c4, Final_COUNT(count$0) AS EXPR$3])
+   +- Exchange(distribution=[hash[a4, e]])
+      +- LocalHashAggregate(groupBy=[a4, e], auxGrouping=[c4], select=[a4, e, c4, Partial_COUNT(b4) AS count$0])
+         +- Calc(select=[a4, c4, w$end AS e, /(-($f2, /(*(CAST($f3), CAST($f3)), $f4)), $f4) AS b4])
+            +- HashWindowAggregate(groupBy=[a4], auxGrouping=[c4], window=[TumblingGroupWindow], properties=[w$start, w$end, w$rowtime], select=[a4, c4 AS $f2, SUM($f4) AS $f2, SUM(b4) AS $f3, COUNT(b4) AS $f4])
+               +- Calc(select=[a4, c4, d4, b4, *(CAST(b4), CAST(b4)) AS $f4])
+                  +- Exchange(distribution=[hash[a4]])
+                     +- TableSourceScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnWindow5">
+    <Resource name="sql">
+      <![CDATA[SELECT a4, b4, c4, COUNT(*) FROM (SELECT a4, c4, VAR_POP(b4) AS b4, TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, b4, c4]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()])
++- LogicalProject(a4=[$0], b4=[$3], c4=[$1])
+   +- LogicalAggregate(group=[{0, 1, 2}], b4=[VAR_POP($3)])
+      +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1])
+         +- LogicalTableScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[true], groupBy=[a4, b4], auxGrouping=[c4], select=[a4, b4, c4, Final_COUNT(count1$0) AS EXPR$3])
++- Exchange(distribution=[hash[a4, b4]])
+   +- LocalHashAggregate(groupBy=[a4, b4], auxGrouping=[c4], select=[a4, b4, c4, Partial_COUNT(*) AS count1$0])
+      +- Calc(select=[a4, /(-($f2, /(*(CAST($f3), CAST($f3)), $f4)), $f4) AS b4, c4])
+         +- HashWindowAggregate(groupBy=[a4], auxGrouping=[c4], window=[TumblingGroupWindow], select=[a4, c4 AS $f2, SUM($f4) AS $f2, SUM(b4) AS $f3, COUNT(b4) AS $f4])
+            +- Calc(select=[a4, c4, d4, b4, *(CAST(b4), CAST(b4)) AS $f4])
+               +- Exchange(distribution=[hash[a4]])
+                  +- TableSourceScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithCube">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c1, COUNT(d1) FROM T1 GROUP BY CUBE (a1, b1, c1)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0, 2}, {0}, {1, 2}, {1}, {2}, {}]], EXPR$3=[COUNT($3)])
++- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, b1, c1, EXPR$3])
++- HashAggregate(isMerge=[true], groupBy=[a1, b1, c1, $e], select=[a1, b1, c1, $e, Final_COUNT(count$0) AS EXPR$3])
+   +- Exchange(distribution=[hash[a1, b1, c1, $e]])
+      +- LocalHashAggregate(groupBy=[a1, b1, c1, $e], select=[a1, b1, c1, $e, Partial_COUNT(d1) AS count$0])
+         +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[2]}, {a1=[$0], b1=[null], c1=[null], d1=[$3], $e=[3]}, {a1=[null], b1=[$1], c1=[$2], d1=[$3], $e=[4]}, {a1=[null], b1=[$1], c1=[null], d1=[$3], $e=[5]}, {a1=[null], b1=[null], c1=[$2], d1=[$3], $e=[6]}, {a1=[null], b1=[null], c1=[null], d1=[$3], $e=[7]}], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e} [...]
+            +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithGroupingSets1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c1, COUNT(d1) FROM T1 GROUP BY GROUPING SETS ((a1, b1), (a1, c1))]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}]], EXPR$3=[COUNT($3)])
++- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, b1, c1, EXPR$3])
++- HashAggregate(isMerge=[false], groupBy=[a1, b1, c1, $e], select=[a1, b1, c1, $e, COUNT(d1) AS EXPR$3])
+   +- Exchange(distribution=[hash[a1, b1, c1, $e]])
+      +- Expand(projects=[{a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[2]}], projects=[{a1, b1, null AS c1, d1, 1 AS $e}, {a1, null AS b1, c1, d1, 2 AS $e}])
+         +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithGroupingSets2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, SUM(b1) AS s FROM T1 GROUP BY GROUPING SETS((a1, c1), (a1), ())]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a1=[$0], s=[$2])
++- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], s=[SUM($2)])
+   +- LogicalProject(a1=[$0], c1=[$2], b1=[$1])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, s])
++- HashAggregate(isMerge=[false], groupBy=[a1, c1, $e], select=[a1, c1, $e, SUM(b1) AS s])
+   +- Exchange(distribution=[hash[a1, c1, $e]])
+      +- Expand(projects=[{a1=[$0], c1=[$1], b1=[$2], $e=[0]}, {a1=[$0], c1=[null], b1=[$2], $e=[1]}, {a1=[null], c1=[null], b1=[$2], $e=[3]}], projects=[{a1, c1, b1, 0 AS $e}, {a1, null AS c1, b1, 1 AS $e}, {null AS a1, null AS c1, b1, 3 AS $e}])
+         +- Calc(select=[a1, c1, b1])
+            +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithGroupingSets3">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c1, COUNT(d1) FROM T1 GROUP BY GROUPING SETS ((a1, b1, c1), (a1, b1, d1))]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], EXPR$3=[$4])
++- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2}, {0, 1, 3}]], EXPR$3=[COUNT($3)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, b1, c1, EXPR$3])
++- HashAggregate(isMerge=[false], groupBy=[a1, c1, d1, $e], auxGrouping=[b1], select=[a1, c1, d1, $e, b1, COUNT(d1_0) AS EXPR$3])
+   +- Exchange(distribution=[hash[a1, c1, d1, $e]])
+      +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[null], $e=[1], d1_0=[$3]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[2], d1_0=[$3]}], projects=[{a1, b1, c1, null AS d1, 1 AS $e, d1 AS d1_0}, {a1, b1, null AS c1, d1, 2 AS $e, d1 AS d1_0}])
+         +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithoutAggCall">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c1 FROM T1 GROUP BY a1, b1, c1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}])
++- LogicalProject(a1=[$0], b1=[$1], c1=[$2])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1, c1], select=[a1, b1, c1])
++- Exchange(distribution=[hash[a1]])
+   +- Calc(select=[a1, b1, c1])
+      +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithoutReduceGrouping">
+    <Resource name="sql">
+      <![CDATA[SELECT a3, b3, count(c3) FROM T3 GROUP BY a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
++- LogicalProject(a3=[$0], b3=[$1], c3=[$2])
+   +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[true], groupBy=[a3, b3], select=[a3, b3, Final_COUNT(count$0) AS EXPR$2])
++- Exchange(distribution=[hash[a3, b3]])
+   +- LocalHashAggregate(groupBy=[a3, b3], select=[a3, b3, Partial_COUNT(c3) AS count$0])
+      +- Calc(select=[a3, b3, c3])
+         +- TableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithRollup">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c1, COUNT(d1) FROM T1 GROUP BY ROLLUP (a1, b1, c1)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0}, {}]], EXPR$3=[COUNT($3)])
++- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, b1, c1, EXPR$3])
++- HashAggregate(isMerge=[false], groupBy=[a1, b1, c1, $e], select=[a1, b1, c1, $e, COUNT(d1) AS EXPR$3])
+   +- Exchange(distribution=[hash[a1, b1, c1, $e]])
+      +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}, {a1=[$0], b1=[null], c1=[null], d1=[$3], $e=[3]}, {a1=[null], b1=[null], c1=[null], d1=[$3], $e=[7]}], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}, {a1, null AS b1, null AS c1, d1, 3 AS $e}, {null AS a1, null AS b1, null AS c1, d1, 7 AS $e}])
+         +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiAggs1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c1, d1, m, COUNT(*) FROM (SELECT a1, b1, c1, COUNT(d1) AS d1, MAX(d1) AS m FROM T1 GROUP BY a1, b1, c1) t GROUP BY a1, b1, c1, d1, m]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3, 4}], EXPR$5=[COUNT()])
++- LogicalAggregate(group=[{0, 1, 2}], d1=[COUNT($3)], m=[MAX($3)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+SortAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1, c1, d1, m], select=[a1, b1, c1, d1, m, COUNT(*) AS EXPR$5])
++- SortAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1, c1], select=[a1, b1, c1, COUNT(d1) AS d1, MAX(d1) AS m])
+   +- Sort(orderBy=[a1 ASC])
+      +- Exchange(distribution=[hash[a1]])
+         +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiAggs2">
+    <Resource name="sql">
+      <![CDATA[SELECT a3, b3, c, s, a, COUNT(*) FROM (SELECT a3, b3, COUNT(c3) AS c, SUM(d3) AS s, AVG(d3) AS a FROM T3 GROUP BY a3, b3) t GROUP BY a3, b3, c, s, a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3, 4}], EXPR$5=[COUNT()])
++- LogicalAggregate(group=[{0, 1}], c=[COUNT($2)], s=[SUM($3)], a=[AVG($3)])
+   +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a3, b3], auxGrouping=[c, s, a], select=[a3, b3, c, s, a, COUNT(*) AS EXPR$5])
++- HashAggregate(isMerge=[true], groupBy=[a3, b3], select=[a3, b3, Final_COUNT(count$0) AS c, Final_SUM(sum$1) AS s, Final_AVG(sum$2, count$3) AS a])
+   +- Exchange(distribution=[hash[a3, b3]])
+      +- LocalHashAggregate(groupBy=[a3, b3], select=[a3, b3, Partial_COUNT(c3) AS count$0, Partial_SUM(d3) AS sum$1, Partial_AVG(d3) AS (sum$2, count$3)])
+         +- TableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiDistinctAggs1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, COUNT(DISTINCT b1), SUM(DISTINCT b1) FROM T1 GROUP BY a1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM(DISTINCT $1)])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a1], select=[a1, COUNT(b1) AS EXPR$1, SUM(b1) AS EXPR$2])
++- Exchange(distribution=[hash[a1]])
+   +- Calc(select=[a1, b1])
+      +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiDistinctAggs2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, d1, COUNT(DISTINCT c1), SUM(DISTINCT b1) FROM T1 GROUP BY a1, d1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[SUM(DISTINCT $3)])
++- LogicalProject(a1=[$0], d1=[$3], c1=[$2], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[d1], select=[a1, d1, COUNT(c1) FILTER $g_4 AS EXPR$2, SUM(b1) FILTER $g_2 AS EXPR$3])
++- Exchange(distribution=[hash[a1]])
+   +- Calc(select=[a1, b1, c1, d1, =(CASE(=($e, 2:BIGINT), 2:BIGINT, 4:BIGINT), 2) AS $g_2, =(CASE(=($e, 2:BIGINT), 2:BIGINT, 4:BIGINT), 4) AS $g_4])
+      +- Expand(projects=[{a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[2]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[4]}], projects=[{a1, b1, null AS c1, d1, 2 AS $e}, {a1, null AS b1, c1, d1, 4 AS $e}])
+         +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiDistinctAggs3">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, SUM(DISTINCT b1), MAX(DISTINCT b1), MIN(DISTINCT c1) FROM T1 GROUP BY a1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM(DISTINCT $1)], EXPR$2=[MAX(DISTINCT $1)], EXPR$3=[MIN(DISTINCT $2)])
++- LogicalProject(a1=[$0], b1=[$1], c1=[$2])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a1, b1, b1 AS b10, c1])
++- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiDistinctAggs_WithNonDistinctAgg1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, d1, COUNT(DISTINCT c1), MAX(DISTINCT b1), SUM(b1) FROM T1 GROUP BY a1, d1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[MAX(DISTINCT $3)], EXPR$4=[SUM($3)])
++- LogicalProject(a1=[$0], d1=[$3], c1=[$2], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+SortAggregate(isMerge=[false], groupBy=[a1], select=[a1, MIN(d1) FILTER $g_3 AS d1, COUNT(c1) FILTER $g_2 AS EXPR$2, MAX(b1) FILTER $g_1 AS EXPR$3, MIN(EXPR$4) FILTER $g_3 AS EXPR$4])
++- Sort(orderBy=[a1 ASC])
+   +- Exchange(distribution=[hash[a1]])
+      +- Calc(select=[a1, b1, c1, d1, b1_0 AS EXPR$4, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 1) AS $g_1, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 2) AS $g_2, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 3) AS $g_3])
+         +- Expand(projects=[{a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1], b1_0=[$1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[2], b1_0=[$1]}, {a1=[$0], b1=[null], c1=[null], d1=[$3], $e=[3], b1_0=[$1]}], projects=[{a1, b1, null AS c1, d1, 1 AS $e, b1 AS b1_0}, {a1, null AS b1, c1, d1, 2 AS $e, b1 AS b1_0}, {a1, null AS b1, null AS c1, d1, 3 AS $e, b1 AS b1_0}])
+            +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleAggOnlyConstantGroupKey">
+    <Resource name="sql">
+      <![CDATA[SELECT count(c1) FROM T1 GROUP BY 1, true]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(EXPR$0=[$2])
++- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT($2)])
+   +- LogicalProject($f0=[1], $f1=[true], c1=[$2])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[EXPR$0])
++- HashAggregate(isMerge=[true], groupBy=[$f0], select=[$f0, Final_COUNT(count$0) AS EXPR$0])
+   +- Exchange(distribution=[hash[$f0]])
+      +- LocalHashAggregate(groupBy=[$f0], select=[$f0, Partial_COUNT(c1) AS count$0])
+         +- Calc(select=[1 AS $f0, true AS $f1, c1])
+            +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleAggOnTableWithoutUniqueKey">
+    <Resource name="sql">
+      <![CDATA[SELECT a3, b3, count(c3) FROM T3 GROUP BY a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
++- LogicalProject(a3=[$0], b3=[$1], c3=[$2])
+   +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[true], groupBy=[a3, b3], select=[a3, b3, Final_COUNT(count$0) AS EXPR$2])
++- Exchange(distribution=[hash[a3, b3]])
+   +- LocalHashAggregate(groupBy=[a3, b3], select=[a3, b3, Partial_COUNT(c3) AS count$0])
+      +- Calc(select=[a3, b3, c3])
+         +- TableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleAggOnTableWithUniqueKey">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, count(c1) FROM T1 GROUP BY a1, b1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
++- LogicalProject(a1=[$0], b1=[$1], c1=[$2])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1], select=[a1, b1, COUNT(c1) AS EXPR$2])
++- Exchange(distribution=[hash[a1]])
+   +- Calc(select=[a1, b1, c1])
+      +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleAggOnTableWithUniqueKeys">
+    <Resource name="sql">
+      <![CDATA[SELECT  b2, c2, avg(a2) FROM T2 GROUP BY b2, c2]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[AVG($2)])
++- LogicalProject(b2=[$1], c2=[$2], a2=[$0])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[b2], auxGrouping=[c2], select=[b2, c2, AVG(a2) AS EXPR$2])
++- Exchange(distribution=[hash[b2]])
+   +- TableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleAggWithConstantGroupKey">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, count(c1) FROM T1 GROUP BY a1, b1, 1, true]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], EXPR$2=[$4])
++- LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$2=[COUNT($4)])
+   +- LogicalProject(a1=[$0], b1=[$1], $f2=[1], $f3=[true], c1=[$2])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1], select=[a1, b1, COUNT(c1) AS EXPR$2])
++- Exchange(distribution=[hash[a1]])
+   +- Calc(select=[a1, b1, 1 AS $f2, true AS $f3, c1])
+      +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleDistinctAgg1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, COUNT(DISTINCT c1) FROM T1 GROUP BY a1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)])
++- LogicalProject(a1=[$0], c1=[$2])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a1], select=[a1, COUNT(c1) AS EXPR$1])
++- Exchange(distribution=[hash[a1]])
+   +- Calc(select=[a1, c1])
+      +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleDistinctAgg2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, COUNT(DISTINCT c1) FROM T1 GROUP BY a1, b1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)])
++- LogicalProject(a1=[$0], b1=[$1], c1=[$2])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1], select=[a1, b1, COUNT(c1) AS EXPR$2])
++- Exchange(distribution=[hash[a1]])
+   +- Calc(select=[a1, b1, c1])
+      +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleDistinctAgg_WithNonDistinctAgg1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, COUNT(DISTINCT b1), SUM(b1) FROM T1 GROUP BY a1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($1)])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a1], select=[a1, COUNT(b1) FILTER $g_0 AS EXPR$1, MIN(EXPR$2) FILTER $g_1 AS EXPR$2])
++- Exchange(distribution=[hash[a1]])
+   +- Calc(select=[a1, b1, b1_0 AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+      +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0], b1_0=[$1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[1], b1_0=[$1]}], projects=[{a1, b1, c1, d1, 0 AS $e, b1 AS b1_0}, {a1, null AS b1, c1, d1, 1 AS $e, b1 AS b1_0}])
+         +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleDistinctAgg_WithNonDistinctAgg2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, c1, COUNT(DISTINCT b1), SUM(b1) FROM T1 GROUP BY a1, c1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[SUM($2)])
++- LogicalProject(a1=[$0], c1=[$2], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+SortAggregate(isMerge=[false], groupBy=[a1], select=[a1, MIN(c1) FILTER $g_1 AS c1, COUNT(b1) FILTER $g_0 AS EXPR$2, MIN(EXPR$3) FILTER $g_1 AS EXPR$3])
++- Sort(orderBy=[a1 ASC])
+   +- Exchange(distribution=[hash[a1]])
+      +- Calc(select=[a1, b1, c1, b1_0 AS EXPR$3, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+         +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0], b1_0=[$1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[1], b1_0=[$1]}], projects=[{a1, b1, c1, d1, 0 AS $e, b1 AS b1_0}, {a1, null AS b1, c1, d1, 1 AS $e, b1 AS b1_0}])
+            +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleDistinctAgg_WithNonDistinctAgg3">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, COUNT(DISTINCT c1), SUM(b1) FROM T1 GROUP BY a1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($2)])
++- LogicalProject(a1=[$0], c1=[$2], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashAggregate(isMerge=[false], groupBy=[a1], select=[a1, COUNT(c1) FILTER $g_0 AS EXPR$1, MIN(EXPR$2) FILTER $g_1 AS EXPR$2])
++- Exchange(distribution=[hash[a1]])
+   +- Calc(select=[a1, c1, b1 AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+      +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}])
+         +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleDistinctAgg_WithNonDistinctAgg4">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, d1, COUNT(DISTINCT c1), SUM(b1) FROM T1 GROUP BY a1, d1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[SUM($3)])
++- LogicalProject(a1=[$0], d1=[$3], c1=[$2], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+SortAggregate(isMerge=[false], groupBy=[a1], select=[a1, MIN(d1) FILTER $g_1 AS d1, COUNT(c1) FILTER $g_0 AS EXPR$2, MIN(EXPR$3) FILTER $g_1 AS EXPR$3])
++- Sort(orderBy=[a1 ASC])
+   +- Exchange(distribution=[hash[a1]])
+      +- Calc(select=[a1, c1, d1, b1 AS EXPR$3, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+         +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}])
+            +- TableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml
index 39ec889..d50dab0 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml
@@ -72,9 +72,9 @@ HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(c, f))], select=[a, b, c
 :- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[f, e])
-      +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f, Final_MAX(max$0) AS EXPR$0])
+      +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f])
          +- Exchange(distribution=[hash[d, e, f]])
-            +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f, Partial_MAX(e) AS max$0])
+            +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f])
                +- Calc(select=[d, e, f], where=[<(d, 100)])
                   +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.xml
index 56faaac..b398e66 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.xml
@@ -187,30 +187,6 @@ Calc(select=[a, b, c, PROCTIME() AS proctime, id])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testLogicalPlanWithImplicitTypeCast">
-    <Resource name="sql">
-      <![CDATA[SELECT * FROM MyTable AS T JOIN temporalTest FOR SYSTEM_TIME AS OF T.proctime AS D ON T.b = D.id]]>
-    </Resource>
-    <Resource name="planBefore">
-      <![CDATA[
-LogicalProject(a=[$0], b=[$1], c=[$2], proctime=[$3], id=[$4], name=[$5], age=[$6])
-+- LogicalFilter(condition=[=($1, $4)])
-   +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{3}])
-      :- LogicalTableScan(table=[[MyTable]])
-      +- LogicalSnapshot(period=[$cor0.proctime])
-         +- LogicalTableScan(table=[[temporalTest, source: [TestTemporalTable(id, name, age)]]])
-]]>
-    </Resource>
-    <Resource name="planAfter">
-      <![CDATA[
-FlinkLogicalCalc(select=[a, b, c, PROCTIME() AS proctime, id, name, age])
-+- FlinkLogicalJoin(condition=[=($1, $3)], joinType=[inner])
-   :- FlinkLogicalDataStreamTableScan(table=[[T0]])
-   +- FlinkLogicalSnapshot(period=[$cor0.proctime])
-      +- FlinkLogicalTableSourceScan(table=[[temporalTest, source: [TestTemporalTable(id, name, age)]]], fields=[id, name, age])
-]]>
-    </Resource>
-  </TestCase>
   <TestCase name="testLogicalPlan">
     <Resource name="sql">
       <![CDATA[
@@ -322,9 +298,9 @@ Calc(select=[EXPR$0, EXPR$1, EXPR$2])
             :- Exchange(distribution=[hash[a]], exchange_mode=[BATCH])
             :  +- LookupJoin(table=[TestTemporalTable(id, name, age)], joinType=[InnerJoin], async=[false], on=[a=id], where=[>(age, 10)], select=[b, a, id], reuse_id=[1])
             :     +- Calc(select=[b, a])
-            :        +- HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b, Final_SUM(sum$0) AS c, Final_SUM(sum$1) AS d])
+            :        +- HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
             :           +- Exchange(distribution=[hash[a, b]])
-            :              +- LocalHashAggregate(groupBy=[a, b], select=[a, b, Partial_SUM(c) AS sum$0, Partial_SUM(d) AS sum$1])
+            :              +- LocalHashAggregate(groupBy=[a, b], select=[a, b])
             :                 +- BoundedStreamScan(table=[[T1]], fields=[a, b, c, d])
             +- Exchange(distribution=[hash[a]])
                +- Calc(select=[id AS a, b])
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml
index 33d6a02..d2bd695 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml
@@ -72,9 +72,9 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(c, f))], select=[a
 :- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[f, e])
-      +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f, Final_MAX(max$0) AS EXPR$0])
+      +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f])
          +- Exchange(distribution=[hash[d, e, f]])
-            +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f, Partial_MAX(e) AS max$0])
+            +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f])
                +- Calc(select=[d, e, f], where=[<(d, 100)])
                   +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
@@ -521,9 +521,9 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[<>(b, e)], select=[a, b, c], buil
 :  :           +- Exchange(distribution=[single])
 :  :              +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
 :  :                 +- Calc(select=[true AS i])
-:  :                    +- HashAggregate(isMerge=[true], groupBy=[l], select=[l, Final_COUNT(count$0) AS EXPR$0])
+:  :                    +- HashAggregate(isMerge=[true], groupBy=[l], select=[l])
 :  :                       +- Exchange(distribution=[hash[l]])
-:  :                          +- LocalHashAggregate(groupBy=[l], select=[l, Partial_COUNT(l) AS count$0])
+:  :                          +- LocalHashAggregate(groupBy=[l], select=[l])
 :  :                             +- Calc(select=[l], where=[LIKE(n, _UTF-16LE'Test')])
 :  :                                +- TableSourceScan(table=[[t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n])
 :  +- Exchange(distribution=[broadcast])
@@ -676,12 +676,11 @@ Calc(select=[b])
    :     :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0], build=[right], singleRowJoin=[true])
    :     :  :- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     :  +- Exchange(distribution=[broadcast])
-   :     :     +- Calc(select=[c])
-   :     :        +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck])
-   :     :           +- Exchange(distribution=[single])
-   :     :              +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1])
-   :     :                 +- Calc(select=[1 AS EXPR$0])
-   :     :                    +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+   :     :     +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c])
+   :     :        +- Exchange(distribution=[single])
+   :     :           +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0])
+   :     :              +- Calc(select=[1 AS EXPR$0])
+   :     :                 +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
    :     +- Exchange(distribution=[broadcast])
    :        +- Calc(select=[true AS i])
    :           +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0])
@@ -1202,12 +1201,11 @@ Calc(select=[b])
    :     :  :- Calc(select=[a, b])
    :     :  :  +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     :  +- Exchange(distribution=[broadcast])
-   :     :     +- Calc(select=[c])
-   :     :        +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck])
-   :     :           +- Exchange(distribution=[single])
-   :     :              +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1])
-   :     :                 +- Calc(select=[1 AS EXPR$0])
-   :     :                    +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+   :     :     +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c])
+   :     :        +- Exchange(distribution=[single])
+   :     :           +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0])
+   :     :              +- Calc(select=[1 AS EXPR$0])
+   :     :                 +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
    :     +- Exchange(distribution=[broadcast])
    :        +- Calc(select=[true AS i])
    :           +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0])
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SemiAntiJoinTest.xml
index 8ad7125..6315d75 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SemiAntiJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SemiAntiJoinTest.xml
@@ -74,9 +74,9 @@ HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(c, f))], select=[a, b, c
 :  +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[e, f]])
    +- Calc(select=[f, e])
-      +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f, Final_MAX(max$0) AS EXPR$0])
+      +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f])
          +- Exchange(distribution=[hash[d, e, f]])
-            +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f, Partial_MAX(e) AS max$0])
+            +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f])
                +- Calc(select=[d, e, f], where=[<(d, 100)])
                   +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
@@ -531,9 +531,9 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[<>(b, e)], select=[a, b, c], buil
 :  :              +- Exchange(distribution=[single])
 :  :                 +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
 :  :                    +- Calc(select=[true AS i])
-:  :                       +- HashAggregate(isMerge=[true], groupBy=[l], select=[l, Final_COUNT(count$0) AS EXPR$0])
+:  :                       +- HashAggregate(isMerge=[true], groupBy=[l], select=[l])
 :  :                          +- Exchange(distribution=[hash[l]])
-:  :                             +- LocalHashAggregate(groupBy=[l], select=[l, Partial_COUNT(l) AS count$0])
+:  :                             +- LocalHashAggregate(groupBy=[l], select=[l])
 :  :                                +- Calc(select=[l], where=[LIKE(n, _UTF-16LE'Test')])
 :  :                                   +- TableSourceScan(table=[[t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n])
 :  +- Exchange(distribution=[hash[k]])
@@ -691,12 +691,11 @@ Calc(select=[b])
    :        :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0], build=[right], singleRowJoin=[true])
    :        :  :- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :        :  +- Exchange(distribution=[broadcast])
-   :        :     +- Calc(select=[c])
-   :        :        +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck])
-   :        :           +- Exchange(distribution=[single])
-   :        :              +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1])
-   :        :                 +- Calc(select=[1 AS EXPR$0])
-   :        :                    +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+   :        :     +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c])
+   :        :        +- Exchange(distribution=[single])
+   :        :           +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0])
+   :        :              +- Calc(select=[1 AS EXPR$0])
+   :        :                 +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
    :        +- Exchange(distribution=[broadcast])
    :           +- Calc(select=[true AS i])
    :              +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0])
@@ -1236,12 +1235,11 @@ Calc(select=[b])
    :        :  :- Calc(select=[a, b])
    :        :  :  +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :        :  +- Exchange(distribution=[broadcast])
-   :        :     +- Calc(select=[c])
-   :        :        +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck])
-   :        :           +- Exchange(distribution=[single])
-   :        :              +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1])
-   :        :                 +- Calc(select=[1 AS EXPR$0])
-   :        :                    +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+   :        :     +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c])
+   :        :        +- Exchange(distribution=[single])
+   :        :           +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0])
+   :        :              +- Calc(select=[1 AS EXPR$0])
+   :        :                 +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
    :        +- Exchange(distribution=[broadcast])
    :           +- Calc(select=[true AS i])
    :              +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0])
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml
index 94b5ca7..0bc53f3 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml
@@ -74,9 +74,9 @@ HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(c, f))], select=[a, b, c
 :  +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[e, f]])
    +- Calc(select=[f, e])
-      +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f, Final_MAX(max$0) AS EXPR$0])
+      +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f])
          +- Exchange(distribution=[hash[d, e, f]])
-            +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f, Partial_MAX(e) AS max$0])
+            +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f])
                +- Calc(select=[d, e, f], where=[<(d, 100)])
                   +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml
index e1ec891..8f51041 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml
@@ -74,9 +74,9 @@ SortMergeJoin(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(c, f))], select=[a,
 :  +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[e, f]])
    +- Calc(select=[f, e])
-      +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f, Final_MAX(max$0) AS EXPR$0])
+      +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f])
          +- Exchange(distribution=[hash[d, e, f]])
-            +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f, Partial_MAX(e) AS max$0])
+            +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f])
                +- Calc(select=[d, e, f], where=[<(d, 100)])
                   +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRuleTest.xml
new file mode 100644
index 0000000..97babfd
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRuleTest.xml
@@ -0,0 +1,909 @@
+<?xml version="1.0" ?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<Root>
+  <TestCase name="testAggOnFullJoin1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a2, b2, COUNT(c1) FROM (SELECT * FROM T1 FULL OUTER JOIN T2 ON a1 = b2) t GROUP BY a1, b1, a2, b2]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a2=[$4], b2=[$5], c1=[$2])
+   +- LogicalJoin(condition=[=($0, $5)], joinType=[full])
+      :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+      +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a1, b1, a2, b2, EXPR$4])
++- FlinkLogicalAggregate(group=[{0, 4}], b1=[AUXILIARY_GROUP($1)], a2=[AUXILIARY_GROUP($3)], EXPR$4=[COUNT($2)])
+   +- FlinkLogicalJoin(condition=[=($0, $4)], joinType=[full])
+      :- FlinkLogicalCalc(select=[a1, b1, c1])
+      :  +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- FlinkLogicalCalc(select=[a2, b2])
+         +- FlinkLogicalTableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnFullJoin2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a3, b3, COUNT(c1) FROM (SELECT * FROM T1 FULL OUTER JOIN T3 ON a1 = a3) t GROUP BY a1, b1, a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a3=[$4], b3=[$5], c1=[$2])
+   +- LogicalJoin(condition=[=($0, $4)], joinType=[full])
+      :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+      +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a1, b1, a3, b3, EXPR$4])
++- FlinkLogicalAggregate(group=[{0, 3, 4}], b1=[AUXILIARY_GROUP($1)], EXPR$4=[COUNT($2)])
+   +- FlinkLogicalJoin(condition=[=($0, $3)], joinType=[full])
+      :- FlinkLogicalCalc(select=[a1, b1, c1])
+      :  +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- FlinkLogicalCalc(select=[a3, b3])
+         +- FlinkLogicalTableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnInnerJoin1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a2, b2, COUNT(c1) FROM (SELECT * FROM T1, T2 WHERE a1 = b2) t GROUP BY a1, b1, a2, b2]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a2=[$4], b2=[$5], c1=[$2])
+   +- LogicalFilter(condition=[=($0, $5)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+         +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], b1=[AUXILIARY_GROUP($1)], a2=[AUXILIARY_GROUP($3)], b2=[AUXILIARY_GROUP($4)], EXPR$4=[COUNT($2)])
++- FlinkLogicalJoin(condition=[=($0, $4)], joinType=[inner])
+   :- FlinkLogicalCalc(select=[a1, b1, c1])
+   :  +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+   +- FlinkLogicalCalc(select=[a2, b2])
+      +- FlinkLogicalTableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnInnerJoin2">
+    <Resource name="sql">
+      <![CDATA[SELECT a2, b2, a3, b3, COUNT(c2), AVG(d3) FROM (SELECT * FROM T2, T3 WHERE b2 = a3) t GROUP BY a2, b2, a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)], EXPR$5=[AVG($5)])
++- LogicalProject(a2=[$0], b2=[$1], a3=[$3], b3=[$4], c2=[$2], d3=[$6])
+   +- LogicalFilter(condition=[=($1, $3)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+         +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a2, b2, a3, b3, *($f2, $f20) AS EXPR$4, /(CAST(CASE(=($f4, 0), null:BIGINT, $f3)), $f4) AS EXPR$5])
++- FlinkLogicalJoin(condition=[=($1, $3)], joinType=[inner])
+   :- FlinkLogicalCalc(select=[a2, b2, CAST(CASE(IS NOT NULL(c2), 1, 0)) AS $f2])
+   :  +- FlinkLogicalTableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   +- FlinkLogicalAggregate(group=[{0, 1}], agg#0=[COUNT()], agg#1=[$SUM0($2)], agg#2=[COUNT($2)])
+      +- FlinkLogicalCalc(select=[a3, b3, d3])
+         +- FlinkLogicalTableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnInnerJoin3">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a2, b2, a3, b3, COUNT(c1) FROM (SELECT * FROM T1, T2, T3 WHERE a1 = b2 AND a1 = a3) t GROUP BY a1, b1, a2, b2, a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3, 4, 5}], EXPR$6=[COUNT($6)])
++- LogicalProject(a1=[$0], b1=[$1], a2=[$4], b2=[$5], a3=[$7], b3=[$8], c1=[$2])
+   +- LogicalFilter(condition=[AND(=($0, $5), =($0, $7))])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalJoin(condition=[true], joinType=[inner])
+         :  :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+         :  +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+         +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a1, b1, a2, b2, a3, b3, *($f4, $f2) AS EXPR$6])
++- FlinkLogicalJoin(condition=[=($0, $5)], joinType=[inner])
+   :- FlinkLogicalCalc(select=[a1, b1, a2, b2, CAST(CASE(IS NOT NULL(c1), 1, 0)) AS $f4])
+   :  +- FlinkLogicalJoin(condition=[=($0, $4)], joinType=[inner])
+   :     :- FlinkLogicalCalc(select=[a1, b1, c1])
+   :     :  +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+   :     +- FlinkLogicalCalc(select=[a2, b2])
+   :        +- FlinkLogicalTableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   +- FlinkLogicalAggregate(group=[{0, 1}], agg#0=[COUNT()])
+      +- FlinkLogicalCalc(select=[a3, b3])
+         +- FlinkLogicalTableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnLeftJoin1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a2, b2, COUNT(c1) FROM (SELECT * FROM T1 LEFT JOIN T2 ON a1 = b2) t GROUP BY a1, b1, a2, b2]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a2=[$4], b2=[$5], c1=[$2])
+   +- LogicalJoin(condition=[=($0, $5)], joinType=[left])
+      :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+      +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], b1=[AUXILIARY_GROUP($1)], a2=[AUXILIARY_GROUP($3)], b2=[AUXILIARY_GROUP($4)], EXPR$4=[COUNT($2)])
++- FlinkLogicalJoin(condition=[=($0, $4)], joinType=[left])
+   :- FlinkLogicalCalc(select=[a1, b1, c1])
+   :  +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+   +- FlinkLogicalCalc(select=[a2, b2])
+      +- FlinkLogicalTableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnLeftJoin2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a3, b3, COUNT(c1) FROM (SELECT * FROM T1 LEFT JOIN T3 ON a1 = a3) t GROUP BY a1, b1, a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a3=[$4], b3=[$5], c1=[$2])
+   +- LogicalJoin(condition=[=($0, $4)], joinType=[left])
+      :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+      +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a1, b1, a3, b3, EXPR$4])
++- FlinkLogicalAggregate(group=[{0, 3, 4}], b1=[AUXILIARY_GROUP($1)], EXPR$4=[COUNT($2)])
+   +- FlinkLogicalJoin(condition=[=($0, $3)], joinType=[left])
+      :- FlinkLogicalCalc(select=[a1, b1, c1])
+      :  +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- FlinkLogicalCalc(select=[a3, b3])
+         +- FlinkLogicalTableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnLeftJoin3">
+    <Resource name="sql">
+      <![CDATA[SELECT a3, b3, a1, b1, COUNT(c1) FROM (SELECT * FROM T3 LEFT JOIN T1 ON a1 = a3) t GROUP BY a3, b3, a1, b1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a3=[$0], b3=[$1], a1=[$4], b1=[$5], c1=[$6])
+   +- LogicalJoin(condition=[=($4, $0)], joinType=[left])
+      :- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0, 1}], a1=[AUXILIARY_GROUP($2)], b1=[AUXILIARY_GROUP($3)], EXPR$4=[COUNT($4)])
++- FlinkLogicalJoin(condition=[=($2, $0)], joinType=[left])
+   :- FlinkLogicalCalc(select=[a3, b3])
+   :  +- FlinkLogicalTableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+   +- FlinkLogicalCalc(select=[a1, b1, c1])
+      +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnOver">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c, COUNT(d1) FROM (SELECT a1, b1, d1, COUNT(*) OVER (PARTITION BY c1) AS c FROM T1) t GROUP BY a1, b1, c]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT($3)])
++- LogicalProject(a1=[$0], b1=[$1], c=[COUNT() OVER (PARTITION BY $2 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)], d1=[$3])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a1, b1, c, EXPR$3])
++- FlinkLogicalAggregate(group=[{0, 2}], b1=[AUXILIARY_GROUP($1)], EXPR$3=[COUNT($3)])
+   +- FlinkLogicalCalc(select=[a1, b1, w0$o0 AS c, d1])
+      +- FlinkLogicalOverAggregate(window#0=[window(partition {2} order by [] range between UNBOUNDED PRECEDING and UNBOUNDED FOLLOWING aggs [COUNT()])])
+         +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnRightJoin1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a2, b2, COUNT(c1) FROM (SELECT * FROM T1 RIGHT JOIN T2 ON a1 = b2) t GROUP BY a1, b1, a2, b2]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a2=[$4], b2=[$5], c1=[$2])
+   +- LogicalJoin(condition=[=($0, $5)], joinType=[right])
+      :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+      +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a1, b1, a2, b2, EXPR$4])
++- FlinkLogicalAggregate(group=[{4}], a1=[AUXILIARY_GROUP($0)], b1=[AUXILIARY_GROUP($1)], a2=[AUXILIARY_GROUP($3)], EXPR$4=[COUNT($2)])
+   +- FlinkLogicalJoin(condition=[=($0, $4)], joinType=[right])
+      :- FlinkLogicalCalc(select=[a1, b1, c1])
+      :  +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- FlinkLogicalCalc(select=[a2, b2])
+         +- FlinkLogicalTableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnRightJoin2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, a3, b3, COUNT(c1) FROM (SELECT * FROM T1 RIGHT JOIN T3 ON a1 = a3) t GROUP BY a1, b1, a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a1=[$0], b1=[$1], a3=[$4], b3=[$5], c1=[$2])
+   +- LogicalJoin(condition=[=($0, $4)], joinType=[right])
+      :- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+      +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a1, b1, a3, b3, EXPR$4])
++- FlinkLogicalAggregate(group=[{3, 4}], a1=[AUXILIARY_GROUP($0)], b1=[AUXILIARY_GROUP($1)], EXPR$4=[COUNT($2)])
+   +- FlinkLogicalJoin(condition=[=($0, $3)], joinType=[right])
+      :- FlinkLogicalCalc(select=[a1, b1, c1])
+      :  +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- FlinkLogicalCalc(select=[a3, b3])
+         +- FlinkLogicalTableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnRightJoin3">
+    <Resource name="sql">
+      <![CDATA[SELECT a3, b3, a1, b1, COUNT(c1) FROM (SELECT * FROM T3 RIGHT JOIN T1 ON a1 = a3) t GROUP BY a3, b3, a1, b1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT($4)])
++- LogicalProject(a3=[$0], b3=[$1], a1=[$4], b1=[$5], c1=[$6])
+   +- LogicalJoin(condition=[=($4, $0)], joinType=[right])
+      :- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0, 1, 2}], b1=[AUXILIARY_GROUP($3)], EXPR$4=[COUNT($4)])
++- FlinkLogicalJoin(condition=[=($2, $0)], joinType=[right])
+   :- FlinkLogicalCalc(select=[a3, b3])
+   :  +- FlinkLogicalTableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+   +- FlinkLogicalCalc(select=[a1, b1, c1])
+      +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnWindow1">
+    <Resource name="sql">
+      <![CDATA[SELECT a4, b4, COUNT(c4) FROM T4 GROUP BY a4, b4, TUMBLE(d4, INTERVAL '15' MINUTE)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a4=[$0], b4=[$1], EXPR$2=[$3])
++- LogicalAggregate(group=[{0, 1, 2}], EXPR$2=[COUNT($3)])
+   +- LogicalProject(a4=[$0], b4=[$1], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], c4=[$2])
+      +- LogicalTableScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalWindowAggregate(group=[{0}], b4=[AUXILIARY_GROUP($1)], EXPR$2=[COUNT($2)], window=[TumblingGroupWindow], properties=[])
++- FlinkLogicalTableSourceScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnWindow2">
+    <Resource name="sql">
+      <![CDATA[SELECT a4, c4, COUNT(b4), AVG(b4) FROM T4 GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a4=[$0], c4=[$1], EXPR$2=[$3], EXPR$3=[$4])
++- LogicalAggregate(group=[{0, 1, 2}], EXPR$2=[COUNT($3)], EXPR$3=[AVG($3)])
+   +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1])
+      +- LogicalTableScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalWindowAggregate(group=[{0}], c4=[AUXILIARY_GROUP($2)], EXPR$2=[COUNT($1)], EXPR$3=[AVG($1)], window=[TumblingGroupWindow], properties=[])
++- FlinkLogicalTableSourceScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnWindow3">
+    <Resource name="sql">
+      <![CDATA[SELECT a4, c4, s, COUNT(b4) FROM (SELECT a4, c4, VAR_POP(b4) AS b4, TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, c4, s]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT($3)])
++- LogicalProject(a4=[$0], c4=[$1], s=[TUMBLE_START($2)], b4=[$3])
+   +- LogicalAggregate(group=[{0, 1, 2}], b4=[VAR_POP($3)])
+      +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1])
+         +- LogicalTableScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a4, c4, s, EXPR$3])
++- FlinkLogicalAggregate(group=[{0, 2}], c4=[AUXILIARY_GROUP($1)], EXPR$3=[COUNT($3)])
+   +- FlinkLogicalCalc(select=[a4, c4, w$start AS s, /(-($f2, /(*(CAST($f3), CAST($f3)), $f4)), $f4) AS b4])
+      +- FlinkLogicalWindowAggregate(group=[{0}], c4=[AUXILIARY_GROUP($1)], agg#1=[SUM($4)], agg#2=[SUM($3)], agg#3=[COUNT($3)], window=[TumblingGroupWindow], properties=[w$start, w$end, w$rowtime])
+         +- FlinkLogicalCalc(select=[a4, c4, d4, b4, *(CAST(b4), CAST(b4)) AS $f4])
+            +- FlinkLogicalTableSourceScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnWindow4">
+    <Resource name="sql">
+      <![CDATA[SELECT a4, c4, e, COUNT(b4) FROM (SELECT a4, c4, VAR_POP(b4) AS b4, TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, c4, e]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT($3)])
++- LogicalProject(a4=[$0], c4=[$1], e=[TUMBLE_END($2)], b4=[$3])
+   +- LogicalAggregate(group=[{0, 1, 2}], b4=[VAR_POP($3)])
+      +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1])
+         +- LogicalTableScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a4, c4, e, EXPR$3])
++- FlinkLogicalAggregate(group=[{0, 2}], c4=[AUXILIARY_GROUP($1)], EXPR$3=[COUNT($3)])
+   +- FlinkLogicalCalc(select=[a4, c4, w$end AS e, /(-($f2, /(*(CAST($f3), CAST($f3)), $f4)), $f4) AS b4])
+      +- FlinkLogicalWindowAggregate(group=[{0}], c4=[AUXILIARY_GROUP($1)], agg#1=[SUM($4)], agg#2=[SUM($3)], agg#3=[COUNT($3)], window=[TumblingGroupWindow], properties=[w$start, w$end, w$rowtime])
+         +- FlinkLogicalCalc(select=[a4, c4, d4, b4, *(CAST(b4), CAST(b4)) AS $f4])
+            +- FlinkLogicalTableSourceScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggOnWindow5">
+    <Resource name="sql">
+      <![CDATA[SELECT a4, b4, c4, COUNT(*) FROM (SELECT a4, c4, VAR_POP(b4) AS b4, TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, b4, c4]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()])
++- LogicalProject(a4=[$0], b4=[$3], c4=[$1])
+   +- LogicalAggregate(group=[{0, 1, 2}], b4=[VAR_POP($3)])
+      +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1])
+         +- LogicalTableScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0, 1}], c4=[AUXILIARY_GROUP($2)], EXPR$3=[COUNT()])
++- FlinkLogicalCalc(select=[a4, /(-($f2, /(*(CAST($f3), CAST($f3)), $f4)), $f4) AS b4, c4])
+   +- FlinkLogicalWindowAggregate(group=[{0}], c4=[AUXILIARY_GROUP($1)], agg#1=[SUM($4)], agg#2=[SUM($3)], agg#3=[COUNT($3)], window=[TumblingGroupWindow], properties=[])
+      +- FlinkLogicalCalc(select=[a4, c4, d4, b4, *(CAST(b4), CAST(b4)) AS $f4])
+         +- FlinkLogicalTableSourceScan(table=[[T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithCube">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c1, COUNT(d1) FROM T1 GROUP BY CUBE (a1, b1, c1)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0, 2}, {0}, {1, 2}, {1}, {2}, {}]], EXPR$3=[COUNT($3)])
++- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a1, b1, c1, EXPR$3])
++- FlinkLogicalAggregate(group=[{0, 1, 2, 4}], EXPR$3=[COUNT($3)])
+   +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[2]}, {a1=[$0], b1=[null], c1=[null], d1=[$3], $e=[3]}, {a1=[null], b1=[$1], c1=[$2], d1=[$3], $e=[4]}, {a1=[null], b1=[$1], c1=[null], d1=[$3], $e=[5]}, {a1=[null], b1=[null], c1=[$2], d1=[$3], $e=[6]}, {a1=[null], b1=[null], c1=[null], d1=[$3], $e=[7]}])
+      +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithGroupingSets1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c1, COUNT(d1) FROM T1 GROUP BY GROUPING SETS ((a1, b1), (a1, c1))]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}]], EXPR$3=[COUNT($3)])
++- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a1, b1, c1, EXPR$3])
++- FlinkLogicalAggregate(group=[{0, 1, 2, 4}], EXPR$3=[COUNT($3)])
+   +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[2]}])
+      +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithGroupingSets2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, SUM(b1) AS s FROM T1 GROUP BY GROUPING SETS((a1, c1), (a1), ())]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a1=[$0], s=[$2])
++- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], s=[SUM($2)])
+   +- LogicalProject(a1=[$0], c1=[$2], b1=[$1])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a1, s])
++- FlinkLogicalAggregate(group=[{0, 1, 3}], s=[SUM($2)])
+   +- FlinkLogicalExpand(projects=[{a1=[$0], c1=[$1], b1=[$2], $e=[0]}, {a1=[$0], c1=[null], b1=[$2], $e=[1]}, {a1=[null], c1=[null], b1=[$2], $e=[3]}])
+      +- FlinkLogicalCalc(select=[a1, c1, b1])
+         +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithGroupingSets3">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c1, COUNT(d1) FROM T1 GROUP BY GROUPING SETS ((a1, b1, c1), (a1, b1, d1))]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], EXPR$3=[$4])
++- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2}, {0, 1, 3}]], EXPR$3=[COUNT($3)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a1, b1, c1, EXPR$3])
++- FlinkLogicalAggregate(group=[{0, 2, 3, 4}], b1=[AUXILIARY_GROUP($1)], EXPR$3=[COUNT($5)])
+   +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[null], $e=[1], d1_0=[$3]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[2], d1_0=[$3]}])
+      +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithoutAggCall">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c1 FROM T1 GROUP BY a1, b1, c1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}])
++- LogicalProject(a1=[$0], b1=[$1], c1=[$2])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], b1=[AUXILIARY_GROUP($1)], c1=[AUXILIARY_GROUP($2)])
++- FlinkLogicalCalc(select=[a1, b1, c1])
+   +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithoutReduceGrouping">
+    <Resource name="sql">
+      <![CDATA[SELECT a3, b3, count(c3) FROM T3 GROUP BY a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
++- LogicalProject(a3=[$0], b3=[$1], c3=[$2])
+   +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
++- FlinkLogicalCalc(select=[a3, b3, c3])
+   +- FlinkLogicalTableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggWithRollup">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c1, COUNT(d1) FROM T1 GROUP BY ROLLUP (a1, b1, c1)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0}, {}]], EXPR$3=[COUNT($3)])
++- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a1, b1, c1, EXPR$3])
++- FlinkLogicalAggregate(group=[{0, 1, 2, 4}], EXPR$3=[COUNT($3)])
+   +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}, {a1=[$0], b1=[null], c1=[null], d1=[$3], $e=[3]}, {a1=[null], b1=[null], c1=[null], d1=[$3], $e=[7]}])
+      +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiAggs1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, c1, d1, m, COUNT(*) FROM (SELECT a1, b1, c1, COUNT(d1) AS d1, MAX(d1) AS m FROM T1 GROUP BY a1, b1, c1) t GROUP BY a1, b1, c1, d1, m]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3, 4}], EXPR$5=[COUNT()])
++- LogicalAggregate(group=[{0, 1, 2}], d1=[COUNT($3)], m=[MAX($3)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], b1=[AUXILIARY_GROUP($1)], c1=[AUXILIARY_GROUP($2)], d1=[AUXILIARY_GROUP($3)], m=[AUXILIARY_GROUP($4)], EXPR$5=[COUNT()])
++- FlinkLogicalAggregate(group=[{0}], b1=[AUXILIARY_GROUP($1)], c1=[AUXILIARY_GROUP($2)], d1=[COUNT($3)], m=[MAX($3)])
+   +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiAggs2">
+    <Resource name="sql">
+      <![CDATA[SELECT a3, b3, c, s, a, COUNT(*) FROM (SELECT a3, b3, COUNT(c3) AS c, SUM(d3) AS s, AVG(d3) AS a FROM T3 GROUP BY a3, b3) t GROUP BY a3, b3, c, s, a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2, 3, 4}], EXPR$5=[COUNT()])
++- LogicalAggregate(group=[{0, 1}], c=[COUNT($2)], s=[SUM($3)], a=[AVG($3)])
+   +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0, 1}], c=[AUXILIARY_GROUP($2)], s=[AUXILIARY_GROUP($3)], a=[AUXILIARY_GROUP($4)], EXPR$5=[COUNT()])
++- FlinkLogicalAggregate(group=[{0, 1}], c=[COUNT($2)], s=[SUM($3)], a=[AVG($3)])
+   +- FlinkLogicalTableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiDistinctAggs1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, COUNT(DISTINCT b1), SUM(DISTINCT b1) FROM T1 GROUP BY a1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM(DISTINCT $1)])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)], EXPR$2=[SUM($1)])
++- FlinkLogicalAggregate(group=[{0}], b1=[AUXILIARY_GROUP($1)])
+   +- FlinkLogicalCalc(select=[a1, b1])
+      +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiDistinctAggs2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, d1, COUNT(DISTINCT c1), SUM(DISTINCT b1) FROM T1 GROUP BY a1, d1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[SUM(DISTINCT $3)])
++- LogicalProject(a1=[$0], d1=[$3], c1=[$2], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], d1=[AUXILIARY_GROUP($3)], EXPR$2=[COUNT($2) FILTER $5], EXPR$3=[SUM($1) FILTER $4])
++- FlinkLogicalCalc(select=[a1, b1, c1, d1, =(CASE(=($e, 2:BIGINT), 2:BIGINT, 4:BIGINT), 2) AS $g_2, =(CASE(=($e, 2:BIGINT), 2:BIGINT, 4:BIGINT), 4) AS $g_4])
+   +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[2]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[4]}])
+      +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiDistinctAggs3">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, SUM(DISTINCT b1), MAX(DISTINCT b1), MIN(DISTINCT c1) FROM T1 GROUP BY a1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM(DISTINCT $1)], EXPR$2=[MAX(DISTINCT $1)], EXPR$3=[MIN(DISTINCT $2)])
++- LogicalProject(a1=[$0], b1=[$1], c1=[$2])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[SUM($1) FILTER $3], EXPR$2=[MAX($1) FILTER $3], EXPR$3=[MIN($2) FILTER $4])
++- FlinkLogicalCalc(select=[a1, b1, c1, =(CASE(=($e, 1:BIGINT), 1:BIGINT, 2:BIGINT), 1) AS $g_1, =(CASE(=($e, 1:BIGINT), 1:BIGINT, 2:BIGINT), 2) AS $g_2])
+   +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[2]}])
+      +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiDistinctAggs_WithNonDistinctAgg1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, d1, COUNT(DISTINCT c1), MAX(DISTINCT b1), SUM(b1) FROM T1 GROUP BY a1, d1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[MAX(DISTINCT $3)], EXPR$4=[SUM($3)])
++- LogicalProject(a1=[$0], d1=[$3], c1=[$2], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], d1=[MIN($3) FILTER $7], EXPR$2=[COUNT($2) FILTER $6], EXPR$3=[MAX($1) FILTER $5], EXPR$4=[MIN($4) FILTER $7])
++- FlinkLogicalCalc(select=[a1, b1, c1, d1, EXPR$4, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 1) AS $g_1, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 2) AS $g_2, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 3) AS $g_3])
+   +- FlinkLogicalAggregate(group=[{0, 1, 2, 4}], d1=[AUXILIARY_GROUP($3)], EXPR$4=[SUM($5)])
+      +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1], b1_0=[$1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[2], b1_0=[$1]}, {a1=[$0], b1=[null], c1=[null], d1=[$3], $e=[3], b1_0=[$1]}])
+         +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleAggOnlyConstantGroupKey">
+    <Resource name="sql">
+      <![CDATA[SELECT count(c1) FROM T1 GROUP BY 1, true]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(EXPR$0=[$2])
++- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT($2)])
+   +- LogicalProject($f0=[1], $f1=[true], c1=[$2])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[EXPR$0])
++- FlinkLogicalAggregate(group=[{0}], EXPR$0=[COUNT($2)])
+   +- FlinkLogicalCalc(select=[1 AS $f0, true AS $f1, c1])
+      +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleAggOnTableWithoutUniqueKey">
+    <Resource name="sql">
+      <![CDATA[SELECT a3, b3, count(c3) FROM T3 GROUP BY a3, b3]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
++- LogicalProject(a3=[$0], b3=[$1], c3=[$2])
+   +- LogicalTableScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
++- FlinkLogicalCalc(select=[a3, b3, c3])
+   +- FlinkLogicalTableSourceScan(table=[[T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleAggOnTableWithUniqueKey">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, count(c1) FROM T1 GROUP BY a1, b1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
++- LogicalProject(a1=[$0], b1=[$1], c1=[$2])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], b1=[AUXILIARY_GROUP($1)], EXPR$2=[COUNT($2)])
++- FlinkLogicalCalc(select=[a1, b1, c1])
+   +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleAggOnTableWithUniqueKeys">
+    <Resource name="sql">
+      <![CDATA[SELECT  b2, c2, avg(a2) FROM T2 GROUP BY b2, c2]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[AVG($2)])
++- LogicalProject(b2=[$1], c2=[$2], a2=[$0])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{1}], c2=[AUXILIARY_GROUP($2)], EXPR$2=[AVG($0)])
++- FlinkLogicalTableSourceScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleAggWithConstantGroupKey">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, count(c1) FROM T1 GROUP BY a1, b1, 1, true]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], EXPR$2=[$4])
++- LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$2=[COUNT($4)])
+   +- LogicalProject(a1=[$0], b1=[$1], $f2=[1], $f3=[true], c1=[$2])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], b1=[AUXILIARY_GROUP($1)], EXPR$2=[COUNT($4)])
++- FlinkLogicalCalc(select=[a1, b1, 1 AS $f2, true AS $f3, c1])
+   +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleDistinctAgg1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, COUNT(DISTINCT c1) FROM T1 GROUP BY a1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)])
++- LogicalProject(a1=[$0], c1=[$2])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)])
++- FlinkLogicalAggregate(group=[{0}], c1=[AUXILIARY_GROUP($1)])
+   +- FlinkLogicalCalc(select=[a1, c1])
+      +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleDistinctAgg2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1, COUNT(DISTINCT c1) FROM T1 GROUP BY a1, b1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)])
++- LogicalProject(a1=[$0], b1=[$1], c1=[$2])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], b1=[AUXILIARY_GROUP($1)], EXPR$2=[COUNT($2)])
++- FlinkLogicalAggregate(group=[{0}], b1=[AUXILIARY_GROUP($1)], c1=[AUXILIARY_GROUP($2)])
+   +- FlinkLogicalCalc(select=[a1, b1, c1])
+      +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleDistinctAgg_WithNonDistinctAgg1">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, COUNT(DISTINCT b1), SUM(b1) FROM T1 GROUP BY a1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($1)])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $3], EXPR$2=[MIN($2) FILTER $4])
++- FlinkLogicalCalc(select=[a1, b1, EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+   +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$2=[SUM($3)])
+      +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], $e=[0], b1_0=[$1]}, {a1=[$0], b1=[null], $e=[1], b1_0=[$1]}])
+         +- FlinkLogicalCalc(select=[a1, b1])
+            +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleDistinctAgg_WithNonDistinctAgg2">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, c1, COUNT(DISTINCT b1), SUM(b1) FROM T1 GROUP BY a1, c1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[SUM($2)])
++- LogicalProject(a1=[$0], c1=[$2], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], c1=[AUXILIARY_GROUP($1)], EXPR$2=[COUNT($2) FILTER $4], EXPR$3=[MIN($3) FILTER $5])
++- FlinkLogicalCalc(select=[a1, c1, b1, EXPR$3, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+   +- FlinkLogicalAggregate(group=[{0, 2, 3}], c1=[AUXILIARY_GROUP($1)], EXPR$3=[SUM($4)])
+      +- FlinkLogicalExpand(projects=[{a1=[$0], c1=[$1], b1=[$2], $e=[0], b1_0=[$2]}, {a1=[$0], c1=[$1], b1=[null], $e=[1], b1_0=[$2]}])
+         +- FlinkLogicalCalc(select=[a1, c1, b1])
+            +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleDistinctAgg_WithNonDistinctAgg3">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, COUNT(DISTINCT c1), SUM(b1) FROM T1 GROUP BY a1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($2)])
++- LogicalProject(a1=[$0], c1=[$2], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $3], EXPR$2=[MIN($2) FILTER $4])
++- FlinkLogicalCalc(select=[a1, c1, EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+   +- FlinkLogicalAggregate(group=[{0, 1, 3}], EXPR$2=[SUM($2)])
+      +- FlinkLogicalExpand(projects=[{a1=[$0], c1=[$1], b1=[$2], $e=[0]}, {a1=[$0], c1=[null], b1=[$2], $e=[1]}])
+         +- FlinkLogicalCalc(select=[a1, c1, b1])
+            +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleDistinctAgg_WithNonDistinctAgg4">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, d1, COUNT(DISTINCT c1), SUM(b1) FROM T1 GROUP BY a1, d1]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[SUM($3)])
++- LogicalProject(a1=[$0], d1=[$3], c1=[$2], b1=[$1])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], d1=[MIN($2) FILTER $5], EXPR$2=[COUNT($1) FILTER $4], EXPR$3=[MIN($3) FILTER $5])
++- FlinkLogicalCalc(select=[a1, c1, d1, EXPR$3, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+   +- FlinkLogicalAggregate(group=[{0, 2, 4}], d1=[AUXILIARY_GROUP($3)], EXPR$3=[SUM($1)])
+      +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}])
+         +- FlinkLogicalTableSourceScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml
new file mode 100644
index 0000000..3b635b3
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml
@@ -0,0 +1,377 @@
+<?xml version="1.0" ?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<Root>
+  <TestCase name="testEmptyGroupKey_WithMoreThanOneAggCalls1">
+    <Resource name="sql">
+      <![CDATA[SELECT 1 FROM (SELECT SUM(a1), COUNT(*) FROM T1) t]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(EXPR$0=[1])
++- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT()])
+   +- LogicalProject(a1=[$0])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
++- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+   +- LogicalCalc(expr#0..3=[{inputs}], a1=[$t0])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testEmptyGroupKey_WithMoreThanOneAggCalls2">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM T2 WHERE EXISTS (SELECT SUM(a1), COUNT(*) FROM T1)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3])
++- LogicalFilter(condition=[EXISTS({
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT()])
+  LogicalProject(a1=[$0])
+    LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+})])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalJoin(condition=[$4], joinType=[semi])
+:- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
++- LogicalCalc(expr#0=[{inputs}], expr#1=[IS NOT NULL($t0)], $f0=[$t1])
+   +- LogicalAggregate(group=[{}], m=[MIN($0)])
+      +- LogicalCalc(expr#0=[{inputs}], expr#1=[true], i=[$t1])
+         +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+            +- LogicalCalc(expr#0..3=[{inputs}], a1=[$t0])
+               +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testEmptyGroupKey_WithMoreThanOneAggCalls3">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM T2 WHERE EXISTS (SELECT SUM(a1), COUNT(*) FROM T1 WHERE 1=2)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3])
++- LogicalFilter(condition=[EXISTS({
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT()])
+  LogicalProject(a1=[$0])
+    LogicalFilter(condition=[=(1, 2)])
+      LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+})])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalJoin(condition=[$4], joinType=[semi])
+:- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
++- LogicalCalc(expr#0=[{inputs}], expr#1=[IS NOT NULL($t0)], $f0=[$t1])
+   +- LogicalAggregate(group=[{}], m=[MIN($0)])
+      +- LogicalCalc(expr#0=[{inputs}], expr#1=[true], i=[$t1])
+         +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+            +- LogicalValues(tuples=[[]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testEmptyGroupKey_WithOneAggCall1">
+    <Resource name="sql">
+      <![CDATA[SELECT 1 FROM (SELECT SUM(a1) FROM T1) t]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(EXPR$0=[1])
++- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+   +- LogicalProject(a1=[$0])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1])
++- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+   +- LogicalCalc(expr#0..3=[{inputs}], a1=[$t0])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testEmptyGroupKey_WithOneAggCall2">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM T2 WHERE EXISTS (SELECT COUNT(*) FROM T1)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3])
++- LogicalFilter(condition=[EXISTS({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+})])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalJoin(condition=[$4], joinType=[semi])
+:- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
++- LogicalCalc(expr#0=[{inputs}], expr#1=[IS NOT NULL($t0)], $f0=[$t1])
+   +- LogicalAggregate(group=[{}], m=[MIN($0)])
+      +- LogicalCalc(expr#0=[{inputs}], expr#1=[true], i=[$t1])
+         +- LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+            +- LogicalCalc(expr#0..3=[{inputs}], expr#4=[0], $f0=[$t4])
+               +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testEmptyGroupKey_WithOneAggCall3">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM T2 WHERE EXISTS (SELECT COUNT(*) FROM T1 WHERE 1=2)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3])
++- LogicalFilter(condition=[EXISTS({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalFilter(condition=[=(1, 2)])
+      LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+})])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalJoin(condition=[$4], joinType=[semi])
+:- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
++- LogicalCalc(expr#0=[{inputs}], expr#1=[IS NOT NULL($t0)], $f0=[$t1])
+   +- LogicalAggregate(group=[{}], m=[MIN($0)])
+      +- LogicalCalc(expr#0=[{inputs}], expr#1=[true], i=[$t1])
+         +- LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+            +- LogicalValues(tuples=[[]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneAuxGroupAggCall_WithFilter1">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a1, c1 FROM
+ (SELECT a1, b1, COUNT(c1) as c1, SUM(d1) as d1 FROM T1 GROUP BY a1, b1) t
+WHERE c1 > 10
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a1=[$0], c1=[$2])
++- LogicalFilter(condition=[>($2, 10)])
+   +- LogicalAggregate(group=[{0, 1}], c1=[COUNT($2)], d1=[SUM($3)])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalCalc(expr#0..1=[{inputs}], expr#2=[10], expr#3=[>($t1, $t2)], proj#0..1=[{exprs}], $condition=[$t3])
++- LogicalAggregate(group=[{0}], c1=[COUNT($2)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneAuxGroupAggCall_WithFilter2">
+    <Resource name="sql">
+      <![CDATA[
+SELECT c1, a1 FROM
+ (SELECT a1, b1, COUNT(c1) as c1, SUM(d1) as d1 FROM T1 GROUP BY a1, b1) t
+WHERE c1 > 10
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(c1=[$2], a1=[$0])
++- LogicalFilter(condition=[>($2, 10)])
+   +- LogicalAggregate(group=[{0, 1}], c1=[COUNT($2)], d1=[SUM($3)])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalCalc(expr#0..1=[{inputs}], expr#2=[10], expr#3=[>($t1, $t2)], c1=[$t1], a1=[$t0], $condition=[$t3])
++- LogicalAggregate(group=[{0}], c1=[COUNT($2)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneAuxGroupAggCall_WithoutFilter1">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a1, c1 FROM
+ (SELECT a1, b1, COUNT(c1) as c1, SUM(d1) as d1 FROM T1 GROUP BY a1, b1) t
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a1=[$0], c1=[$2])
++- LogicalAggregate(group=[{0, 1}], c1=[COUNT($2)], d1=[SUM($3)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalAggregate(group=[{0}], c1=[COUNT($2)])
++- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneAuxGroupAggCall_WithoutFilter2">
+    <Resource name="sql">
+      <![CDATA[
+SELECT c1, a1 FROM
+ (SELECT a1, b1, COUNT(c1) as c1, SUM(d1) as d1 FROM T1 GROUP BY a1, b1) t
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(c1=[$2], a1=[$0])
++- LogicalAggregate(group=[{0, 1}], c1=[COUNT($2)], d1=[SUM($3)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalCalc(expr#0..1=[{inputs}], c1=[$t1], a1=[$t0])
++- LogicalAggregate(group=[{0}], c1=[COUNT($2)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneRegularAggCall_WithFilter1">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a2, b2, d2 FROM
+ (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+WHERE d2 > 0
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a2=[$0], b2=[$1], d2=[$3])
++- LogicalFilter(condition=[>($3, 0)])
+   +- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)])
+      +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalCalc(expr#0..2=[{inputs}], expr#3=[0], expr#4=[>($t2, $t3)], proj#0..2=[{exprs}], $condition=[$t4])
++- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneRegularAggCall_WithFilter2">
+    <Resource name="sql">
+      <![CDATA[
+SELECT b2, a2, d2 FROM
+ (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+WHERE d2 > 0
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(b2=[$1], a2=[$0], d2=[$3])
++- LogicalFilter(condition=[>($3, 0)])
+   +- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)])
+      +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalCalc(expr#0..2=[{inputs}], expr#3=[0], expr#4=[>($t2, $t3)], b2=[$t1], a2=[$t0], d2=[$t2], $condition=[$t4])
++- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneRegularAggCall_WithoutFilter1">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a2, b2, d2 FROM
+ (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a2=[$0], b2=[$1], d2=[$3])
++- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], d2=[SUM($3)])
++- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneRegularAggCall_WithoutFilter2">
+    <Resource name="sql">
+      <![CDATA[
+SELECT b2, a2, d2 FROM
+ (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(b2=[$1], a2=[$0], d2=[$3])
++- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalCalc(expr#0..2=[{inputs}], b2=[$t1], a2=[$t0], d2=[$t2])
++- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneRegularAggCall_WithoutFilter3">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a2 as a, b2, d2 FROM
+ (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], b2=[$1], d2=[$3])
++- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalCalc(expr#0..2=[{inputs}], proj#0..2=[{exprs}])
++- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/FlinkAggregateInnerJoinTransposeRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/FlinkAggregateInnerJoinTransposeRuleTest.xml
new file mode 100644
index 0000000..31daeed
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/FlinkAggregateInnerJoinTransposeRuleTest.xml
@@ -0,0 +1,257 @@
+<?xml version="1.0" ?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<Root>
+  <TestCase name="testAggregateWithAuxGroup_JoinKeyIsNotUnique1">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a2, b2, c2, SUM(a) FROM (SELECT * FROM T2, T WHERE a2 = a) GROUP BY a2, b2, c2
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[SUM($3)])
++- LogicalProject(a2=[$0], b2=[$1], c2=[$2], a=[$3])
+   +- LogicalFilter(condition=[=($0, $3)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(a2=[$1], b2=[$0], c2=[$2], EXPR$3=[$3])
++- LogicalProject(b2=[$1], a2=[$0], c2=[$2], EXPR$3=[$4])
+   +- LogicalJoin(condition=[=($0, $3)], joinType=[inner])
+      :- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+      +- LogicalAggregate(group=[{0}], EXPR$3=[SUM($0)])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggregateWithAuxGroup_JoinKeyIsNotUnique2">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a2, b2, c, SUM(a) FROM (SELECT * FROM T2, T WHERE a2 = a) GROUP BY a2, b2, c
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[SUM($3)])
++- LogicalProject(a2=[$0], b2=[$1], c=[$5], a=[$3])
+   +- LogicalFilter(condition=[=($0, $3)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(a2=[$2], b2=[$0], c=[$1], EXPR$3=[$3])
++- LogicalProject(b2=[$1], c=[$3], a2=[$0], EXPR$3=[$4])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+      :- LogicalProject(a2=[$0], b2=[$1])
+      :  +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+      +- LogicalAggregate(group=[{0, 2}], EXPR$3=[SUM($0)])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggregateWithAuxGroup_JoinKeyIsUnique1">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a2, b2, c2, SUM(a) FROM (SELECT * FROM T2, T WHERE b2 = b) GROUP BY a2, b2, c2
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[SUM($3)])
++- LogicalProject(a2=[$0], b2=[$1], c2=[$2], a=[$3])
+   +- LogicalFilter(condition=[=($1, $4)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(a2=[$1], b2=[$0], c2=[$2], EXPR$3=[$3])
++- LogicalProject(b2=[$1], a2=[$0], c2=[$2], EXPR$3=[$4])
+   +- LogicalJoin(condition=[=($1, $3)], joinType=[inner])
+      :- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+      +- LogicalAggregate(group=[{1}], EXPR$3=[SUM($0)])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggregateWithAuxGroup_JoinKeyIsUnique2">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a2, b2, c, SUM(a) FROM (SELECT * FROM T2, T WHERE b2 = b) GROUP BY a2, b2, c
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[SUM($3)])
++- LogicalProject(a2=[$0], b2=[$1], c=[$5], a=[$3])
+   +- LogicalFilter(condition=[=($1, $4)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(a2=[$2], b2=[$0], c=[$1], EXPR$3=[$3])
++- LogicalProject(b2=[$1], c=[$3], a2=[$0], EXPR$3=[$4])
+   +- LogicalJoin(condition=[=($1, $2)], joinType=[inner])
+      :- LogicalProject(a2=[$0], b2=[$1])
+      :  +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+      +- LogicalAggregate(group=[{1, 2}], EXPR$3=[SUM($0)])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPushAggThroughJoinWithUniqueJoinKey">
+    <Resource name="sql">
+      <![CDATA[
+WITH T1 AS (SELECT a AS a1, COUNT(b) AS b1 FROM T GROUP BY a),
+     T2 AS (SELECT COUNT(a) AS a2, b AS b2 FROM T GROUP BY b)
+SELECT MIN(a1), MIN(b1), MIN(a2), MIN(b2), a, b, COUNT(c) FROM
+  (SELECT * FROM T1, T2, T WHERE a1 = b2 AND a1 = a) t GROUP BY a, b
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4], EXPR$3=[$5], a=[$0], b=[$1], EXPR$6=[$6])
++- LogicalAggregate(group=[{0, 1}], EXPR$0=[MIN($2)], EXPR$1=[MIN($3)], EXPR$2=[MIN($4)], EXPR$3=[MIN($5)], EXPR$6=[COUNT($6)])
+   +- LogicalProject(a=[$4], b=[$5], a1=[$0], b1=[$1], a2=[$2], b2=[$3], c=[$6])
+      +- LogicalFilter(condition=[AND(=($0, $3), =($0, $4))])
+         +- LogicalJoin(condition=[true], joinType=[inner])
+            :- LogicalJoin(condition=[true], joinType=[inner])
+            :  :- LogicalAggregate(group=[{0}], b1=[COUNT($1)])
+            :  :  +- LogicalProject(a1=[$0], b=[$1])
+            :  :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+            :  +- LogicalProject(a2=[$1], b2=[$0])
+            :     +- LogicalAggregate(group=[{0}], a2=[COUNT($1)])
+            :        +- LogicalProject(b2=[$1], a=[$0])
+            :           +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+            +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4], EXPR$3=[$5], a=[$0], b=[$1], EXPR$6=[$6])
++- LogicalProject(a=[$4], b=[$5], a0=[$0], b1=[$1], a2=[$2], b2=[$3], EXPR$6=[$6])
+   +- LogicalJoin(condition=[=($0, $4)], joinType=[inner])
+      :- LogicalJoin(condition=[=($0, $3)], joinType=[inner])
+      :  :- LogicalAggregate(group=[{0}], b1=[COUNT($1)])
+      :  :  +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+      :  +- LogicalProject(a2=[$1], b2=[$0])
+      :     +- LogicalAggregate(group=[{1}], a2=[COUNT($0)])
+      :        +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0, 1}], EXPR$6=[COUNT($2)])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSomeAggCallColumnsAndJoinConditionColumnsIsSame">
+    <Resource name="sql">
+      <![CDATA[
+SELECT MIN(a2), MIN(b2), a, b, COUNT(c2) FROM
+    (SELECT * FROM T2, T WHERE b2 = a) t GROUP BY a, b
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(EXPR$0=[$2], EXPR$1=[$3], a=[$0], b=[$1], EXPR$4=[$4])
++- LogicalAggregate(group=[{0, 1}], EXPR$0=[MIN($2)], EXPR$1=[MIN($3)], EXPR$4=[COUNT($4)])
+   +- LogicalProject(a=[$3], b=[$4], a2=[$0], b2=[$1], c2=[$2])
+      +- LogicalFilter(condition=[=($1, $3)])
+         +- LogicalJoin(condition=[true], joinType=[inner])
+            :- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+            +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(EXPR$0=[$2], EXPR$1=[$3], a=[$0], b=[$1], EXPR$4=[$4])
++- LogicalProject(a=[$3], b=[$4], a2=[$1], b2=[$0], $f6=[*($2, $5)])
+   +- LogicalJoin(condition=[=($0, $3)], joinType=[inner])
+      :- LogicalProject(b2=[$1], a2=[$0], $f2=[CAST(CASE(IS NOT NULL($2), 1, 0)):BIGINT NOT NULL])
+      :  +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2)]]])
+      +- LogicalAggregate(group=[{0, 1}], agg#0=[COUNT()])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPushCountAggThroughJoinOverUniqueColumn">
+    <Resource name="sql">
+      <![CDATA[SELECT COUNT(A.a) FROM (SELECT DISTINCT a FROM T) AS A JOIN T AS B ON A.a=B.a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)])
++- LogicalProject(a=[$0])
+   +- LogicalJoin(condition=[=($0, $1)], joinType=[inner])
+      :- LogicalAggregate(group=[{0}])
+      :  +- LogicalProject(a=[$0])
+      :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+      +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[$SUM0($4)])
++- LogicalProject(a=[$0], $f1=[$1], a0=[$2], $f10=[$3], $f4=[*($1, $3)])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+      :- LogicalProject(a=[$0], $f1=[CAST(CASE(IS NOT NULL($0), 1, 0)):BIGINT NOT NULL])
+      :  +- LogicalAggregate(group=[{0}])
+      :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], agg#0=[COUNT()])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPushSumAggThroughJoinOverUniqueColumn">
+    <Resource name="sql">
+      <![CDATA[SELECT SUM(A.a) FROM (SELECT DISTINCT a FROM T) AS A JOIN T AS B ON A.a=B.a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
++- LogicalProject(a=[$0])
+   +- LogicalJoin(condition=[=($0, $1)], joinType=[inner])
+      :- LogicalAggregate(group=[{0}])
+      :  +- LogicalProject(a=[$0])
+      :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+      +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[SUM($3)])
++- LogicalProject(a=[$0], a0=[$1], $f1=[$2], $f3=[CAST(*($0, $2)):INTEGER])
+   +- LogicalJoin(condition=[=($0, $1)], joinType=[inner])
+      :- LogicalAggregate(group=[{0}])
+      :  +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], agg#0=[COUNT()])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/FlinkAggregateOuterJoinTransposeRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/FlinkAggregateOuterJoinTransposeRuleTest.xml
new file mode 100644
index 0000000..1f0abf2
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/FlinkAggregateOuterJoinTransposeRuleTest.xml
@@ -0,0 +1,267 @@
+<?xml version="1.0" ?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<Root>
+  <TestCase name="testPushCountAggThroughJoinOverUniqueColumn">
+    <Resource name="sql">
+      <![CDATA[SELECT COUNT(A.a) FROM (SELECT DISTINCT a FROM T) AS A JOIN T AS B ON A.a=B.a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)])
++- LogicalProject(a=[$0])
+   +- LogicalJoin(condition=[=($0, $1)], joinType=[inner])
+      :- LogicalAggregate(group=[{0}])
+      :  +- LogicalProject(a=[$0])
+      :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[$SUM0($4)])
++- LogicalProject(a=[$0], $f1=[$1], a0=[$2], $f10=[$3], $f4=[*($1, $3)])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+      :- LogicalProject(a=[$0], $f1=[CAST(CASE(IS NOT NULL($0), 1, 0)):BIGINT NOT NULL])
+      :  +- LogicalAggregate(group=[{0}])
+      :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalAggregate(group=[{0}], agg#0=[COUNT()])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPushCountAggThroughLeftJoinAndGroupByLeft">
+    <Resource name="sql">
+      <![CDATA[SELECT COUNT(B.b) FROM (SELECT a FROM T) AS A LEFT OUTER JOIN T AS B ON A.a=B.a GROUP BY A.a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(EXPR$0=[$1])
++- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($1)])
+   +- LogicalProject(a=[$0], b=[$2])
+      +- LogicalJoin(condition=[=($0, $1)], joinType=[left])
+         :- LogicalProject(a=[$0])
+         :  +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(EXPR$0=[$1])
++- LogicalProject(a=[$0], $f4=[*($1, CASE(IS NULL($3), 0, CAST($3):BIGINT NOT NULL))])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[left])
+      :- LogicalAggregate(group=[{0}], agg#0=[COUNT()])
+      :  +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($1)])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPushCountAggThroughLeftJoinOverUniqueColumnAndGroupByLeft">
+    <Resource name="sql">
+      <![CDATA[SELECT COUNT(B.b) FROM (SELECT DISTINCT a FROM T) AS A LEFT OUTER JOIN T AS B ON A.a=B.a GROUP BY A.a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(EXPR$0=[$1])
++- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($1)])
+   +- LogicalProject(a=[$0], b=[$2])
+      +- LogicalJoin(condition=[=($0, $1)], joinType=[left])
+         :- LogicalAggregate(group=[{0}])
+         :  +- LogicalProject(a=[$0])
+         :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(EXPR$0=[$1])
++- LogicalProject(a=[$0], $f3=[CASE(IS NULL($2), 0, CAST($2):BIGINT NOT NULL)])
+   +- LogicalJoin(condition=[=($0, $1)], joinType=[left])
+      :- LogicalAggregate(group=[{0}])
+      :  +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($1)])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPushCountAggThroughLeftJoinOverUniqueColumn">
+    <Resource name="sql">
+      <![CDATA[SELECT COUNT(A.a) FROM (SELECT DISTINCT a FROM T) AS A LEFT OUTER JOIN T AS B ON A.a=B.a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)])
++- LogicalProject(a=[$0])
+   +- LogicalJoin(condition=[=($0, $1)], joinType=[left])
+      :- LogicalAggregate(group=[{0}])
+      :  +- LogicalProject(a=[$0])
+      :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[$SUM0($4)])
++- LogicalProject(a=[$0], $f1=[$1], a0=[$2], $f10=[$3], $f4=[*($1, CASE(IS NULL($3), 0, CAST($3):BIGINT NOT NULL))])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[left])
+      :- LogicalProject(a=[$0], $f1=[CAST(CASE(IS NOT NULL($0), 1, 0)):BIGINT NOT NULL])
+      :  +- LogicalAggregate(group=[{0}])
+      :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalAggregate(group=[{0}], agg#0=[COUNT()])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPushSumAggThroughLeftJoinOverUniqueColumn">
+    <Resource name="sql">
+      <![CDATA[SELECT SUM(A.a) FROM (SELECT DISTINCT a FROM T) AS A LEFT OUTER JOIN T AS B ON A.a=B.a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
++- LogicalProject(a=[$0])
+   +- LogicalJoin(condition=[=($0, $1)], joinType=[left])
+      :- LogicalAggregate(group=[{0}])
+      :  +- LogicalProject(a=[$0])
+      :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[SUM($3)])
++- LogicalProject(a=[$0], a0=[$1], $f1=[$2], $f3=[CAST(*($0, $2)):INTEGER])
+   +- LogicalJoin(condition=[=($0, $1)], joinType=[left])
+      :- LogicalAggregate(group=[{0}])
+      :  +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalAggregate(group=[{0}], agg#0=[COUNT()])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPushCountAggThroughLeftJoinOverUniqueColumnAndGroupByRight">
+    <Resource name="sql">
+      <![CDATA[SELECT COUNT(B.b) FROM (SELECT DISTINCT a FROM T) AS A LEFT OUTER JOIN T AS B ON A.a=B.a GROUP BY B.a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(EXPR$0=[$1])
++- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($1)])
+   +- LogicalProject(a0=[$1], b=[$2])
+      +- LogicalJoin(condition=[=($0, $1)], joinType=[left])
+         :- LogicalAggregate(group=[{0}])
+         :  +- LogicalProject(a=[$0])
+         :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(EXPR$0=[$1])
++- LogicalAggregate(group=[{1}], EXPR$0=[$SUM0($3)])
+   +- LogicalProject(a=[$0], a0=[$1], EXPR$0=[$2], $f3=[CASE(IS NULL($2), 0, CAST($2):BIGINT NOT NULL)])
+      +- LogicalJoin(condition=[=($0, $1)], joinType=[left])
+         :- LogicalAggregate(group=[{0}])
+         :  +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+         +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($1)])
+            +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPushCountAggThroughRightJoin">
+    <Resource name="sql">
+      <![CDATA[SELECT COUNT(B.b) FROM T AS B RIGHT OUTER JOIN (SELECT a FROM T) AS A ON A.a=B.a GROUP BY A.a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(EXPR$0=[$1])
++- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($1)])
+   +- LogicalProject(a0=[$4], b=[$1])
+      +- LogicalJoin(condition=[=($4, $0)], joinType=[right])
+         :- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+         +- LogicalProject(a=[$0])
+            +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(EXPR$0=[$1])
++- LogicalProject(a0=[$2], $f4=[*(CASE(IS NULL($1), 0, CAST($1):BIGINT NOT NULL), $3)])
+   +- LogicalJoin(condition=[=($2, $0)], joinType=[right])
+      :- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($1)])
+      :  +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalAggregate(group=[{0}], agg#0=[COUNT()])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPushSumAggThroughJoinOverUniqueColumn">
+    <Resource name="sql">
+      <![CDATA[SELECT SUM(A.a) FROM (SELECT DISTINCT a FROM T) AS A JOIN T AS B ON A.a=B.a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
++- LogicalProject(a=[$0])
+   +- LogicalJoin(condition=[=($0, $1)], joinType=[inner])
+      :- LogicalAggregate(group=[{0}])
+      :  +- LogicalProject(a=[$0])
+      :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[SUM($3)])
++- LogicalProject(a=[$0], a0=[$1], $f1=[$2], $f3=[CAST(*($0, $2)):INTEGER])
+   +- LogicalJoin(condition=[=($0, $1)], joinType=[inner])
+      :- LogicalAggregate(group=[{0}])
+      :  +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalAggregate(group=[{0}], agg#0=[COUNT()])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPushCountAllAggThroughLeftJoinOverUniqueColumn">
+    <Resource name="sql">
+      <![CDATA[SELECT COUNT(*) FROM (SELECT DISTINCT a FROM T) AS A LEFT OUTER JOIN T AS B ON A.a=B.a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
++- LogicalProject($f0=[0])
+   +- LogicalJoin(condition=[=($0, $1)], joinType=[left])
+      :- LogicalAggregate(group=[{0}])
+      :  +- LogicalProject(a=[$0])
+      :     +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[$SUM0($3)])
++- LogicalProject(a=[$0], a0=[$1], EXPR$0=[$2], $f3=[CASE(IS NULL($2), 1, CAST($2):BIGINT NOT NULL)])
+   +- LogicalJoin(condition=[=($0, $1)], joinType=[left])
+      :- LogicalAggregate(group=[{0}])
+      :  +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+      +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
+         +- LogicalTableScan(table=[[T, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRuleTest.xml
new file mode 100644
index 0000000..a6e8987
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRuleTest.xml
@@ -0,0 +1,528 @@
+<?xml version="1.0" ?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<Root>
+  <TestCase name="testAggRemove_Count">
+    <Resource name="sql">
+      <![CDATA[SELECT a, COUNT(c) FROM MyTable2 GROUP BY a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)])
++- LogicalProject(a=[$0], c=[$2])
+   +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)])
++- FlinkLogicalCalc(select=[a, c])
+   +- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_CountStar">
+    <Resource name="sql">
+      <![CDATA[SELECT a, COUNT(*) FROM MyTable2 GROUP BY a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT()])
++- LogicalProject(a=[$0])
+   +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT()])
++- FlinkLogicalCalc(select=[a])
+   +- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_Cube">
+    <Resource name="sql">
+      <![CDATA[SELECT a, SUM(b) AS s FROM MyTable3 GROUP BY CUBE(a, c, d)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], s=[$3])
++- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0, 2}, {0}, {1, 2}, {1}, {2}, {}]], s=[SUM($3)])
+   +- LogicalProject(a=[$0], c=[$2], d=[$3], b=[$1])
+      +- LogicalTableScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, s])
++- FlinkLogicalAggregate(group=[{0, 1, 2, 4}], s=[SUM($3)])
+   +- FlinkLogicalExpand(projects=[{a=[$0], c=[$1], d=[$2], b=[$3], $e=[0]}, {a=[$0], c=[$1], d=[null], b=[$3], $e=[1]}, {a=[$0], c=[null], d=[$2], b=[$3], $e=[2]}, {a=[$0], c=[null], d=[null], b=[$3], $e=[3]}, {a=[null], c=[$1], d=[$2], b=[$3], $e=[4]}, {a=[null], c=[$1], d=[null], b=[$3], $e=[5]}, {a=[null], c=[null], d=[$2], b=[$3], $e=[6]}, {a=[null], c=[null], d=[null], b=[$3], $e=[7]}])
+      +- FlinkLogicalCalc(select=[a, c, d, b])
+         +- FlinkLogicalTableSourceScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_GroupKeyIsNotUnique">
+    <Resource name="sql">
+      <![CDATA[SELECT a, MAX(c) from MyTable1 GROUP BY a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[MAX($1)])
++- LogicalProject(a=[$0], c=[$2])
+   +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[MAX($1)])
++- FlinkLogicalCalc(select=[a, c])
+   +- FlinkLogicalTableSourceScan(table=[[MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_GroupSets1">
+    <Resource name="sql">
+      <![CDATA[SELECT a, SUM(b) AS s FROM MyTable3 GROUP BY GROUPING SETS((a, c), (a, d))]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], s=[$3])
++- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}]], s=[SUM($3)])
+   +- LogicalProject(a=[$0], c=[$2], d=[$3], b=[$1])
+      +- LogicalTableScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, b AS s])
++- FlinkLogicalExpand(projects=[{a=[$0], c=[$1], d=[null], b=[$3], $e=[1]}, {a=[$0], c=[null], d=[$2], b=[$3], $e=[2]}])
+   +- FlinkLogicalCalc(select=[a, c, d, b])
+      +- FlinkLogicalTableSourceScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_GroupSets2">
+    <Resource name="sql">
+      <![CDATA[SELECT a, SUM(b) AS s FROM MyTable3 GROUP BY GROUPING SETS((a, c), (a), ())]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], s=[$2])
++- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], s=[SUM($2)])
+   +- LogicalProject(a=[$0], c=[$2], b=[$1])
+      +- LogicalTableScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, s])
++- FlinkLogicalAggregate(group=[{0, 1, 3}], s=[SUM($2)])
+   +- FlinkLogicalExpand(projects=[{a=[$0], c=[$1], b=[$2], $e=[0]}, {a=[$0], c=[null], b=[$2], $e=[1]}, {a=[null], c=[null], b=[$2], $e=[3]}])
+      +- FlinkLogicalCalc(select=[a, c, b])
+         +- FlinkLogicalTableSourceScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_MultiDistinctAggs1">
+    <Resource name="sql">
+      <![CDATA[SELECT a, COUNT(DISTINCT b), SUM(DISTINCT b) FROM MyTable2 GROUP BY a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM(DISTINCT $1)])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)], EXPR$2=[SUM($1)])
++- FlinkLogicalCalc(select=[a, b])
+   +- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_MultiDistinctAggs2">
+    <Resource name="sql">
+      <![CDATA[SELECT a, COUNT(DISTINCT c), SUM(DISTINCT b) FROM MyTable3 GROUP BY a, d]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], EXPR$1=[$2], EXPR$2=[$3])
++- LogicalAggregate(group=[{0, 1}], EXPR$1=[COUNT(DISTINCT $2)], EXPR$2=[SUM(DISTINCT $3)])
+   +- LogicalProject(a=[$0], d=[$3], c=[$2], b=[$1])
+      +- LogicalTableScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2])
++- FlinkLogicalAggregate(group=[{0}], d=[AUXILIARY_GROUP($1)], EXPR$1=[COUNT($2) FILTER $4], EXPR$2=[SUM($3) FILTER $5])
+   +- FlinkLogicalCalc(select=[a, d, c, b, =(CASE(=($e, 1:BIGINT), 1:BIGINT, 2:BIGINT), 1) AS $g_1, =(CASE(=($e, 1:BIGINT), 1:BIGINT, 2:BIGINT), 2) AS $g_2])
+      +- FlinkLogicalExpand(projects=[{a=[$0], d=[$1], c=[$2], b=[null], $e=[1]}, {a=[$0], d=[$1], c=[null], b=[$3], $e=[2]}])
+         +- FlinkLogicalCalc(select=[a, d, c, b])
+            +- FlinkLogicalTableSourceScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_MultiDistinctAggs3">
+    <Resource name="sql">
+      <![CDATA[SELECT a, SUM(DISTINCT b), MAX(DISTINCT b), MIN(DISTINCT c) FROM MyTable2 GROUP BY a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM(DISTINCT $1)], EXPR$2=[MAX(DISTINCT $1)], EXPR$3=[MIN(DISTINCT $2)])
++- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, b AS EXPR$1, b AS EXPR$2, c AS EXPR$3])
++- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_MultiDistinctAggs_WithNonDistinctAgg1">
+    <Resource name="sql">
+      <![CDATA[SELECT a, COUNT(DISTINCT c), SUM(b) FROM MyTable3 GROUP BY a, d]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], EXPR$1=[$2], EXPR$2=[$3])
++- LogicalAggregate(group=[{0, 1}], EXPR$1=[COUNT(DISTINCT $2)], EXPR$2=[SUM($3)])
+   +- LogicalProject(a=[$0], d=[$3], c=[$2], b=[$1])
+      +- LogicalTableScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2])
++- FlinkLogicalAggregate(group=[{0}], d=[AUXILIARY_GROUP($1)], EXPR$1=[COUNT($2) FILTER $4], EXPR$2=[MIN($3) FILTER $5])
+   +- FlinkLogicalCalc(select=[a, d, c, b AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+      +- FlinkLogicalExpand(projects=[{a=[$0], d=[$1], c=[$2], b=[$3], $e=[0]}, {a=[$0], d=[$1], c=[null], b=[$3], $e=[1]}])
+         +- FlinkLogicalCalc(select=[a, d, c, b])
+            +- FlinkLogicalTableSourceScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_Rollup">
+    <Resource name="sql">
+      <![CDATA[SELECT a, SUM(b) AS s FROM MyTable3 GROUP BY ROLLUP(a, c, d)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], s=[$3])
++- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0}, {}]], s=[SUM($3)])
+   +- LogicalProject(a=[$0], c=[$2], d=[$3], b=[$1])
+      +- LogicalTableScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, s])
++- FlinkLogicalAggregate(group=[{0, 1, 2, 4}], s=[SUM($3)])
+   +- FlinkLogicalExpand(projects=[{a=[$0], c=[$1], d=[$2], b=[$3], $e=[0]}, {a=[$0], c=[$1], d=[null], b=[$3], $e=[1]}, {a=[$0], c=[null], d=[null], b=[$3], $e=[3]}, {a=[null], c=[null], d=[null], b=[$3], $e=[7]}])
+      +- FlinkLogicalCalc(select=[a, c, d, b])
+         +- FlinkLogicalTableSourceScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_SingleDistinctAgg1">
+    <Resource name="sql">
+      <![CDATA[SELECT a, COUNT(DISTINCT c) FROM MyTable2 GROUP BY a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)])
++- LogicalProject(a=[$0], c=[$2])
+   +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)])
++- FlinkLogicalCalc(select=[a, c])
+   +- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_SingleDistinctAgg2">
+    <Resource name="sql">
+      <![CDATA[SELECT a, COUNT(DISTINCT c) FROM MyTable2 GROUP BY a, b]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], EXPR$1=[$2])
++- LogicalAggregate(group=[{0, 1}], EXPR$1=[COUNT(DISTINCT $2)])
+   +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, EXPR$1])
++- FlinkLogicalAggregate(group=[{0}], b=[AUXILIARY_GROUP($1)], EXPR$1=[COUNT($2)])
+   +- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_SingleDistinctAgg_WithNonDistinctAgg1">
+    <Resource name="sql">
+      <![CDATA[SELECT a, COUNT(DISTINCT b), SUM(b) FROM MyTable2 GROUP BY a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($1)])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $3], EXPR$2=[MIN($2) FILTER $4])
++- FlinkLogicalCalc(select=[a, b, b_0 AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+   +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], $e=[0], b_0=[$1]}, {a=[$0], b=[null], $e=[1], b_0=[$1]}])
+      +- FlinkLogicalCalc(select=[a, b])
+         +- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_SingleDistinctAgg_WithNonDistinctAgg2">
+    <Resource name="sql">
+      <![CDATA[SELECT a, COUNT(DISTINCT b), SUM(b) FROM MyTable2 GROUP BY a, c]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], EXPR$1=[$2], EXPR$2=[$3])
++- LogicalAggregate(group=[{0, 1}], EXPR$1=[COUNT(DISTINCT $2)], EXPR$2=[SUM($2)])
+   +- LogicalProject(a=[$0], c=[$2], b=[$1])
+      +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2])
++- FlinkLogicalAggregate(group=[{0}], c=[AUXILIARY_GROUP($1)], EXPR$1=[COUNT($2) FILTER $4], EXPR$2=[MIN($3) FILTER $5])
+   +- FlinkLogicalCalc(select=[a, c, b, b_0 AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+      +- FlinkLogicalExpand(projects=[{a=[$0], c=[$1], b=[$2], $e=[0], b_0=[$2]}, {a=[$0], c=[$1], b=[null], $e=[1], b_0=[$2]}])
+         +- FlinkLogicalCalc(select=[a, c, b])
+            +- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_SingleDistinctAgg_WithNonDistinctAgg3">
+    <Resource name="sql">
+      <![CDATA[SELECT a, COUNT(DISTINCT c), SUM(b) FROM MyTable3 GROUP BY a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($2)])
++- LogicalProject(a=[$0], c=[$2], b=[$1])
+   +- LogicalTableScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $3], EXPR$2=[MIN($2) FILTER $4])
++- FlinkLogicalCalc(select=[a, c, b AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+   +- FlinkLogicalExpand(projects=[{a=[$0], c=[$1], b=[$2], $e=[0]}, {a=[$0], c=[null], b=[$2], $e=[1]}])
+      +- FlinkLogicalCalc(select=[a, c, b])
+         +- FlinkLogicalTableSourceScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_SingleDistinctAgg_WithNonDistinctAgg4">
+    <Resource name="sql">
+      <![CDATA[SELECT a, COUNT(DISTINCT c), SUM(b) FROM MyTable3 GROUP BY a, d]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], EXPR$1=[$2], EXPR$2=[$3])
++- LogicalAggregate(group=[{0, 1}], EXPR$1=[COUNT(DISTINCT $2)], EXPR$2=[SUM($3)])
+   +- LogicalProject(a=[$0], d=[$3], c=[$2], b=[$1])
+      +- LogicalTableScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2])
++- FlinkLogicalAggregate(group=[{0}], d=[AUXILIARY_GROUP($1)], EXPR$1=[COUNT($2) FILTER $4], EXPR$2=[MIN($3) FILTER $5])
+   +- FlinkLogicalCalc(select=[a, d, c, b AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
+      +- FlinkLogicalExpand(projects=[{a=[$0], d=[$1], c=[$2], b=[$3], $e=[0]}, {a=[$0], d=[$1], c=[null], b=[$3], $e=[1]}])
+         +- FlinkLogicalCalc(select=[a, d, c, b])
+            +- FlinkLogicalTableSourceScan(table=[[MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_WithFilter">
+    <Resource name="sql">
+      <![CDATA[SELECT a, MIN(c) FILTER (WHERE b > 0), MAX(b) FROM MyTable2 GROUP BY a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[MIN($1) FILTER $2], EXPR$2=[MAX($3)])
++- LogicalProject(a=[$0], c=[$2], $f2=[IS TRUE(>($1, 0))], b=[$1])
+   +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{0}], EXPR$1=[MIN($1) FILTER $2], EXPR$2=[MAX($3)])
++- FlinkLogicalCalc(select=[a, c, IS TRUE(>(b, 0)) AS $f2, b])
+   +- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_WithoutAggCall">
+    <Resource name="sql">
+      <![CDATA[SELECT a, b FROM MyTable2 GROUP BY a, b]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, b])
++- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_WithoutFilter1">
+    <Resource name="sql">
+      <![CDATA[SELECT a, b + 1, c, s FROM (SELECT a, MIN(b) AS b, SUM(b) AS s, MAX(c) AS c FROM MyTable2 GROUP BY a)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], EXPR$1=[+($1, 1)], c=[$3], s=[$2])
++- LogicalAggregate(group=[{0}], b=[MIN($1)], s=[SUM($1)], c=[MAX($2)])
+   +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, +(b, 1) AS EXPR$1, c, b AS s])
++- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_WithoutFilter2">
+    <Resource name="sql">
+      <![CDATA[SELECT a, SUM(b) AS s FROM MyTable2 GROUP BY a]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{0}], s=[SUM($1)])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, b AS s])
++- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_WithoutGroupBy1">
+    <Resource name="sql">
+      <![CDATA[SELECT MAX(a), SUM(b), MIN(c) FROM MyTable2]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[MAX($0)], EXPR$1=[SUM($1)], EXPR$2=[MIN($2)])
++- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{}], EXPR$0=[MAX($0)], EXPR$1=[SUM($1)], EXPR$2=[MIN($2)])
++- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_WithoutGroupBy2">
+    <Resource name="sql">
+      <![CDATA[SELECT MAX(a), SUM(b), MIN(c) FROM (VALUES (1, 2, 3)) T(a, b, c)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[MAX($0)], EXPR$1=[SUM($1)], EXPR$2=[MIN($2)])
++- LogicalValues(tuples=[[{ 1, 2, 3 }]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{}], EXPR$0=[MAX($0)], EXPR$1=[SUM($1)], EXPR$2=[MIN($2)])
++- FlinkLogicalValues(tuples=[[{ 1, 2, 3 }]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_WithoutGroupBy3">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable2 WHERE EXISTS (SELECT SUM(a) FROM MyTable1 WHERE 1=2)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], c=[$2])
++- LogicalFilter(condition=[EXISTS({
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+  LogicalProject(a=[$0])
+    LogicalFilter(condition=[=(1, 2)])
+      LogicalTableScan(table=[[MyTable1, source: [TestTableSource(a, b, c)]]])
+})])
+   +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalCalc(select=[a, b, c])
++- FlinkLogicalJoin(condition=[$3], joinType=[semi])
+   :- FlinkLogicalTableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- FlinkLogicalCalc(select=[IS NOT NULL(m) AS $f0])
+      +- FlinkLogicalAggregate(group=[{}], m=[MIN($0)])
+         +- FlinkLogicalCalc(select=[true AS i])
+            +- FlinkLogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+               +- FlinkLogicalCalc(select=[a])
+                  +- FlinkLogicalValues(tuples=[[]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAggRemove_WithoutGroupBy4">
+    <Resource name="sql">
+      <![CDATA[SELECT SUM(a) FROM (SELECT a FROM MyTable2 WHERE 1=2)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
++- LogicalProject(a=[$0])
+   +- LogicalFilter(condition=[=(1, 2)])
+      +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+FlinkLogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
++- FlinkLogicalCalc(select=[a])
+   +- FlinkLogicalValues(tuples=[[]])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml
new file mode 100644
index 0000000..c77788a
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml
@@ -0,0 +1,379 @@
+<?xml version="1.0" ?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<Root>
+  <TestCase name="testEmptyGroupKey_WithMoreThanOneAggCalls1">
+    <Resource name="sql">
+      <![CDATA[SELECT 1 FROM (SELECT SUM(a1), COUNT(*) FROM T1) t]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(EXPR$0=[1])
++- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT()])
+   +- LogicalProject(a1=[$0])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(EXPR$0=[1])
++- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+   +- LogicalProject(a1=[$0])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testEmptyGroupKey_WithMoreThanOneAggCalls2">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM T2 WHERE EXISTS (SELECT SUM(a1), COUNT(*) FROM T1)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3])
++- LogicalFilter(condition=[EXISTS({
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT()])
+  LogicalProject(a1=[$0])
+    LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+})])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalJoin(condition=[$4], joinType=[semi])
+:- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
++- LogicalProject($f0=[IS NOT NULL($0)])
+   +- LogicalAggregate(group=[{}], m=[MIN($0)])
+      +- LogicalProject(i=[true])
+         +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+            +- LogicalProject(a1=[$0])
+               +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testEmptyGroupKey_WithMoreThanOneAggCalls3">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM T2 WHERE EXISTS (SELECT SUM(a1), COUNT(*) FROM T1 WHERE 1=2)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3])
++- LogicalFilter(condition=[EXISTS({
+LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[COUNT()])
+  LogicalProject(a1=[$0])
+    LogicalFilter(condition=[=(1, 2)])
+      LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+})])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalJoin(condition=[$4], joinType=[semi])
+:- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
++- LogicalProject($f0=[IS NOT NULL($0)])
+   +- LogicalAggregate(group=[{}], m=[MIN($0)])
+      +- LogicalProject(i=[true])
+         +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+            +- LogicalValues(tuples=[[]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testEmptyGroupKey_WithOneAggCall1">
+    <Resource name="sql">
+      <![CDATA[SELECT 1 FROM (SELECT SUM(a1) FROM T1) t]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(EXPR$0=[1])
++- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+   +- LogicalProject(a1=[$0])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(EXPR$0=[1])
++- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
+   +- LogicalProject(a1=[$0])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testEmptyGroupKey_WithOneAggCall2">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM T2 WHERE EXISTS (SELECT COUNT(*) FROM T1)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3])
++- LogicalFilter(condition=[EXISTS({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+})])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalJoin(condition=[$4], joinType=[semi])
+:- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
++- LogicalProject($f0=[IS NOT NULL($0)])
+   +- LogicalAggregate(group=[{}], m=[MIN($0)])
+      +- LogicalProject(i=[true])
+         +- LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+            +- LogicalProject($f0=[0])
+               +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testEmptyGroupKey_WithOneAggCall3">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM T2 WHERE EXISTS (SELECT COUNT(*) FROM T1 WHERE 1=2)]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3])
++- LogicalFilter(condition=[EXISTS({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalFilter(condition=[=(1, 2)])
+      LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+})])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalJoin(condition=[$4], joinType=[semi])
+:- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
++- LogicalProject($f0=[IS NOT NULL($0)])
+   +- LogicalAggregate(group=[{}], m=[MIN($0)])
+      +- LogicalProject(i=[true])
+         +- LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+            +- LogicalValues(tuples=[[]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneAuxGroupAggCall_WithFilter1">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a1, c1 FROM
+ (SELECT a1, b1, COUNT(c1) as c1, SUM(d1) as d1 FROM T1 GROUP BY a1, b1) t
+WHERE c1 > 10
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a1=[$0], c1=[$2])
++- LogicalFilter(condition=[>($2, 10)])
+   +- LogicalAggregate(group=[{0, 1}], c1=[COUNT($2)], d1=[SUM($3)])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalFilter(condition=[>($1, 10)])
++- LogicalAggregate(group=[{0}], c1=[COUNT($2)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneAuxGroupAggCall_WithFilter2">
+    <Resource name="sql">
+      <![CDATA[
+SELECT c1, a1 FROM
+ (SELECT a1, b1, COUNT(c1) as c1, SUM(d1) as d1 FROM T1 GROUP BY a1, b1) t
+WHERE c1 > 10
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(c1=[$2], a1=[$0])
++- LogicalFilter(condition=[>($2, 10)])
+   +- LogicalAggregate(group=[{0, 1}], c1=[COUNT($2)], d1=[SUM($3)])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(c1=[$1], a1=[$0])
++- LogicalFilter(condition=[>($1, 10)])
+   +- LogicalAggregate(group=[{0}], c1=[COUNT($2)])
+      +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneAuxGroupAggCall_WithoutFilter1">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a1, c1 FROM
+ (SELECT a1, b1, COUNT(c1) as c1, SUM(d1) as d1 FROM T1 GROUP BY a1, b1) t
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a1=[$0], c1=[$2])
++- LogicalAggregate(group=[{0, 1}], c1=[COUNT($2)], d1=[SUM($3)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalAggregate(group=[{0}], c1=[COUNT($2)])
++- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneAuxGroupAggCall_WithoutFilter2">
+    <Resource name="sql">
+      <![CDATA[
+SELECT c1, a1 FROM
+ (SELECT a1, b1, COUNT(c1) as c1, SUM(d1) as d1 FROM T1 GROUP BY a1, b1) t
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(c1=[$2], a1=[$0])
++- LogicalAggregate(group=[{0, 1}], c1=[COUNT($2)], d1=[SUM($3)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(c1=[$1], a1=[$0])
++- LogicalAggregate(group=[{0}], c1=[COUNT($2)])
+   +- LogicalTableScan(table=[[T1, source: [TestTableSource(a1, b1, c1, d1)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneRegularAggCall_WithFilter1">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a2, b2, d2 FROM
+ (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+WHERE d2 > 0
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a2=[$0], b2=[$1], d2=[$3])
++- LogicalFilter(condition=[>($3, 0)])
+   +- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)])
+      +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalFilter(condition=[>($2, 0)])
++- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneRegularAggCall_WithFilter2">
+    <Resource name="sql">
+      <![CDATA[
+SELECT b2, a2, d2 FROM
+ (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+WHERE d2 > 0
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(b2=[$1], a2=[$0], d2=[$3])
++- LogicalFilter(condition=[>($3, 0)])
+   +- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)])
+      +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(b2=[$1], a2=[$0], d2=[$2])
++- LogicalFilter(condition=[>($2, 0)])
+   +- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)])
+      +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneRegularAggCall_WithoutFilter1">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a2, b2, d2 FROM
+ (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a2=[$0], b2=[$1], d2=[$3])
++- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalAggregate(group=[{0, 1}], d2=[SUM($3)])
++- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneRegularAggCall_WithoutFilter2">
+    <Resource name="sql">
+      <![CDATA[
+SELECT b2, a2, d2 FROM
+ (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(b2=[$1], a2=[$0], d2=[$3])
++- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(b2=[$1], a2=[$0], d2=[$2])
++- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testPruneRegularAggCall_WithoutFilter3">
+    <Resource name="sql">
+      <![CDATA[
+SELECT a2 as a, b2, d2 FROM
+ (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+      ]]>
+    </Resource>
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalProject(a=[$0], b2=[$1], d2=[$3])
++- LogicalAggregate(group=[{0, 1}], c2=[COUNT($2)], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+LogicalProject(a=[$0], b2=[$1], d2=[$2])
++- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)])
+   +- LogicalTableScan(table=[[T2, source: [TestTableSource(a2, b2, c2, d2)]]])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/join/JoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/join/JoinTest.xml
index 807d557..9c41ba5 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/join/JoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/join/JoinTest.xml
@@ -87,17 +87,15 @@ LogicalProject(a1=[$1], b1=[$3])
       <![CDATA[
 Join(joinType=[FullOuterJoin], where=[=(a1, b1)], select=[a1, b1], updateAsRetraction=[false], accMode=[AccRetract])
 :- Exchange(distribution=[hash[a1]], updateAsRetraction=[false], accMode=[Acc])
-:  +- Calc(select=[a1], updateAsRetraction=[false], accMode=[Acc])
-:     +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], updateAsRetraction=[false], accMode=[Acc])
-:        +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
-:           +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
-:              +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
+:  +- GroupAggregate(groupBy=[a1], select=[a1], updateAsRetraction=[false], accMode=[Acc])
+:     +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+:        +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
+:           +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
 +- Exchange(distribution=[hash[b1]], updateAsRetraction=[false], accMode=[Acc])
-   +- Calc(select=[b1], updateAsRetraction=[false], accMode=[Acc])
-      +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], updateAsRetraction=[false], accMode=[Acc])
-         +- Exchange(distribution=[hash[b1]], updateAsRetraction=[true], accMode=[Acc])
-            +- Calc(select=[b1, b2], updateAsRetraction=[true], accMode=[Acc])
-               +- TableSourceScan(table=[[B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], updateAsRetraction=[true], accMode=[Acc])
+   +- GroupAggregate(groupBy=[b1], select=[b1], updateAsRetraction=[false], accMode=[Acc])
+      +- Exchange(distribution=[hash[b1]], updateAsRetraction=[true], accMode=[Acc])
+         +- Calc(select=[b1, b2], updateAsRetraction=[true], accMode=[Acc])
+            +- TableSourceScan(table=[[B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], updateAsRetraction=[true], accMode=[Acc])
 ]]>
     </Resource>
   </TestCase>
@@ -157,11 +155,10 @@ LogicalProject(a1=[$1], b1=[$2])
       <![CDATA[
 Join(joinType=[FullOuterJoin], where=[=(a1, b1)], select=[a1, b1], updateAsRetraction=[false], accMode=[AccRetract])
 :- Exchange(distribution=[hash[a1]], updateAsRetraction=[false], accMode=[Acc])
-:  +- Calc(select=[a1], updateAsRetraction=[false], accMode=[Acc])
-:     +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], updateAsRetraction=[false], accMode=[Acc])
-:        +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
-:           +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
-:              +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
+:  +- GroupAggregate(groupBy=[a1], select=[a1], updateAsRetraction=[false], accMode=[Acc])
+:     +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+:        +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
+:           +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
 +- Exchange(distribution=[hash[b1]], updateAsRetraction=[true], accMode=[Acc])
    +- Calc(select=[b1], updateAsRetraction=[true], accMode=[Acc])
       +- TableSourceScan(table=[[B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], updateAsRetraction=[true], accMode=[Acc])
@@ -318,17 +315,15 @@ LogicalProject(a1=[$1], b1=[$3])
       <![CDATA[
 Join(joinType=[InnerJoin], where=[=(a1, b1)], select=[a1, b1], updateAsRetraction=[false], accMode=[Acc])
 :- Exchange(distribution=[hash[a1]], updateAsRetraction=[false], accMode=[Acc])
-:  +- Calc(select=[a1], updateAsRetraction=[false], accMode=[Acc])
-:     +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], updateAsRetraction=[false], accMode=[Acc])
-:        +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
-:           +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
-:              +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
+:  +- GroupAggregate(groupBy=[a1], select=[a1], updateAsRetraction=[false], accMode=[Acc])
+:     +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+:        +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
+:           +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
 +- Exchange(distribution=[hash[b1]], updateAsRetraction=[false], accMode=[Acc])
-   +- Calc(select=[b1], updateAsRetraction=[false], accMode=[Acc])
-      +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], updateAsRetraction=[false], accMode=[Acc])
-         +- Exchange(distribution=[hash[b1]], updateAsRetraction=[true], accMode=[Acc])
-            +- Calc(select=[b1, b2], updateAsRetraction=[true], accMode=[Acc])
-               +- TableSourceScan(table=[[B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], updateAsRetraction=[true], accMode=[Acc])
+   +- GroupAggregate(groupBy=[b1], select=[b1], updateAsRetraction=[false], accMode=[Acc])
+      +- Exchange(distribution=[hash[b1]], updateAsRetraction=[true], accMode=[Acc])
+         +- Calc(select=[b1, b2], updateAsRetraction=[true], accMode=[Acc])
+            +- TableSourceScan(table=[[B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], updateAsRetraction=[true], accMode=[Acc])
 ]]>
     </Resource>
   </TestCase>
@@ -378,17 +373,15 @@ LogicalProject(a1=[$1], b1=[$3])
       <![CDATA[
 Join(joinType=[LeftOuterJoin], where=[=(a1, b1)], select=[a1, b1], updateAsRetraction=[false], accMode=[AccRetract])
 :- Exchange(distribution=[hash[a1]], updateAsRetraction=[false], accMode=[Acc])
-:  +- Calc(select=[a1], updateAsRetraction=[false], accMode=[Acc])
-:     +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], updateAsRetraction=[false], accMode=[Acc])
-:        +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
-:           +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
-:              +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
+:  +- GroupAggregate(groupBy=[a1], select=[a1], updateAsRetraction=[false], accMode=[Acc])
+:     +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+:        +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
+:           +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
 +- Exchange(distribution=[hash[b1]], updateAsRetraction=[false], accMode=[Acc])
-   +- Calc(select=[b1], updateAsRetraction=[false], accMode=[Acc])
-      +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], updateAsRetraction=[false], accMode=[Acc])
-         +- Exchange(distribution=[hash[b1]], updateAsRetraction=[true], accMode=[Acc])
-            +- Calc(select=[b1, b2], updateAsRetraction=[true], accMode=[Acc])
-               +- TableSourceScan(table=[[B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], updateAsRetraction=[true], accMode=[Acc])
+   +- GroupAggregate(groupBy=[b1], select=[b1], updateAsRetraction=[false], accMode=[Acc])
+      +- Exchange(distribution=[hash[b1]], updateAsRetraction=[true], accMode=[Acc])
+         +- Calc(select=[b1, b2], updateAsRetraction=[true], accMode=[Acc])
+            +- TableSourceScan(table=[[B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], updateAsRetraction=[true], accMode=[Acc])
 ]]>
     </Resource>
   </TestCase>
@@ -636,11 +629,10 @@ LogicalProject(a1=[$1], b1=[$2])
       <![CDATA[
 Join(joinType=[LeftOuterJoin], where=[=(a1, b1)], select=[a1, b1], updateAsRetraction=[false], accMode=[AccRetract])
 :- Exchange(distribution=[hash[a1]], updateAsRetraction=[false], accMode=[Acc])
-:  +- Calc(select=[a1], updateAsRetraction=[false], accMode=[Acc])
-:     +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], updateAsRetraction=[false], accMode=[Acc])
-:        +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
-:           +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
-:              +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
+:  +- GroupAggregate(groupBy=[a1], select=[a1], updateAsRetraction=[false], accMode=[Acc])
+:     +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+:        +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
+:           +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
 +- Exchange(distribution=[hash[b1]], updateAsRetraction=[true], accMode=[Acc])
    +- Calc(select=[b1], updateAsRetraction=[true], accMode=[Acc])
       +- TableSourceScan(table=[[B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], updateAsRetraction=[true], accMode=[Acc])
@@ -848,17 +840,15 @@ LogicalProject(a1=[$1], b1=[$3])
       <![CDATA[
 Join(joinType=[RightOuterJoin], where=[=(a1, b1)], select=[a1, b1], updateAsRetraction=[false], accMode=[AccRetract])
 :- Exchange(distribution=[hash[a1]], updateAsRetraction=[false], accMode=[Acc])
-:  +- Calc(select=[a1], updateAsRetraction=[false], accMode=[Acc])
-:     +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], updateAsRetraction=[false], accMode=[Acc])
-:        +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
-:           +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
-:              +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
+:  +- GroupAggregate(groupBy=[a1], select=[a1], updateAsRetraction=[false], accMode=[Acc])
+:     +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+:        +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
+:           +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
 +- Exchange(distribution=[hash[b1]], updateAsRetraction=[false], accMode=[Acc])
-   +- Calc(select=[b1], updateAsRetraction=[false], accMode=[Acc])
-      +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], updateAsRetraction=[false], accMode=[Acc])
-         +- Exchange(distribution=[hash[b1]], updateAsRetraction=[true], accMode=[Acc])
-            +- Calc(select=[b1, b2], updateAsRetraction=[true], accMode=[Acc])
-               +- TableSourceScan(table=[[B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], updateAsRetraction=[true], accMode=[Acc])
+   +- GroupAggregate(groupBy=[b1], select=[b1], updateAsRetraction=[false], accMode=[Acc])
+      +- Exchange(distribution=[hash[b1]], updateAsRetraction=[true], accMode=[Acc])
+         +- Calc(select=[b1, b2], updateAsRetraction=[true], accMode=[Acc])
+            +- TableSourceScan(table=[[B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], updateAsRetraction=[true], accMode=[Acc])
 ]]>
     </Resource>
   </TestCase>
@@ -955,11 +945,10 @@ LogicalProject(a1=[$1], b1=[$2])
       <![CDATA[
 Join(joinType=[RightOuterJoin], where=[=(a1, b1)], select=[a1, b1], updateAsRetraction=[false], accMode=[AccRetract])
 :- Exchange(distribution=[hash[a1]], updateAsRetraction=[false], accMode=[Acc])
-:  +- Calc(select=[a1], updateAsRetraction=[false], accMode=[Acc])
-:     +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], updateAsRetraction=[false], accMode=[Acc])
-:        +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
-:           +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
-:              +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
+:  +- GroupAggregate(groupBy=[a1], select=[a1], updateAsRetraction=[false], accMode=[Acc])
+:     +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+:        +- Calc(select=[a1, a2], updateAsRetraction=[true], accMode=[Acc])
+:           +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], updateAsRetraction=[true], accMode=[Acc])
 +- Exchange(distribution=[hash[b1]], updateAsRetraction=[true], accMode=[Acc])
    +- Calc(select=[b1], updateAsRetraction=[true], accMode=[Acc])
       +- TableSourceScan(table=[[B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], updateAsRetraction=[true], accMode=[Acc])
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/join/SemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/join/SemiAntiJoinTest.xml
index bbc870f..eb04022 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/join/SemiAntiJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/join/SemiAntiJoinTest.xml
@@ -74,7 +74,7 @@ Join(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(c, f))], select=[a, b, c])
 :  +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[e, f]])
    +- Calc(select=[f, e])
-      +- GroupAggregate(groupBy=[d, e, f], select=[d, e, f, MAX(e) AS EXPR$0])
+      +- GroupAggregate(groupBy=[d, e, f], select=[d, e, f])
          +- Exchange(distribution=[hash[d, e, f]])
             +- Calc(select=[d, e, f], where=[<(d, 100)])
                +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
@@ -528,10 +528,10 @@ Join(joinType=[LeftAntiJoin], where=[<>(b, e)], select=[a, b, c])
 :     :     :           +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], reuse_id=[1])
 :     :     +- Exchange(distribution=[single])
 :     :        +- Calc(select=[IS NOT NULL(m) AS $f0])
-:     :           +- GroupAggregate(select=[MIN_RETRACT(i) AS m])
+:     :           +- GroupAggregate(select=[MIN(i) AS m])
 :     :              +- Exchange(distribution=[single])
 :     :                 +- Calc(select=[true AS i])
-:     :                    +- GroupAggregate(groupBy=[l], select=[l, COUNT(l) AS EXPR$0])
+:     :                    +- GroupAggregate(groupBy=[l], select=[l])
 :     :                       +- Exchange(distribution=[hash[l]])
 :     :                          +- Calc(select=[l], where=[LIKE(n, _UTF-16LE'Test')])
 :     :                             +- TableSourceScan(table=[[t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n])
@@ -690,11 +690,10 @@ Calc(select=[b])
    :        :     :- Exchange(distribution=[single])
    :        :     :  +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :        :     +- Exchange(distribution=[single])
-   :        :        +- Calc(select=[c])
-   :        :           +- GroupAggregate(select=[COUNT(*) AS c, COUNT(EXPR$0) AS ck])
-   :        :              +- Exchange(distribution=[single])
-   :        :                 +- Calc(select=[1 AS EXPR$0])
-   :        :                    +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+   :        :        +- GroupAggregate(select=[COUNT(*) AS c])
+   :        :           +- Exchange(distribution=[single])
+   :        :              +- Calc(select=[1 AS EXPR$0])
+   :        :                 +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
    :        +- Exchange(distribution=[single])
    :           +- Calc(select=[i])
    :              +- GroupAggregate(groupBy=[EXPR$0, i], select=[EXPR$0, i])
@@ -1234,11 +1233,10 @@ Calc(select=[b])
    :        :     :  +- Calc(select=[a, b])
    :        :     :     +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :        :     +- Exchange(distribution=[single])
-   :        :        +- Calc(select=[c])
-   :        :           +- GroupAggregate(select=[COUNT(*) AS c, COUNT(EXPR$0) AS ck])
-   :        :              +- Exchange(distribution=[single])
-   :        :                 +- Calc(select=[1 AS EXPR$0])
-   :        :                    +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+   :        :        +- GroupAggregate(select=[COUNT(*) AS c])
+   :        :           +- Exchange(distribution=[single])
+   :        :              +- Calc(select=[1 AS EXPR$0])
+   :        :                 +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
    :        +- Exchange(distribution=[single])
    :           +- Calc(select=[i])
    :              +- GroupAggregate(groupBy=[EXPR$0, i], select=[EXPR$0, i])
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala
index 6090227..fd9237a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.batch.sql
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.{PlannerConfigOptions, TableConfigOptions, Types}
-import org.apache.flink.table.plan.stats.TableStats
+import org.apache.flink.table.plan.stats.{FlinkStatistic, TableStats}
 import org.apache.flink.table.runtime.utils.JavaUserDefinedTableFunctions.StringSplit
 import org.apache.flink.table.util.{TableFunc1, TableTestBase}
 
@@ -37,22 +37,22 @@ class RemoveCollationTest extends TableTestBase {
     util.addTableSource("x",
       Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING),
       Array("a", "b", "c"),
-      tableStats = Some(new TableStats(100L))
+      FlinkStatistic.builder().tableStats(new TableStats(100L)).build()
     )
     util.addTableSource("y",
       Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING),
       Array("d", "e", "f"),
-      tableStats = Some(new TableStats(100L))
+      FlinkStatistic.builder().tableStats(new TableStats(100L)).build()
     )
     util.addTableSource("t1",
       Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING),
       Array("a1", "b1", "c1"),
-      tableStats = Some(new TableStats(100L))
+      FlinkStatistic.builder().tableStats(new TableStats(100L)).build()
     )
     util.addTableSource("t2",
       Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING),
       Array("d1", "e1", "f1"),
-      tableStats = Some(new TableStats(100L))
+      FlinkStatistic.builder().tableStats(new TableStats(100L)).build()
     )
 
     util.tableEnv.getConfig.getConf.setBoolean(
@@ -269,27 +269,27 @@ class RemoveCollationTest extends TableTestBase {
       Array[TypeInformation[_]](
         Types.STRING, Types.STRING, Types.STRING, Types.STRING, Types.STRING),
       Array("id", "key", "tb2_ids", "tb3_ids", "name"),
-      uniqueKeys = Some(ImmutableSet.of(ImmutableSet.of("id")))
+      FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("id"))).build()
     )
     util.addTableSource("tb2",
       Array[TypeInformation[_]](Types.STRING, Types.STRING),
       Array("id",  "name"),
-      uniqueKeys = Some(ImmutableSet.of(ImmutableSet.of("id")))
+      FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("id"))).build()
     )
     util.addTableSource("tb3",
       Array[TypeInformation[_]](Types.STRING, Types.STRING),
       Array("id",  "name"),
-      uniqueKeys = Some(ImmutableSet.of(ImmutableSet.of("id")))
+      FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("id"))).build()
     )
     util.addTableSource("tb4",
       Array[TypeInformation[_]](Types.STRING, Types.STRING),
       Array("id",  "name"),
-      uniqueKeys = Some(ImmutableSet.of(ImmutableSet.of("id")))
+      FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("id"))).build()
     )
     util.addTableSource("tb5",
       Array[TypeInformation[_]](Types.STRING, Types.STRING),
       Array("id",  "name"),
-      uniqueKeys = Some(ImmutableSet.of(ImmutableSet.of("id")))
+      FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("id"))).build()
     )
     util.tableEnv.registerFunction("split", new StringSplit())
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala
index 29b14b9..e81190b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.batch.sql
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.{PlannerConfigOptions, TableConfigOptions, Types}
-import org.apache.flink.table.plan.stats.TableStats
+import org.apache.flink.table.plan.stats.{FlinkStatistic, TableStats}
 import org.apache.flink.table.util.{TableFunc1, TableTestBase}
 
 import org.junit.{Before, Test}
@@ -35,12 +35,12 @@ class RemoveShuffleTest extends TableTestBase {
     util.addTableSource("x",
       Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING),
       Array("a", "b", "c"),
-      tableStats = Some(new TableStats(100L))
+      FlinkStatistic.builder().tableStats(new TableStats(100L)).build()
     )
     util.addTableSource("y",
       Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING),
       Array("d", "e", "f"),
-      tableStats = Some(new TableStats(100L))
+      FlinkStatistic.builder().tableStats(new TableStats(100L)).build()
     )
     util.tableEnv.getConfig.getConf.setBoolean(
       PlannerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/AggregateReduceGroupingTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/AggregateReduceGroupingTest.scala
new file mode 100644
index 0000000..e55b8b4
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/AggregateReduceGroupingTest.scala
@@ -0,0 +1,24 @@
+/*
+ * 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.flink.table.plan.batch.sql.agg
+
+import org.apache.flink.table.plan.common.AggregateReduceGroupingTestBase
+
+class AggregateReduceGroupingTest extends AggregateReduceGroupingTestBase {
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/AggregateReduceGroupingTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/AggregateReduceGroupingTestBase.scala
new file mode 100644
index 0000000..687f730
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/AggregateReduceGroupingTestBase.scala
@@ -0,0 +1,318 @@
+/*
+ * 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.flink.table.plan.common
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.calcite.CalciteConfig
+import org.apache.flink.table.plan.optimize.program.FlinkBatchProgram
+import org.apache.flink.table.plan.rules.logical.FlinkAggregateRemoveRule
+import org.apache.flink.table.plan.stats.{FlinkStatistic, TableStats}
+import org.apache.flink.table.util.{BatchTableTestUtil, TableTestBase}
+
+import com.google.common.collect.ImmutableSet
+import org.apache.calcite.tools.RuleSets
+import org.junit.{Before, Test}
+
+abstract class AggregateReduceGroupingTestBase extends TableTestBase {
+  protected val util: BatchTableTestUtil = batchTestUtil()
+
+  @Before
+  def setup(): Unit = {
+    util.addTableSource("T1",
+      Array[TypeInformation[_]](Types.INT, Types.INT, Types.STRING, Types.STRING),
+      Array("a1", "b1", "c1", "d1"),
+      FlinkStatistic.builder()
+        .tableStats(new TableStats(100000000))
+        .uniqueKeys(ImmutableSet.of(ImmutableSet.of("a1")))
+        .build()
+    )
+    util.addTableSource("T2",
+      Array[TypeInformation[_]](Types.INT, Types.INT, Types.STRING),
+      Array("a2", "b2", "c2"),
+      FlinkStatistic.builder()
+        .tableStats(new TableStats(100000000))
+        .uniqueKeys(ImmutableSet.of(ImmutableSet.of("b2"), ImmutableSet.of("a2", "b2")))
+        .build()
+    )
+    util.addTableSource("T3",
+      Array[TypeInformation[_]](Types.INT, Types.INT, Types.STRING, Types.LONG),
+      Array("a3", "b3", "c3", "d3"),
+      FlinkStatistic.builder()
+        .tableStats(new TableStats(1000))
+        .build()
+    )
+    util.addTableSource("T4",
+      Array[TypeInformation[_]](Types.INT, Types.INT, Types.STRING, Types.SQL_TIMESTAMP),
+      Array("a4", "b4", "c4", "d4"),
+      FlinkStatistic.builder()
+        .tableStats(new TableStats(100000000))
+        .uniqueKeys(ImmutableSet.of(ImmutableSet.of("a4")))
+        .build()
+    )
+  }
+
+  @Test
+  def testAggWithoutAggCall(): Unit = {
+    val programs = util.tableEnv.getConfig.getCalciteConfig.getBatchProgram
+      .getOrElse(FlinkBatchProgram.buildProgram(util.getTableEnv.getConfig.getConf))
+    programs.getFlinkRuleSetProgram(FlinkBatchProgram.LOGICAL)
+      .get.remove(RuleSets.ofList(FlinkAggregateRemoveRule.INSTANCE)) // to prevent the agg from
+    // removing
+    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
+      .replaceBatchProgram(programs).build()
+    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.verifyPlan("SELECT a1, b1, c1 FROM T1 GROUP BY a1, b1, c1")
+  }
+
+  @Test
+  def testAggWithoutReduceGrouping(): Unit = {
+    util.verifyPlan("SELECT a3, b3, count(c3) FROM T3 GROUP BY a3, b3")
+  }
+
+  @Test
+  def testSingleAggOnTableWithUniqueKey(): Unit = {
+    util.verifyPlan("SELECT a1, b1, count(c1) FROM T1 GROUP BY a1, b1")
+  }
+
+  @Test
+  def testSingleAggOnTableWithoutUniqueKey(): Unit = {
+    util.verifyPlan("SELECT a3, b3, count(c3) FROM T3 GROUP BY a3, b3")
+  }
+
+  @Test
+  def testSingleAggOnTableWithUniqueKeys(): Unit = {
+    util.verifyPlan("SELECT  b2, c2, avg(a2) FROM T2 GROUP BY b2, c2")
+  }
+
+  @Test
+  def testSingleAggWithConstantGroupKey(): Unit = {
+    util.verifyPlan("SELECT a1, b1, count(c1) FROM T1 GROUP BY a1, b1, 1, true")
+  }
+
+  @Test
+  def testSingleAggOnlyConstantGroupKey(): Unit = {
+    util.verifyPlan("SELECT count(c1) FROM T1 GROUP BY 1, true")
+  }
+
+  @Test
+  def testMultiAggs1(): Unit = {
+    util.verifyPlan("SELECT a1, b1, c1, d1, m, COUNT(*) FROM " +
+      "(SELECT a1, b1, c1, COUNT(d1) AS d1, MAX(d1) AS m FROM T1 GROUP BY a1, b1, c1) t " +
+      "GROUP BY a1, b1, c1, d1, m")
+  }
+
+  @Test
+  def testMultiAggs2(): Unit = {
+    util.verifyPlan("SELECT a3, b3, c, s, a, COUNT(*) FROM " +
+      "(SELECT a3, b3, COUNT(c3) AS c, SUM(d3) AS s, AVG(d3) AS a FROM T3 GROUP BY a3, b3) t " +
+      "GROUP BY a3, b3, c, s, a")
+  }
+
+  @Test
+  def testAggOnInnerJoin1(): Unit = {
+    util.verifyPlan("SELECT a1, b1, a2, b2, COUNT(c1) FROM " +
+      "(SELECT * FROM T1, T2 WHERE a1 = b2) t GROUP BY a1, b1, a2, b2")
+  }
+
+  @Test
+  def testAggOnInnerJoin2(): Unit = {
+    util.verifyPlan("SELECT a2, b2, a3, b3, COUNT(c2), AVG(d3) FROM " +
+      "(SELECT * FROM T2, T3 WHERE b2 = a3) t GROUP BY a2, b2, a3, b3")
+  }
+
+  @Test
+  def testAggOnInnerJoin3(): Unit = {
+    util.verifyPlan("SELECT a1, b1, a2, b2, a3, b3, COUNT(c1) FROM " +
+      "(SELECT * FROM T1, T2, T3 WHERE a1 = b2 AND a1 = a3) t GROUP BY a1, b1, a2, b2, a3, b3")
+  }
+
+  @Test
+  def testAggOnLeftJoin1(): Unit = {
+    util.verifyPlan("SELECT a1, b1, a2, b2, COUNT(c1) FROM " +
+      "(SELECT * FROM T1 LEFT JOIN T2 ON a1 = b2) t GROUP BY a1, b1, a2, b2")
+  }
+
+  @Test
+  def testAggOnLeftJoin2(): Unit = {
+    util.verifyPlan("SELECT a1, b1, a3, b3, COUNT(c1) FROM " +
+      "(SELECT * FROM T1 LEFT JOIN T3 ON a1 = a3) t GROUP BY a1, b1, a3, b3")
+  }
+
+  @Test
+  def testAggOnLeftJoin3(): Unit = {
+    util.verifyPlan("SELECT a3, b3, a1, b1, COUNT(c1) FROM " +
+      "(SELECT * FROM T3 LEFT JOIN T1 ON a1 = a3) t GROUP BY a3, b3, a1, b1")
+  }
+
+  @Test
+  def testAggOnRightJoin1(): Unit = {
+    util.verifyPlan("SELECT a1, b1, a2, b2, COUNT(c1) FROM " +
+      "(SELECT * FROM T1 RIGHT JOIN T2 ON a1 = b2) t GROUP BY a1, b1, a2, b2")
+  }
+
+  @Test
+  def testAggOnRightJoin2(): Unit = {
+    util.verifyPlan("SELECT a1, b1, a3, b3, COUNT(c1) FROM " +
+      "(SELECT * FROM T1 RIGHT JOIN T3 ON a1 = a3) t GROUP BY a1, b1, a3, b3")
+  }
+
+  @Test
+  def testAggOnRightJoin3(): Unit = {
+    util.verifyPlan("SELECT a3, b3, a1, b1, COUNT(c1) FROM " +
+      "(SELECT * FROM T3 RIGHT JOIN T1 ON a1 = a3) t GROUP BY a3, b3, a1, b1")
+  }
+
+  @Test
+  def testAggOnFullJoin1(): Unit = {
+    util.verifyPlan("SELECT a1, b1, a2, b2, COUNT(c1) FROM " +
+      "(SELECT * FROM T1 FULL OUTER JOIN T2 ON a1 = b2) t GROUP BY a1, b1, a2, b2")
+  }
+
+  @Test
+  def testAggOnFullJoin2(): Unit = {
+    util.verifyPlan("SELECT a1, b1, a3, b3, COUNT(c1) FROM " +
+      "(SELECT * FROM T1 FULL OUTER JOIN T3 ON a1 = a3) t GROUP BY a1, b1, a3, b3")
+  }
+
+  @Test
+  def testAggOnOver(): Unit = {
+    util.verifyPlan("SELECT a1, b1, c, COUNT(d1) FROM " +
+      "(SELECT a1, b1, d1, COUNT(*) OVER (PARTITION BY c1) AS c FROM T1) t GROUP BY a1, b1, c")
+  }
+
+  @Test
+  def testAggOnWindow1(): Unit = {
+    util.verifyPlan("SELECT a4, b4, COUNT(c4) FROM T4 " +
+      "GROUP BY a4, b4, TUMBLE(d4, INTERVAL '15' MINUTE)")
+  }
+
+  @Test
+  def testAggOnWindow2(): Unit = {
+    util.verifyPlan("SELECT a4, c4, COUNT(b4), AVG(b4) FROM T4 " +
+      "GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)")
+  }
+
+  @Test
+  def testAggOnWindow3(): Unit = {
+    util.verifyPlan("SELECT a4, c4, s, COUNT(b4) FROM " +
+      "(SELECT a4, c4, VAR_POP(b4) AS b4, " +
+      "TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, " +
+      "TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 " +
+      "GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, c4, s")
+  }
+
+  @Test
+  def testAggOnWindow4(): Unit = {
+    util.verifyPlan("SELECT a4, c4, e, COUNT(b4) FROM " +
+      "(SELECT a4, c4, VAR_POP(b4) AS b4, " +
+      "TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, " +
+      "TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 " +
+      "GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, c4, e")
+  }
+
+  @Test
+  def testAggOnWindow5(): Unit = {
+    util.verifyPlan("SELECT a4, b4, c4, COUNT(*) FROM " +
+      "(SELECT a4, c4, VAR_POP(b4) AS b4, " +
+      "TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, " +
+      "TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 " +
+      "GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, b4, c4")
+  }
+
+  @Test
+  def testAggWithGroupingSets1(): Unit = {
+    util.verifyPlan("SELECT a1, b1, c1, COUNT(d1) FROM T1 " +
+      "GROUP BY GROUPING SETS ((a1, b1), (a1, c1))")
+  }
+
+  @Test
+  def testAggWithGroupingSets2(): Unit = {
+    util.verifyPlan("SELECT a1, SUM(b1) AS s FROM T1 GROUP BY GROUPING SETS((a1, c1), (a1), ())")
+  }
+
+  @Test
+  def testAggWithGroupingSets3(): Unit = {
+    util.verifyPlan("SELECT a1, b1, c1, COUNT(d1) FROM T1 " +
+      "GROUP BY GROUPING SETS ((a1, b1, c1), (a1, b1, d1))")
+  }
+
+  @Test
+  def testAggWithRollup(): Unit = {
+    util.verifyPlan("SELECT a1, b1, c1, COUNT(d1) FROM T1 GROUP BY ROLLUP (a1, b1, c1)")
+  }
+
+  @Test
+  def testAggWithCube(): Unit = {
+    util.verifyPlan("SELECT a1, b1, c1, COUNT(d1) FROM T1 GROUP BY CUBE (a1, b1, c1)")
+  }
+
+  @Test
+  def testSingleDistinctAgg1(): Unit = {
+    util.verifyPlan("SELECT a1, COUNT(DISTINCT c1) FROM T1 GROUP BY a1")
+  }
+
+  @Test
+  def testSingleDistinctAgg2(): Unit = {
+    util.verifyPlan("SELECT a1, b1, COUNT(DISTINCT c1) FROM T1 GROUP BY a1, b1")
+  }
+
+  @Test
+  def testSingleDistinctAgg_WithNonDistinctAgg1(): Unit = {
+    util.verifyPlan("SELECT a1, COUNT(DISTINCT b1), SUM(b1) FROM T1 GROUP BY a1")
+  }
+
+  @Test
+  def testSingleDistinctAgg_WithNonDistinctAgg2(): Unit = {
+    util.verifyPlan("SELECT a1, c1, COUNT(DISTINCT b1), SUM(b1) FROM T1 GROUP BY a1, c1")
+  }
+
+  @Test
+  def testSingleDistinctAgg_WithNonDistinctAgg3(): Unit = {
+    util.verifyPlan("SELECT a1, COUNT(DISTINCT c1), SUM(b1) FROM T1 GROUP BY a1")
+  }
+
+  @Test
+  def testSingleDistinctAgg_WithNonDistinctAgg4(): Unit = {
+    util.verifyPlan("SELECT a1, d1, COUNT(DISTINCT c1), SUM(b1) FROM T1 GROUP BY a1, d1")
+  }
+
+  @Test
+  def testMultiDistinctAggs1(): Unit = {
+    util.verifyPlan("SELECT a1, COUNT(DISTINCT b1), SUM(DISTINCT b1) FROM T1 GROUP BY a1")
+  }
+
+  @Test
+  def testMultiDistinctAggs2(): Unit = {
+    util.verifyPlan("SELECT a1, d1, COUNT(DISTINCT c1), SUM(DISTINCT b1) FROM T1 GROUP BY a1, d1")
+  }
+
+  @Test
+  def testMultiDistinctAggs3(): Unit = {
+    util.verifyPlan(
+      "SELECT a1, SUM(DISTINCT b1), MAX(DISTINCT b1), MIN(DISTINCT c1) FROM T1 GROUP BY a1")
+  }
+
+  @Test
+  def testMultiDistinctAggs_WithNonDistinctAgg1(): Unit = {
+    util.verifyPlan(
+      "SELECT a1, d1, COUNT(DISTINCT c1), MAX(DISTINCT b1), SUM(b1) FROM T1 GROUP BY a1, d1")
+  }
+
+}
+
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRuleTest.scala
new file mode 100644
index 0000000..7c826cf
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRuleTest.scala
@@ -0,0 +1,48 @@
+/*
+ * 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.flink.table.plan.rules.logical
+
+import org.apache.flink.table.calcite.CalciteConfig
+import org.apache.flink.table.plan.common.AggregateReduceGroupingTestBase
+import org.apache.flink.table.plan.optimize.program.FlinkBatchProgram
+
+import org.apache.calcite.tools.RuleSets
+import org.junit.Before
+
+/**
+  * Test for [[AggregateReduceGroupingRule]].
+  */
+class AggregateReduceGroupingRuleTest extends AggregateReduceGroupingTestBase {
+
+  @Before
+  override def setup(): Unit = {
+    util.buildBatchProgram(FlinkBatchProgram.LOGICAL_REWRITE)
+
+    // remove FlinkAggregateRemoveRule to prevent the agg from removing
+    val programs = util.getTableEnv.getConfig.getCalciteConfig.getBatchProgram
+      .getOrElse(FlinkBatchProgram.buildProgram(util.getTableEnv.getConfig.getConf))
+    programs.getFlinkRuleSetProgram(FlinkBatchProgram.LOGICAL).get
+      .remove(RuleSets.ofList(FlinkAggregateRemoveRule.INSTANCE))
+    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
+      .replaceBatchProgram(programs).build()
+    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+
+    super.setup()
+  }
+}
+
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala
new file mode 100644
index 0000000..8e901b7
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala
@@ -0,0 +1,55 @@
+/*
+ * 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.flink.table.plan.rules.logical
+
+import org.apache.flink.table.calcite.CalciteConfig
+import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
+
+import org.apache.calcite.plan.hep.HepMatchOrder
+import org.apache.calcite.rel.rules.{FilterCalcMergeRule, FilterToCalcRule, ProjectCalcMergeRule, ProjectToCalcRule}
+import org.apache.calcite.tools.RuleSets
+
+/**
+  * Test for [[PruneAggregateCallRule]]#CALC_ON_AGGREGATE.
+  */
+class CalcPruneAggregateCallRuleTest extends PruneAggregateCallRuleTestBase {
+
+  override def setup(): Unit = {
+    super.setup()
+    util.buildBatchProgram(FlinkBatchProgram.LOGICAL)
+
+    val programs = util.getTableEnv.getConfig.getCalciteConfig.getBatchProgram.get
+    programs.addLast("rules",
+      FlinkHepRuleSetProgramBuilder.newBuilder
+      .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
+      .setHepMatchOrder(HepMatchOrder.BOTTOM_UP)
+      .add(RuleSets.ofList(
+        AggregateReduceGroupingRule.INSTANCE,
+        FilterCalcMergeRule.INSTANCE,
+        ProjectCalcMergeRule.INSTANCE,
+        FilterToCalcRule.INSTANCE,
+        ProjectToCalcRule.INSTANCE,
+        FlinkCalcMergeRule.INSTANCE,
+        PruneAggregateCallRule.CALC_ON_AGGREGATE)
+      ).build())
+
+    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
+      .replaceBatchProgram(programs).build()
+    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+  }
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateInnerJoinTransposeRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateInnerJoinTransposeRuleTest.scala
new file mode 100644
index 0000000..ea096e3
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateInnerJoinTransposeRuleTest.scala
@@ -0,0 +1,150 @@
+/*
+ * 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.flink.table.plan.rules.logical
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.calcite.CalciteConfig
+import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkGroupProgramBuilder, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
+import org.apache.flink.table.plan.stats.FlinkStatistic
+import org.apache.flink.table.util.TableTestBase
+
+import com.google.common.collect.ImmutableSet
+import org.apache.calcite.plan.hep.HepMatchOrder
+import org.apache.calcite.rel.rules._
+import org.apache.calcite.tools.RuleSets
+import org.junit.{Before, Test}
+
+/**
+  * Test for [[FlinkAggregateJoinTransposeRule]].
+  * this class only test inner join.
+  */
+class FlinkAggregateInnerJoinTransposeRuleTest extends TableTestBase {
+  private val util = batchTestUtil()
+
+  @Before
+  def setup(): Unit = {
+    val program = new FlinkChainedProgram[BatchOptimizeContext]()
+    program.addLast(
+      "rules",
+      FlinkGroupProgramBuilder.newBuilder[BatchOptimizeContext]
+        .addProgram(
+          FlinkHepRuleSetProgramBuilder.newBuilder
+            .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
+            .setHepMatchOrder(HepMatchOrder.BOTTOM_UP)
+            .add(RuleSets.ofList(AggregateReduceGroupingRule.INSTANCE
+            )).build(), "reduce unless grouping")
+        .addProgram(
+          FlinkHepRuleSetProgramBuilder.newBuilder
+            .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
+            .setHepMatchOrder(HepMatchOrder.BOTTOM_UP)
+            .add(RuleSets.ofList(
+              AggregateReduceGroupingRule.INSTANCE,
+              FlinkFilterJoinRule.FILTER_ON_JOIN,
+              FlinkFilterJoinRule.JOIN,
+              FilterAggregateTransposeRule.INSTANCE,
+              FilterProjectTransposeRule.INSTANCE,
+              FilterMergeRule.INSTANCE,
+              AggregateProjectMergeRule.INSTANCE,
+              FlinkAggregateJoinTransposeRule.EXTENDED
+            )).build(), "aggregate join transpose")
+        .build()
+    )
+    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
+      .replaceBatchProgram(program).build()
+    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+
+    util.addTableSource[(Int, Int, String)]("T", 'a, 'b, 'c)
+    util.addTableSource("T2",
+      Array[TypeInformation[_]](Types.INT, Types.INT, Types.STRING),
+      Array("a2", "b2", "c2"),
+      FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("b2"))).build()
+    )
+  }
+
+  @Test
+  def testPushCountAggThroughJoinOverUniqueColumn(): Unit = {
+    util.verifyPlan("SELECT COUNT(A.a) FROM (SELECT DISTINCT a FROM T) AS A JOIN T AS B ON A.a=B.a")
+  }
+
+  @Test
+  def testPushSumAggThroughJoinOverUniqueColumn(): Unit = {
+    util.verifyPlan("SELECT SUM(A.a) FROM (SELECT DISTINCT a FROM T) AS A JOIN T AS B ON A.a=B.a")
+  }
+
+  @Test
+  def testPushAggThroughJoinWithUniqueJoinKey(): Unit = {
+    val sqlQuery =
+      """
+        |WITH T1 AS (SELECT a AS a1, COUNT(b) AS b1 FROM T GROUP BY a),
+        |     T2 AS (SELECT COUNT(a) AS a2, b AS b2 FROM T GROUP BY b)
+        |SELECT MIN(a1), MIN(b1), MIN(a2), MIN(b2), a, b, COUNT(c) FROM
+        |  (SELECT * FROM T1, T2, T WHERE a1 = b2 AND a1 = a) t GROUP BY a, b
+      """.stripMargin
+    util.verifyPlan(sqlQuery)
+  }
+
+  @Test
+  def testSomeAggCallColumnsAndJoinConditionColumnsIsSame(): Unit = {
+    val sqlQuery =
+      """
+        |SELECT MIN(a2), MIN(b2), a, b, COUNT(c2) FROM
+        |    (SELECT * FROM T2, T WHERE b2 = a) t GROUP BY a, b
+      """.stripMargin
+    util.verifyPlan(sqlQuery)
+  }
+
+  @Test
+  def testAggregateWithAuxGroup_JoinKeyIsUnique1(): Unit = {
+    val sqlQuery =
+      """
+        |SELECT a2, b2, c2, SUM(a) FROM (SELECT * FROM T2, T WHERE b2 = b) GROUP BY a2, b2, c2
+      """.stripMargin
+    util.verifyPlan(sqlQuery)
+  }
+
+  @Test
+  def testAggregateWithAuxGroup_JoinKeyIsUnique2(): Unit = {
+    val sqlQuery =
+      """
+        |SELECT a2, b2, c, SUM(a) FROM (SELECT * FROM T2, T WHERE b2 = b) GROUP BY a2, b2, c
+      """.stripMargin
+    util.verifyPlan(sqlQuery)
+  }
+
+  @Test
+  def testAggregateWithAuxGroup_JoinKeyIsNotUnique1(): Unit = {
+    val sqlQuery =
+      """
+        |SELECT a2, b2, c2, SUM(a) FROM (SELECT * FROM T2, T WHERE a2 = a) GROUP BY a2, b2, c2
+      """.stripMargin
+    util.verifyPlan(sqlQuery)
+  }
+
+  @Test
+  def testAggregateWithAuxGroup_JoinKeyIsNotUnique2(): Unit = {
+    val sqlQuery =
+      """
+        |SELECT a2, b2, c, SUM(a) FROM (SELECT * FROM T2, T WHERE a2 = a) GROUP BY a2, b2, c
+      """.stripMargin
+    util.verifyPlan(sqlQuery)
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateOuterJoinTransposeRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateOuterJoinTransposeRuleTest.scala
new file mode 100644
index 0000000..486bea1
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateOuterJoinTransposeRuleTest.scala
@@ -0,0 +1,124 @@
+/*
+ * 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.flink.table.plan.rules.logical
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.calcite.CalciteConfig
+import org.apache.flink.table.plan.optimize.program.{FlinkChainedProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE, StreamOptimizeContext}
+import org.apache.flink.table.util.TableTestBase
+
+import org.apache.calcite.plan.hep.HepMatchOrder
+import org.apache.calcite.rel.rules._
+import org.apache.calcite.tools.RuleSets
+import org.junit.{Before, Test}
+
+/**
+  * Test for [[FlinkAggregateJoinTransposeRule]].
+  * this class only test left/right outer join.
+  */
+class FlinkAggregateOuterJoinTransposeRuleTest extends TableTestBase {
+
+  private val util = streamTestUtil()
+
+  @Before
+  def setup(): Unit = {
+    val program = new FlinkChainedProgram[StreamOptimizeContext]()
+    program.addLast(
+      "rules",
+      FlinkHepRuleSetProgramBuilder.newBuilder
+        .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
+        .setHepMatchOrder(HepMatchOrder.BOTTOM_UP)
+        .add(RuleSets.ofList(
+          FlinkFilterJoinRule.FILTER_ON_JOIN,
+          FlinkFilterJoinRule.JOIN,
+          FilterAggregateTransposeRule.INSTANCE,
+          FilterProjectTransposeRule.INSTANCE,
+          FilterMergeRule.INSTANCE,
+          AggregateProjectMergeRule.INSTANCE,
+          FlinkAggregateJoinTransposeRule.LEFT_RIGHT_OUTER_JOIN_EXTENDED
+        ))
+        .build()
+    )
+    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
+      .replaceStreamProgram(program).build()
+    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+
+    util.addTableSource[(Int, Long, String, Int)]("T", 'a, 'b, 'c, 'd)
+  }
+
+  @Test
+  def testPushCountAggThroughJoinOverUniqueColumn(): Unit = {
+    util.verifyPlan("SELECT COUNT(A.a) FROM (SELECT DISTINCT a FROM T) AS A JOIN T AS B ON A.a=B.a")
+  }
+
+  @Test
+  def testPushSumAggThroughJoinOverUniqueColumn(): Unit = {
+    util.verifyPlan("SELECT SUM(A.a) FROM (SELECT DISTINCT a FROM T) AS A JOIN T AS B ON A.a=B.a")
+  }
+
+  @Test
+  def testPushCountAggThroughLeftJoinOverUniqueColumn(): Unit = {
+    val sqlQuery = "SELECT COUNT(A.a) FROM (SELECT DISTINCT a FROM T) AS A " +
+      "LEFT OUTER JOIN T AS B ON A.a=B.a"
+    util.verifyPlan(sqlQuery)
+  }
+
+  @Test
+  def testPushSumAggThroughLeftJoinOverUniqueColumn(): Unit = {
+    val sqlQuery = "SELECT SUM(A.a) FROM (SELECT DISTINCT a FROM T) AS A " +
+      "LEFT OUTER JOIN T AS B ON A.a=B.a"
+    util.verifyPlan(sqlQuery)
+  }
+
+  @Test
+  def testPushCountAllAggThroughLeftJoinOverUniqueColumn(): Unit = {
+    val sqlQuery = "SELECT COUNT(*) FROM (SELECT DISTINCT a FROM T) AS A " +
+      "LEFT OUTER JOIN T AS B ON A.a=B.a"
+    util.verifyPlan(sqlQuery)
+  }
+
+  @Test
+  def testPushCountAggThroughLeftJoinOverUniqueColumnAndGroupByLeft(): Unit = {
+    val sqlQuery = "SELECT COUNT(B.b) FROM (SELECT DISTINCT a FROM T) AS A " +
+      "LEFT OUTER JOIN T AS B ON A.a=B.a GROUP BY A.a"
+    util.verifyPlan(sqlQuery)
+  }
+
+  @Test
+  def testPushCountAggThroughLeftJoinOverUniqueColumnAndGroupByRight(): Unit = {
+    val sqlQuery = "SELECT COUNT(B.b) FROM (SELECT DISTINCT a FROM T) AS A " +
+      "LEFT OUTER JOIN T AS B ON A.a=B.a GROUP BY B.a"
+    util.verifyPlan(sqlQuery)
+  }
+
+  @Test
+  def testPushCountAggThroughLeftJoinAndGroupByLeft(): Unit = {
+    val sqlQuery = "SELECT COUNT(B.b) FROM (SELECT a FROM T) AS A " +
+      "LEFT OUTER JOIN T AS B ON A.a=B.a GROUP BY A.a"
+    util.verifyPlan(sqlQuery)
+  }
+
+  @Test
+  def testPushCountAggThroughRightJoin(): Unit = {
+    val sqlQuery = "SELECT COUNT(B.b) FROM T AS B RIGHT OUTER JOIN (SELECT a FROM T) AS A " +
+      "ON A.a=B.a GROUP BY A.a"
+    util.verifyPlan(sqlQuery)
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala
new file mode 100644
index 0000000..1c69669
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala
@@ -0,0 +1,237 @@
+/*
+ * 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.flink.table.plan.rules.logical
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.calcite.CalciteConfig
+import org.apache.flink.table.plan.nodes.FlinkConventions
+import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalAggregate, FlinkLogicalCalc, FlinkLogicalExpand, FlinkLogicalJoin, FlinkLogicalSink, FlinkLogicalTableSourceScan, FlinkLogicalValues}
+import org.apache.flink.table.plan.optimize.program._
+import org.apache.flink.table.plan.rules.FlinkBatchRuleSets
+import org.apache.flink.table.plan.stats.FlinkStatistic
+import org.apache.flink.table.util.TableTestBase
+
+import com.google.common.collect.ImmutableSet
+import org.apache.calcite.plan.hep.HepMatchOrder
+import org.apache.calcite.rel.rules.{FilterCalcMergeRule, FilterToCalcRule, ProjectCalcMergeRule, ProjectToCalcRule, ReduceExpressionsRule}
+import org.apache.calcite.tools.RuleSets
+import org.junit.{Before, Test}
+
+/**
+  * Test for [[FlinkAggregateRemoveRule]].
+  */
+class FlinkAggregateRemoveRuleTest extends TableTestBase {
+  private val util = batchTestUtil()
+
+  @Before
+  def setup(): Unit = {
+    val programs = new FlinkChainedProgram[BatchOptimizeContext]()
+    programs.addLast(
+      // rewrite sub-queries to joins
+      "subquery_rewrite",
+      FlinkGroupProgramBuilder.newBuilder[BatchOptimizeContext]
+        .addProgram(FlinkHepRuleSetProgramBuilder.newBuilder
+          .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE)
+          .setHepMatchOrder(HepMatchOrder.BOTTOM_UP)
+          .add(FlinkBatchRuleSets.SEMI_JOIN_RULES)
+          .build(), "rewrite sub-queries to semi/anti join")
+        .build())
+
+    programs.addLast(
+      "rules",
+      // use volcano planner because
+      // rel.getCluster.getPlanner is volcano planner used in FlinkAggregateRemoveRule
+      FlinkVolcanoProgramBuilder.newBuilder
+        .add(RuleSets.ofList(
+          ReduceExpressionsRule.FILTER_INSTANCE,
+          FlinkAggregateExpandDistinctAggregatesRule.INSTANCE,
+          FilterCalcMergeRule.INSTANCE,
+          ProjectCalcMergeRule.INSTANCE,
+          FilterToCalcRule.INSTANCE,
+          ProjectToCalcRule.INSTANCE,
+          FlinkCalcMergeRule.INSTANCE,
+          FlinkAggregateRemoveRule.INSTANCE,
+          DecomposeGroupingSetsRule.INSTANCE,
+          AggregateReduceGroupingRule.INSTANCE,
+          FlinkLogicalAggregate.BATCH_CONVERTER,
+          FlinkLogicalCalc.CONVERTER,
+          FlinkLogicalJoin.CONVERTER,
+          FlinkLogicalValues.CONVERTER,
+          FlinkLogicalExpand.CONVERTER,
+          FlinkLogicalTableSourceScan.CONVERTER,
+          FlinkLogicalSink.CONVERTER))
+        .setRequiredOutputTraits(Array(FlinkConventions.LOGICAL))
+        .build())
+    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
+      .replaceBatchProgram(programs).build()
+    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+
+    util.addTableSource[(Int, Int, String)]("MyTable1", 'a, 'b, 'c)
+    util.addTableSource("MyTable2",
+      Array[TypeInformation[_]](Types.INT, Types.INT, Types.STRING),
+      Array("a", "b", "c"),
+      FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("a"))).build()
+    )
+    util.addTableSource("MyTable3",
+      Array[TypeInformation[_]](Types.INT, Types.INT, Types.STRING, Types.STRING),
+      Array("a", "b", "c", "d"),
+      FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("a"))).build()
+    )
+  }
+
+  @Test
+  def testAggRemove_GroupKeyIsNotUnique(): Unit = {
+    // can not remove agg
+    util.verifyPlan("SELECT a, MAX(c) from MyTable1 GROUP BY a")
+  }
+
+  @Test
+  def testAggRemove_WithoutFilter1(): Unit = {
+    util.verifyPlan("SELECT a, b + 1, c, s FROM (" +
+      "SELECT a, MIN(b) AS b, SUM(b) AS s, MAX(c) AS c FROM MyTable2 GROUP BY a)")
+  }
+
+  @Test
+  def testAggRemove_WithoutFilter2(): Unit = {
+    util.verifyPlan("SELECT a, SUM(b) AS s FROM MyTable2 GROUP BY a")
+  }
+
+  @Test
+  def testAggRemove_WithoutGroupBy1(): Unit = {
+    // can not remove agg
+    util.verifyPlan("SELECT MAX(a), SUM(b), MIN(c) FROM MyTable2")
+  }
+
+  @Test
+  def testAggRemove_WithoutGroupBy2(): Unit = {
+    util.verifyPlan("SELECT MAX(a), SUM(b), MIN(c) FROM (VALUES (1, 2, 3)) T(a, b, c)")
+  }
+
+  @Test
+  def testAggRemove_WithoutGroupBy3(): Unit = {
+    // can not remove agg
+    util.verifyPlan("SELECT * FROM MyTable2 WHERE EXISTS (SELECT SUM(a) FROM MyTable1 WHERE 1=2)")
+  }
+
+  @Test
+  def testAggRemove_WithoutGroupBy4(): Unit = {
+    // can not remove agg
+    util.verifyPlan("SELECT SUM(a) FROM (SELECT a FROM MyTable2 WHERE 1=2)")
+  }
+
+  @Test
+  def testAggRemove_WithoutAggCall(): Unit = {
+    util.verifyPlan("SELECT a, b FROM MyTable2 GROUP BY a, b")
+  }
+
+  @Test
+  def testAggRemove_WithFilter(): Unit = {
+    // can not remove agg
+    util.verifyPlan("SELECT a, MIN(c) FILTER (WHERE b > 0), MAX(b) FROM MyTable2 GROUP BY a")
+  }
+
+  @Test
+  def testAggRemove_Count(): Unit = {
+    // can not remove agg
+    util.verifyPlan("SELECT a, COUNT(c) FROM MyTable2 GROUP BY a")
+  }
+
+  @Test
+  def testAggRemove_CountStar(): Unit = {
+    // can not remove agg
+    util.verifyPlan("SELECT a, COUNT(*) FROM MyTable2 GROUP BY a")
+  }
+
+  @Test
+  def testAggRemove_GroupSets1(): Unit = {
+    // a is unique
+    util.verifyPlan("SELECT a, SUM(b) AS s FROM MyTable3 GROUP BY GROUPING SETS((a, c), (a, d))")
+  }
+
+  @Test
+  def testAggRemove_GroupSets2(): Unit = {
+    // can not remove agg
+    util.verifyPlan("SELECT a, SUM(b) AS s FROM MyTable3 GROUP BY GROUPING SETS((a, c), (a), ())")
+  }
+
+  @Test
+  def testAggRemove_Rollup(): Unit = {
+    // can not remove agg
+    util.verifyPlan("SELECT a, SUM(b) AS s FROM MyTable3 GROUP BY ROLLUP(a, c, d)")
+  }
+
+  @Test
+  def testAggRemove_Cube(): Unit = {
+    // can not remove agg
+    util.verifyPlan("SELECT a, SUM(b) AS s FROM MyTable3 GROUP BY CUBE(a, c, d)")
+  }
+
+  @Test
+  def testAggRemove_SingleDistinctAgg1(): Unit = {
+    util.verifyPlan("SELECT a, COUNT(DISTINCT c) FROM MyTable2 GROUP BY a")
+  }
+
+  @Test
+  def testAggRemove_SingleDistinctAgg2(): Unit = {
+    util.verifyPlan("SELECT a, COUNT(DISTINCT c) FROM MyTable2 GROUP BY a, b")
+  }
+
+  @Test
+  def testAggRemove_SingleDistinctAgg_WithNonDistinctAgg1(): Unit = {
+    util.verifyPlan("SELECT a, COUNT(DISTINCT b), SUM(b) FROM MyTable2 GROUP BY a")
+  }
+
+  @Test
+  def testAggRemove_SingleDistinctAgg_WithNonDistinctAgg2(): Unit = {
+    util.verifyPlan("SELECT a, COUNT(DISTINCT b), SUM(b) FROM MyTable2 GROUP BY a, c")
+  }
+
+  @Test
+  def testAggRemove_SingleDistinctAgg_WithNonDistinctAgg3(): Unit = {
+    util.verifyPlan("SELECT a, COUNT(DISTINCT c), SUM(b) FROM MyTable3 GROUP BY a")
+  }
+
+  @Test
+  def testAggRemove_SingleDistinctAgg_WithNonDistinctAgg4(): Unit = {
+    util.verifyPlan("SELECT a, COUNT(DISTINCT c), SUM(b) FROM MyTable3 GROUP BY a, d")
+  }
+
+  @Test
+  def testAggRemove_MultiDistinctAggs1(): Unit = {
+    util.verifyPlan("SELECT a, COUNT(DISTINCT b), SUM(DISTINCT b) FROM MyTable2 GROUP BY a")
+  }
+
+  @Test
+  def testAggRemove_MultiDistinctAggs2(): Unit = {
+    util.verifyPlan("SELECT a, COUNT(DISTINCT c), SUM(DISTINCT b) FROM MyTable3 GROUP BY a, d")
+  }
+
+  @Test
+  def testAggRemove_MultiDistinctAggs3(): Unit = {
+    util.verifyPlan(
+      "SELECT a, SUM(DISTINCT b), MAX(DISTINCT b), MIN(DISTINCT c) FROM MyTable2 GROUP BY a")
+  }
+
+  @Test
+  def testAggRemove_MultiDistinctAggs_WithNonDistinctAgg1(): Unit = {
+    util.verifyPlan("SELECT a, COUNT(DISTINCT c), SUM(b) FROM MyTable3 GROUP BY a, d")
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
index b70cdaa..dfb986f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.rules.logical
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.api.{PlannerConfigOptions, Types}
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.plan.stats.{ColumnStats, TableStats}
+import org.apache.flink.table.plan.stats.{ColumnStats, FlinkStatistic, TableStats}
 import org.apache.flink.table.util.TableTestBase
 
 import org.apache.calcite.plan.hep.HepMatchOrder
@@ -55,19 +55,19 @@ class JoinDeriveNullFilterRuleTest extends TableTestBase {
     util.addTableSource("MyTable1",
       Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING, Types.INT, Types.LONG),
       Array("a1", "b1", "c1", "d1", "e1"),
-      Some(new TableStats(1000000000, Map(
+      FlinkStatistic.builder().tableStats(new TableStats(1000000000, Map(
         "a1" -> new ColumnStats(null, 10000000L, 4.0, 4, null, null),
         "c1" -> new ColumnStats(null, 5000000L, 10.2, 16, null, null),
         "e1" -> new ColumnStats(null, 500000L, 8.0, 8, null, null)
-      ))))
+      ))).build())
     util.addTableSource("MyTable2",
       Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING, Types.INT, Types.LONG),
       Array("a2", "b2", "c2", "d2", "e2"),
-      Some(new TableStats(2000000000, Map(
+      FlinkStatistic.builder().tableStats(new TableStats(2000000000, Map(
         "b2" -> new ColumnStats(null, 10000000L, 8.0, 8, null, null),
         "c2" -> new ColumnStats(null, 3000000L, 18.6, 32, null, null),
         "e2" -> new ColumnStats(null, 1500000L, 8.0, 8, null, null)
-      ))))
+      ))).build())
   }
 
   @Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala
new file mode 100644
index 0000000..01893f3
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala
@@ -0,0 +1,51 @@
+/*
+ * 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.flink.table.plan.rules.logical
+
+import org.apache.flink.table.calcite.CalciteConfig
+import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
+
+import org.apache.calcite.plan.hep.HepMatchOrder
+import org.apache.calcite.rel.rules.ProjectFilterTransposeRule
+import org.apache.calcite.tools.RuleSets
+
+/**
+  * Test for [[PruneAggregateCallRule]]#PROJECT_ON_AGGREGATE.
+  */
+class ProjectPruneAggregateCallRuleTest extends PruneAggregateCallRuleTestBase {
+
+  override def setup(): Unit = {
+    super.setup()
+    util.buildBatchProgram(FlinkBatchProgram.LOGICAL)
+
+    val programs = util.getTableEnv.getConfig.getCalciteConfig.getBatchProgram.get
+    programs.addLast("rules",
+      FlinkHepRuleSetProgramBuilder.newBuilder
+        .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
+        .setHepMatchOrder(HepMatchOrder.BOTTOM_UP)
+        .add(RuleSets.ofList(
+          AggregateReduceGroupingRule.INSTANCE,
+          ProjectFilterTransposeRule.INSTANCE,
+          PruneAggregateCallRule.PROJECT_ON_AGGREGATE)
+        ).build())
+
+    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
+      .replaceBatchProgram(programs).build()
+    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+  }
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PruneAggregateCallRuleTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PruneAggregateCallRuleTestBase.scala
new file mode 100644
index 0000000..1a3e9dc
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PruneAggregateCallRuleTestBase.scala
@@ -0,0 +1,175 @@
+/*
+ * 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.flink.table.plan.rules.logical
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.plan.stats.FlinkStatistic
+import org.apache.flink.table.util.{BatchTableTestUtil, TableTestBase}
+
+import com.google.common.collect.ImmutableSet
+import org.junit.{Before, Test}
+
+/**
+  * Base test class for [[PruneAggregateCallRule]].
+  */
+abstract class PruneAggregateCallRuleTestBase extends TableTestBase {
+  protected val util: BatchTableTestUtil = batchTestUtil()
+
+  @Before
+  def setup(): Unit = {
+    util.addTableSource("T1",
+      Array[TypeInformation[_]](Types.INT, Types.INT, Types.STRING, Types.INT),
+      Array("a1", "b1", "c1", "d1"),
+      FlinkStatistic.builder().uniqueKeys(ImmutableSet.of(ImmutableSet.of("a1"))).build()
+    )
+    util.addTableSource[(Int, Int, String, Long)]("T2", 'a2, 'b2, 'c2, 'd2)
+  }
+
+  @Test
+  def testPruneRegularAggCall_WithoutFilter1(): Unit = {
+    val sql =
+      """
+        |SELECT a2, b2, d2 FROM
+        | (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+      """.stripMargin
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testPruneRegularAggCall_WithoutFilter2(): Unit = {
+    val sql =
+      """
+        |SELECT b2, a2, d2 FROM
+        | (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+      """.stripMargin
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testPruneRegularAggCall_WithoutFilter3(): Unit = {
+    val sql =
+      """
+        |SELECT a2 as a, b2, d2 FROM
+        | (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+      """.stripMargin
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testPruneRegularAggCall_WithFilter1(): Unit = {
+    val sql =
+      """
+        |SELECT a2, b2, d2 FROM
+        | (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+        |WHERE d2 > 0
+      """.stripMargin
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testPruneRegularAggCall_WithFilter2(): Unit = {
+    val sql =
+      """
+        |SELECT b2, a2, d2 FROM
+        | (SELECT a2, b2, COUNT(c2) as c2, SUM(d2) as d2 FROM T2 GROUP BY a2, b2) t
+        |WHERE d2 > 0
+      """.stripMargin
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testPruneAuxGroupAggCall_WithoutFilter1(): Unit = {
+    val sql =
+      """
+        |SELECT a1, c1 FROM
+        | (SELECT a1, b1, COUNT(c1) as c1, SUM(d1) as d1 FROM T1 GROUP BY a1, b1) t
+      """.stripMargin
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testPruneAuxGroupAggCall_WithoutFilter2(): Unit = {
+    val sql =
+      """
+        |SELECT c1, a1 FROM
+        | (SELECT a1, b1, COUNT(c1) as c1, SUM(d1) as d1 FROM T1 GROUP BY a1, b1) t
+      """.stripMargin
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testPruneAuxGroupAggCall_WithFilter1(): Unit = {
+    val sql =
+      """
+        |SELECT a1, c1 FROM
+        | (SELECT a1, b1, COUNT(c1) as c1, SUM(d1) as d1 FROM T1 GROUP BY a1, b1) t
+        |WHERE c1 > 10
+      """.stripMargin
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testPruneAuxGroupAggCall_WithFilter2(): Unit = {
+    val sql =
+      """
+        |SELECT c1, a1 FROM
+        | (SELECT a1, b1, COUNT(c1) as c1, SUM(d1) as d1 FROM T1 GROUP BY a1, b1) t
+        |WHERE c1 > 10
+      """.stripMargin
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testEmptyGroupKey_WithOneAggCall1(): Unit = {
+    val sql = "SELECT 1 FROM (SELECT SUM(a1) FROM T1) t"
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testEmptyGroupKey_WithOneAggCall2(): Unit = {
+    val sql = "SELECT * FROM T2 WHERE EXISTS (SELECT COUNT(*) FROM T1)"
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testEmptyGroupKey_WithOneAggCall3(): Unit = {
+    val sql = "SELECT * FROM T2 WHERE EXISTS (SELECT COUNT(*) FROM T1 WHERE 1=2)"
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testEmptyGroupKey_WithMoreThanOneAggCalls1(): Unit = {
+    val sql = "SELECT 1 FROM (SELECT SUM(a1), COUNT(*) FROM T1) t"
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testEmptyGroupKey_WithMoreThanOneAggCalls2(): Unit = {
+    val sql = "SELECT * FROM T2 WHERE EXISTS (SELECT SUM(a1), COUNT(*) FROM T1)"
+    util.verifyPlan(sql)
+  }
+
+  @Test
+  def testEmptyGroupKey_WithMoreThanOneAggCalls3(): Unit = {
+    val sql = "SELECT * FROM T2 WHERE EXISTS (SELECT SUM(a1), COUNT(*) FROM T1 WHERE 1=2)"
+    util.verifyPlan(sql)
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala
index bfeb27a..c732b13 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala
@@ -45,9 +45,9 @@ class CalcITCase extends BatchTestBase {
 
   @Before
   def before(): Unit = {
-    registerCollection("Table3", data3, type3, nullablesOfData3, "a, b, c")
-    registerCollection("NullTable3", nullData3, type3, nullablesOfData3, "a, b, c")
-    registerCollection("SmallTable3", smallData3, type3, nullablesOfData3, "a, b, c")
+    registerCollection("Table3", data3, type3, "a, b, c", nullablesOfData3)
+    registerCollection("NullTable3", nullData3, type3, "a, b, c", nullablesOfData3)
+    registerCollection("SmallTable3", smallData3, type3, "a, b, c", nullablesOfData3)
     registerCollection("testTable", buildInData, buildInType, "a,b,c,d,e,f,g,h,i,j")
   }
 
@@ -99,8 +99,9 @@ class CalcITCase extends BatchTestBase {
   def testManySelect(): Unit = {
     registerCollection(
       "ProjectionTestTable",
-      projectionTestData, projectionTestDataType, nullablesOfProjectionTestData,
-      "a, b, c, d, e, f, g, h")
+      projectionTestData, projectionTestDataType,
+      "a, b, c, d, e, f, g, h",
+      nullablesOfProjectionTestData)
     checkResult(
       """
         |SELECT
@@ -658,7 +659,7 @@ class CalcITCase extends BatchTestBase {
   def testValueConstructor(): Unit = {
     val data = Seq(row("foo", 12, UTCTimestamp("1984-07-12 14:34:24")))
     val tpe = new RowTypeInfo(STRING_TYPE_INFO, INT_TYPE_INFO, TIMESTAMP)
-    registerCollection("MyTable", data, tpe, Array(false, false, false), "a, b, c")
+    registerCollection("MyTable", data, tpe, "a, b, c" , Array(false, false, false))
 
     val table = parseQuery("SELECT ROW(a, b, c), ARRAY[12, b], MAP[a, c] FROM MyTable " +
         "WHERE (a, b, c) = ('foo', 12, TIMESTAMP '1984-07-12 14:34:24')")
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/LimitITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/LimitITCase.scala
index ef7ce80..22b2c5e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/LimitITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/LimitITCase.scala
@@ -29,7 +29,7 @@ class LimitITCase extends BatchTestBase {
   @Before
   def before(): Unit = {
     tEnv.getConfig.getConf.setInteger(TableConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 3)
-    registerCollection("Table3", data3, type3, nullablesOfData3, "a, b, c")
+    registerCollection("Table3", data3, type3, "a, b, c", nullablesOfData3)
 
     // TODO support LimitableTableSource
 //    val rowType = new RowTypeInfo(type3.getFieldTypes, Array("a", "b", "c"))
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/OverWindowITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/OverWindowITCase.scala
index 7ecea2f..5ca852f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/OverWindowITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/OverWindowITCase.scala
@@ -42,11 +42,11 @@ class OverWindowITCase extends BatchTestBase {
   @Before
   def before(): Unit = {
     tEnv.getConfig.getConf.setInteger(TableConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 3)
-    registerCollection("Table5", data5, type5, nullablesOfData5, "d, e, f, g, h")
+    registerCollection("Table5", data5, type5, "d, e, f, g, h", nullablesOfData5)
     registerCollection("ShuflledTable5",
-      Random.shuffle(data5), type5, nullablesOfData5, "sd, se, sf, sg, sh")
-    registerCollection("Table6", data6, type6, nullablesOfData6, "a, b, c, d, e, f")
-    registerCollection("NullTable5", nullData5, type5, nullablesOfNullData5, "d, e, f, g, h")
+      Random.shuffle(data5), type5, "sd, se, sf, sg, sh", nullablesOfData5)
+    registerCollection("Table6", data6, type6, "a, b, c, d, e, f", nullablesOfData6)
+    registerCollection("NullTable5", nullData5, type5, "d, e, f, g, h", nullablesOfNullData5)
   }
 
   @Test
@@ -1606,7 +1606,7 @@ class OverWindowITCase extends BatchTestBase {
       row(null, 3L, 3, "NullTuple", 3L),
       row(null, 3L, 3, "NullTuple", 3L)
     )
-    registerCollection("NullTable", nullData, type5, nullablesOfNullData5, "d, e, f, g, h")
+    registerCollection("NullTable", nullData, type5, "d, e, f, g, h", nullablesOfNullData5)
 
     checkResult(
       "SELECT h, d, count(*) over (partition by h order by d range between 0 PRECEDING and " +
@@ -1826,7 +1826,7 @@ class OverWindowITCase extends BatchTestBase {
       row(null, 3L, 3, "NullTuple", 3L),
       row(null, 3L, 3, "NullTuple", 3L)
     )
-    registerCollection("NullTable", nullData, type5, nullablesOfNullData5, "d, e, f, g, h")
+    registerCollection("NullTable", nullData, type5, "d, e, f, g, h", nullablesOfNullData5)
     checkResult(
       "SELECT h, d, count(*) over (partition by h order by d range between 1 PRECEDING and 2 " +
           "FOLLOWING) FROM NullTable",
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/RankITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/RankITCase.scala
index 2d529f4..bed1f38 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/RankITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/RankITCase.scala
@@ -32,8 +32,8 @@ class RankITCase extends BatchTestBase {
   @Before
   def before(): Unit = {
     tEnv.getConfig.getConf.setInteger(TableConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 3)
-    registerCollection("Table3", data3, type3, nullablesOfData3, "a, b, c")
-    registerCollection("Table5", data5, type5, nullablesOfData5, "a, b, c, d, e")
+    registerCollection("Table3", data3, type3, "a, b, c", nullablesOfData3)
+    registerCollection("Table5", data5, type5, "a, b, c, d, e", nullablesOfData5)
     registerCollection("Table2", data2_1, INT_DOUBLE, "a, b")
   }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
index 874a132..00f9fde 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
@@ -18,7 +18,6 @@
 
 package org.apache.flink.table.runtime.batch.sql
 
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{INT_TYPE_INFO, LONG_TYPE_INFO, STRING_TYPE_INFO}
 import org.apache.flink.table.`type`.InternalTypes
 import org.apache.flink.table.api.{PlannerConfigOptions, TableConfigOptions}
 import org.apache.flink.table.dataformat.BinaryString.fromString
@@ -46,9 +45,9 @@ class UnionITCase extends BatchTestBase {
   @Before
   def before(): Unit = {
     tEnv.getConfig.getConf.setInteger(TableConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 3)
-    registerCollection("Table3", smallData3, type3, nullablesOfSmallData3, "a, b, c")
-    registerCollection("Table5", data5, type5, nullablesOfData5, "d, e, f, g, h")
-    registerCollection("Table6", data6, type6, Array(false, false, false), "a, b, c")
+    registerCollection("Table3", smallData3, type3, "a, b, c", nullablesOfSmallData3)
+    registerCollection("Table5", data5, type5, "d, e, f, g, h", nullablesOfData5)
+    registerCollection("Table6", data6, type6, "a, b, c", Array(false, false, false))
     tEnv.getConfig.getConf.setString(
       TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
new file mode 100644
index 0000000..14abd2c
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
@@ -0,0 +1,205 @@
+/*
+ * 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.flink.table.runtime.batch.sql.agg
+
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.{TableConfigOptions, TableException, Types}
+import org.apache.flink.table.calcite.CalciteConfig
+import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkBatchProgram, FlinkGroupProgramBuilder, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
+import org.apache.flink.table.plan.rules.logical.{AggregateReduceGroupingRule, FlinkAggregateJoinTransposeRule}
+import org.apache.flink.table.plan.stats.FlinkStatistic
+import org.apache.flink.table.runtime.utils.BatchTestBase
+import org.apache.flink.table.runtime.utils.BatchTestBase.row
+import org.apache.flink.table.runtime.utils.TestData._
+
+import org.apache.calcite.plan.hep.HepMatchOrder
+import org.apache.calcite.rel.rules._
+import org.apache.calcite.tools.RuleSets
+import org.junit.{Before, Test}
+
+import scala.collection.JavaConverters._
+import scala.collection.Seq
+
+class AggregateJoinTransposeITCase extends BatchTestBase {
+
+  @Before
+  def before(): Unit = {
+    val programs = FlinkBatchProgram.buildProgram(tEnv.getConfig.getConf)
+    // remove FlinkAggregateJoinTransposeRule from logical program (volcano planner)
+    programs.getFlinkRuleSetProgram(FlinkBatchProgram.LOGICAL)
+      .getOrElse(throw new TableException(s"${FlinkBatchProgram.LOGICAL} does not exist"))
+      .remove(RuleSets.ofList(FlinkAggregateJoinTransposeRule.EXTENDED))
+
+    // add FlinkAggregateJoinTransposeRule to hep program
+    // to make sure that the aggregation must be pushed down
+    programs.addBefore(
+      FlinkBatchProgram.LOGICAL,
+      "FlinkAggregateJoinTransposeRule",
+      FlinkGroupProgramBuilder.newBuilder[BatchOptimizeContext]
+        .addProgram(
+          FlinkHepRuleSetProgramBuilder.newBuilder
+            .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
+            .setHepMatchOrder(HepMatchOrder.BOTTOM_UP)
+            .add(RuleSets.ofList(
+              AggregateReduceGroupingRule.INSTANCE
+            )).build(), "reduce unless grouping")
+        .addProgram(
+          FlinkHepRuleSetProgramBuilder.newBuilder
+            .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
+            .setHepMatchOrder(HepMatchOrder.BOTTOM_UP)
+            .add(RuleSets.ofList(
+              AggregateReduceGroupingRule.INSTANCE,
+              AggregateProjectMergeRule.INSTANCE,
+              FlinkAggregateJoinTransposeRule.EXTENDED
+            )).build(), "aggregate join transpose")
+        .build()
+    )
+    val calciteConfig = CalciteConfig.createBuilder(tEnv.getConfig.getCalciteConfig)
+      .replaceBatchProgram(programs).build()
+    tEnv.getConfig.setCalciteConfig(calciteConfig)
+
+    tEnv.getConfig.getConf.setInteger(TableConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 3)
+    // HashJoin is disabled due to translateToPlanInternal method is not implemented yet
+    tEnv.getConfig.getConf.setString(TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin")
+    registerCollection("T3", data3, type3, "a, b, c", nullablesOfData3)
+
+    registerCollection("MyTable",
+      Seq(row(1, 1L, "X"),
+        row(1, 2L, "Y"),
+        row(2, 3L, null),
+        row(2, 4L, "Z")),
+      new RowTypeInfo(Types.INT, Types.LONG, Types.STRING),
+      "a2, b2, c2",
+      Array(true, true, true),
+      FlinkStatistic.builder().uniqueKeys(Set(Set("b2").asJava).asJava).build()
+    )
+  }
+
+  @Test
+  def testPushCountAggThroughJoinOverUniqueColumn(): Unit = {
+    checkResult(
+      "SELECT COUNT(A.a) FROM (SELECT DISTINCT a FROM T3) AS A JOIN T3 AS B ON A.a=B.a",
+      Seq(row(21))
+    )
+  }
+
+  @Test
+  def testPushSumAggThroughJoinOverUniqueColumn(): Unit = {
+    checkResult(
+      "SELECT SUM(A.a) FROM (SELECT DISTINCT a FROM T3) AS A JOIN T3 AS B ON A.a=B.a",
+      Seq(row(231))
+    )
+  }
+
+  @Test
+  def testSomeAggCallColumnsAndJoinConditionColumnsIsSame(): Unit = {
+    checkResult(
+      "SELECT MIN(a2), MIN(b2), a, b, COUNT(c2) FROM " +
+        "(SELECT * FROM MyTable, T3 WHERE b2 = b) t GROUP BY b, a",
+      Seq(row(1, 1, 1, 1, 1), row(1, 2, 2, 2, 1), row(1, 2, 3, 2, 1),
+        row(2, 3, 4, 3, 0), row(2, 3, 5, 3, 0), row(2, 3, 6, 3, 0),
+        row(2, 4, 10, 4, 1), row(2, 4, 7, 4, 1), row(2, 4, 8, 4, 1), row(2, 4, 9, 4, 1))
+    )
+  }
+
+  @Test
+  def testAggregateWithAuxGroup_JoinKeyIsUnique1(): Unit = {
+    checkResult(
+      """
+        |select a2, b2, c2, SUM(a) FROM (
+        | SELECT * FROM MyTable, T3 WHERE b2 = b
+        |) GROUP BY a2, b2, c2
+      """.stripMargin,
+      Seq(row(1, 1, "X", 1), row(1, 2, "Y", 5), row(2, 3, null, 15), row(2, 4, "Z", 34)))
+
+    checkResult(
+      """
+        |select a2, b2, c2, SUM(a), COUNT(c) FROM (
+        | SELECT * FROM MyTable, T3 WHERE b2 = b
+        |) GROUP BY a2, b2, c2
+      """.stripMargin,
+      Seq(row(1, 1, "X", 1, 1), row(1, 2, "Y", 5, 2),
+        row(2, 3, null, 15, 3), row(2, 4, "Z", 34, 4)))
+  }
+
+  @Test
+  def testAggregateWithAuxGroup_JoinKeyIsUnique2(): Unit = {
+    checkResult(
+      """
+        |select a2, b2, c, SUM(a) FROM (
+        | SELECT * FROM MyTable, T3 WHERE b2 = b
+        |) GROUP BY a2, b2, c
+      """.stripMargin,
+      Seq(row(1, 1, "Hi", 1), row(1, 2, "Hello world", 3), row(1, 2, "Hello", 2),
+        row(2, 3, "Hello world, how are you?", 4), row(2, 3, "I am fine.", 5),
+        row(2, 3, "Luke Skywalker", 6), row(2, 4, "Comment#1", 7), row(2, 4, "Comment#2", 8),
+        row(2, 4, "Comment#3", 9), row(2, 4, "Comment#4", 10)))
+
+    checkResult(
+      """
+        |select a2, b2, c, SUM(a), MAX(b) FROM (
+        | SELECT * FROM MyTable, T3 WHERE b2 = b
+        |) GROUP BY a2, b2, c
+      """.stripMargin,
+      Seq(row(1, 1, "Hi", 1, 1), row(1, 2, "Hello world", 3, 2), row(1, 2, "Hello", 2, 2),
+        row(2, 3, "Hello world, how are you?", 4, 3), row(2, 3, "I am fine.", 5, 3),
+        row(2, 3, "Luke Skywalker", 6, 3), row(2, 4, "Comment#1", 7, 4),
+        row(2, 4, "Comment#2", 8, 4), row(2, 4, "Comment#3", 9, 4), row(2, 4, "Comment#4", 10, 4)))
+  }
+
+  @Test
+  def testAggregateWithAuxGroup_JoinKeyIsNotUnique1(): Unit = {
+    checkResult(
+      """
+        |select a2, b2, c2, SUM(a) FROM (
+        | SELECT * FROM MyTable, T3 WHERE a2 = a
+        |) GROUP BY a2, b2, c2
+      """.stripMargin,
+      Seq(row(1, 1, "X", 1), row(1, 2, "Y", 1), row(2, 3, null, 2), row(2, 4, "Z", 2)))
+
+    checkResult(
+      """
+        |select a2, b2, c2, SUM(a), COUNT(c) FROM (
+        | SELECT * FROM MyTable, T3 WHERE a2 = a
+        |) GROUP BY a2, b2, c2
+      """.stripMargin,
+      Seq(row(1, 1, "X", 1, 1), row(1, 2, "Y", 1, 1), row(2, 3, null, 2, 1), row(2, 4, "Z", 2, 1)))
+  }
+
+  @Test
+  def testAggregateWithAuxGroup_JoinKeyIsNotUnique2(): Unit = {
+    checkResult(
+      """
+        |select a2, b2, c, SUM(a) FROM (
+        | SELECT * FROM MyTable, T3 WHERE a2 = a
+        |) GROUP BY a2, b2, c
+      """.stripMargin,
+      Seq(row(1, 1, "Hi", 1), row(1, 2, "Hi", 1), row(2, 3, "Hello", 2), row(2, 4, "Hello", 2)))
+
+    checkResult(
+      """
+        |select a2, b2, c, SUM(a), MIN(b) FROM (
+        | SELECT * FROM MyTable, T3 WHERE a2 = a
+        |) GROUP BY a2, b2, c
+      """.stripMargin,
+      Seq(row(1, 1, "Hi", 1, 1), row(1, 2, "Hi", 1, 1),
+        row(2, 3, "Hello", 2, 2), row(2, 4, "Hello", 2, 2)))
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
new file mode 100644
index 0000000..670a6d1
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
@@ -0,0 +1,405 @@
+/*
+ * 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.flink.table.runtime.batch.sql.agg
+
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.{PlannerConfigOptions, TableConfigOptions, Types}
+import org.apache.flink.table.plan.stats.FlinkStatistic
+import org.apache.flink.table.runtime.utils.BatchTestBase
+import org.apache.flink.table.runtime.utils.BatchTestBase.row
+import org.apache.flink.table.util.DateTimeTestUtil.UTCTimestamp
+
+import org.junit.{Before, Test}
+
+import java.sql.Date
+
+import scala.collection.JavaConverters._
+import scala.collection.Seq
+
+class AggregateReduceGroupingITCase extends BatchTestBase {
+
+  @Before
+  def before(): Unit = {
+    registerCollection("T1",
+      Seq(row(2, 1, "A", null),
+        row(3, 2, "A", "Hi"),
+        row(5, 2, "B", "Hello"),
+        row(6, 3, "C", "Hello world")),
+      new RowTypeInfo(Types.INT, Types.INT, Types.STRING, Types.STRING),
+      "a1, b1, c1, d1",
+      Array(true, true, true, true),
+      FlinkStatistic.builder().uniqueKeys(Set(Set("a1").asJava).asJava).build()
+    )
+
+    registerCollection("T2",
+      Seq(row(1, 1, "X"),
+        row(1, 2, "Y"),
+        row(2, 3, null),
+        row(2, 4, "Z")),
+      new RowTypeInfo(Types.INT, Types.INT, Types.STRING),
+      "a2, b2, c2",
+      Array(true, true, true),
+      FlinkStatistic.builder()
+        .uniqueKeys(Set(Set("b2").asJava, Set("a2", "b2").asJava).asJava).build()
+    )
+
+    registerCollection("T3",
+      Seq(row(1, 10, "Hi", 1L),
+        row(2, 20, "Hello", 1L),
+        row(2, 20, "Hello world", 2L),
+        row(3, 10, "Hello world, how are you?", 1L),
+        row(4, 20, "I am fine.", 2L),
+        row(4, null, "Luke Skywalker", 2L)),
+      new RowTypeInfo(Types.INT, Types.INT, Types.STRING, Types.LONG),
+      "a3, b3, c3, d3",
+      Array(true, true, true, true),
+      FlinkStatistic.builder().uniqueKeys(Set(Set("a1").asJava).asJava).build()
+    )
+
+    registerCollection("T4",
+      Seq(row(1, 1, "A", UTCTimestamp("2018-06-01 10:05:30"), "Hi"),
+        row(2, 1, "B", UTCTimestamp("2018-06-01 10:10:10"), "Hello"),
+        row(3, 2, "B", UTCTimestamp("2018-06-01 10:15:25"), "Hello world"),
+        row(4, 3, "C", UTCTimestamp("2018-06-01 10:36:49"), "I am fine.")),
+      new RowTypeInfo(Types.INT, Types.INT, Types.STRING, Types.SQL_TIMESTAMP, Types.STRING),
+      "a4, b4, c4, d4, e4",
+      Array(true, true, true, true, true),
+      FlinkStatistic.builder().uniqueKeys(Set(Set("a4").asJava).asJava).build()
+    )
+
+    registerCollection("T5",
+      Seq(row(2, 1, "A", null),
+        row(3, 2, "B", "Hi"),
+        row(1, null, "C", "Hello"),
+        row(4, 3, "D", "Hello world"),
+        row(3, 1, "E", "Hello world, how are you?"),
+        row(5, null, "F", null),
+        row(7, 2, "I", "hahaha"),
+        row(6, 1, "J", "I am fine.")),
+      new RowTypeInfo(Types.INT, Types.INT, Types.STRING, Types.STRING),
+      "a5, b5, c5, d5",
+      Array(true, true, true, true),
+      FlinkStatistic.builder().uniqueKeys(Set(Set("c5").asJava).asJava).build()
+    )
+
+    registerCollection("T6",
+      (0 until 50000).map(
+        i => row(i, 1L, if (i % 500 == 0) null else s"Hello$i", "Hello world", 10,
+          new Date(i + 1531820000000L))),
+      new RowTypeInfo(Types.INT, Types.LONG, Types.STRING, Types.STRING, Types.INT, Types.SQL_DATE),
+      "a6, b6, c6, d6, e6, f6",
+      Array(true, true, true, true, true, true),
+      FlinkStatistic.builder().uniqueKeys(Set(Set("a6").asJava).asJava).build()
+    )
+    // HashJoin is disabled due to translateToPlanInternal method is not implemented yet
+    tEnv.getConfig.getConf.setString(TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin")
+  }
+
+  @Test
+  def testSingleAggOnTable_SortAgg(): Unit = {
+    tEnv.getConfig.getConf.setString(TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
+    testSingleAggOnTable()
+    checkResult("SELECT a6, b6, max(c6), count(d6), sum(e6) FROM T6 GROUP BY a6, b6",
+      (0 until 50000).map(i => row(i, 1L, if (i % 500 == 0) null else s"Hello$i", 1L, 10))
+    )
+  }
+
+  @Test
+  def testSingleAggOnTable_HashAgg_WithLocalAgg(): Unit = {
+    tEnv.getConfig.getConf.setString(TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+    tEnv.getConfig.getConf.setString(
+      PlannerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_ENFORCER, "TWO_PHASE")
+    tEnv.getConfig.getConf.setInteger(TableConfigOptions.SQL_RESOURCE_HASH_AGG_TABLE_MEM, 2) // 1M
+    testSingleAggOnTable()
+  }
+
+  @Test
+  def testSingleAggOnTable_HashAgg_WithoutLocalAgg(): Unit = {
+    tEnv.getConfig.getConf.setString(TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+    tEnv.getConfig.getConf.setString(
+      PlannerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_ENFORCER, "ONE_PHASE")
+    tEnv.getConfig.getConf.setInteger(TableConfigOptions.SQL_RESOURCE_HASH_AGG_TABLE_MEM, 2) // 1M
+    testSingleAggOnTable()
+  }
+
+  private def testSingleAggOnTable(): Unit = {
+    // group by fix length
+    checkResult("SELECT a1, b1, count(c1) FROM T1 GROUP BY a1, b1",
+      Seq(row(2, 1, 1), row(3, 2, 1), row(5, 2, 1), row(6, 3, 1)))
+    // group by string
+    checkResult("SELECT a1, c1, count(d1), avg(b1) FROM T1 GROUP BY a1, c1",
+      Seq(row(2, "A", 0, 1.0), row(3, "A", 1, 2.0), row(5, "B", 1, 2.0), row(6, "C", 1, 3.0)))
+    checkResult("SELECT c5, d5, avg(b5), avg(a5) FROM T5 WHERE d5 IS NOT NULL GROUP BY c5, d5",
+      Seq(row("B", "Hi", 2.0, 3.0), row("C", "Hello", null, 1.0),
+        row("D", "Hello world", 3.0, 4.0), row("E", "Hello world, how are you?", 1.0, 3.0),
+        row("I", "hahaha", 2.0, 7.0), row("J", "I am fine.", 1.0, 6.0)))
+    // group by string with null
+    checkResult("SELECT a1, d1, count(d1) FROM T1 GROUP BY a1, d1",
+      Seq(row(2, null, 0), row(3, "Hi", 1), row(5, "Hello", 1), row(6, "Hello world", 1)))
+    checkResult("SELECT c5, d5, avg(b5), avg(a5) FROM T5 GROUP BY c5, d5",
+      Seq(row("A", null, 1.0, 2.0), row("B", "Hi", 2.0, 3.0), row("C", "Hello", null, 1.0),
+        row("D", "Hello world", 3.0, 4.0), row("E", "Hello world, how are you?", 1.0, 3.0),
+        row("F", null, null, 5.0), row("I", "hahaha", 2.0, 7.0), row("J", "I am fine.", 1.0, 6.0)))
+
+    checkResult("SELECT a3, b3, count(c3) FROM T3 GROUP BY a3, b3",
+      Seq(row(1, 10, 1), row(2, 20, 2), row(3, 10, 1), row(4, 20, 1), row(4, null, 1)))
+    checkResult("SELECT a2, b2, count(c2) FROM T2 GROUP BY a2, b2",
+      Seq(row(1, 1, 1), row(1, 2, 1), row(2, 3, 0), row(2, 4, 1)))
+
+    // group by constants
+    checkResult("SELECT a1, b1, count(c1) FROM T1 GROUP BY a1, b1, 1, true",
+      Seq(row(2, 1, 1), row(3, 2, 1), row(5, 2, 1), row(6, 3, 1)))
+    checkResult("SELECT count(c1) FROM T1 GROUP BY 1, true", Seq(row(4)))
+
+    // large data, for hash agg mode it will fallback
+    checkResult("SELECT a6, c6, avg(b6), count(d6), avg(e6) FROM T6 GROUP BY a6, c6",
+      (0 until 50000).map(i => row(i, if (i % 500 == 0) null else s"Hello$i", 1D, 1L, 10D))
+    )
+    checkResult("SELECT a6, d6, avg(b6), count(c6), avg(e6) FROM T6 GROUP BY a6, d6",
+      (0 until 50000).map(i => row(i, "Hello world", 1D, if (i % 500 == 0) 0L else 1L, 10D))
+    )
+    checkResult("SELECT a6, f6, avg(b6), count(c6), avg(e6) FROM T6 GROUP BY a6, f6",
+      (0 until 50000).map(i => row(i, new Date(i + 1531820000000L), 1D,
+        if (i % 500 == 0) 0L else 1L, 10D))
+    )
+  }
+
+  @Test
+  def testMultiAggs(): Unit = {
+    checkResult("SELECT a1, b1, c1, d1, m, COUNT(*) FROM " +
+      "(SELECT a1, b1, c1, COUNT(d1) AS d1, MAX(d1) AS m FROM T1 GROUP BY a1, b1, c1) t " +
+      "GROUP BY a1, b1, c1, d1, m",
+      Seq(row(2, 1, "A", 0, null, 1), row(3, 2, "A", 1, "Hi", 1),
+        row(5, 2, "B", 1, "Hello", 1), row(6, 3, "C", 1, "Hello world", 1)))
+
+    checkResult("SELECT a3, b3, c, s, COUNT(*) FROM " +
+      "(SELECT a3, b3, COUNT(d3) AS c, SUM(d3) AS s, MAX(d3) AS m FROM T3 GROUP BY a3, b3) t " +
+      "GROUP BY a3, b3, c, s",
+      Seq(row(1, 10, 1, 1, 1), row(2, 20, 2, 3, 1), row(3, 10, 1, 1, 1),
+        row(4, 20, 1, 2, 1), row(4, null, 1, 2, 1)))
+  }
+
+  @Test
+  def testAggOnInnerJoin(): Unit = {
+    checkResult("SELECT a1, b1, a2, b2, COUNT(c1) FROM " +
+      "(SELECT * FROM T1, T2 WHERE a1 = b2) t GROUP BY a1, b1, a2, b2",
+      Seq(row(2, 1, 1, 2, 1), row(3, 2, 2, 3, 1)))
+
+    checkResult("SELECT a2, b2, a3, b3, COUNT(c2) FROM " +
+      "(SELECT * FROM T2, T3 WHERE b2 = a3) t GROUP BY a2, b2, a3, b3",
+      Seq(row(1, 1, 1, 10, 1), row(1, 2, 2, 20, 2), row(2, 3, 3, 10, 0),
+        row(2, 4, 4, 20, 1), row(2, 4, 4, null, 1)))
+
+    checkResult("SELECT a1, b1, a2, b2, a3, b3, COUNT(c1) FROM " +
+      "(SELECT * FROM T1, T2, T3 WHERE a1 = b2 AND a1 = a3) t GROUP BY a1, b1, a2, b2, a3, b3",
+      Seq(row(2, 1, 1, 2, 2, 20, 2), row(3, 2, 2, 3, 3, 10, 1)))
+  }
+
+  @Test
+  def testAggOnLeftJoin(): Unit = {
+    checkResult("SELECT a1, b1, a2, b2, COUNT(c1) FROM " +
+      "(SELECT * FROM T1 LEFT JOIN T2 ON a1 = b2) t GROUP BY a1, b1, a2, b2",
+      Seq(row(2, 1, 1, 2, 1), row(3, 2, 2, 3, 1),
+        row(5, 2, null, null, 1), row(6, 3, null, null, 1)))
+
+    checkResult("SELECT a1, b1, a3, b3, COUNT(c1) FROM " +
+      "(SELECT * FROM T1 LEFT JOIN T3 ON a1 = a3) t GROUP BY a1, b1, a3, b3",
+      Seq(row(2, 1, 2, 20, 2), row(3, 2, 3, 10, 1),
+        row(5, 2, null, null, 1), row(6, 3, null, null, 1)))
+
+    checkResult("SELECT a3, b3, a1, b1, COUNT(c1) FROM " +
+      "(SELECT * FROM T3 LEFT JOIN T1 ON a1 = a3) t GROUP BY a3, b3, a1, b1",
+      Seq(row(1, 10, null, null, 0), row(2, 20, 2, 1, 2), row(3, 10, 3, 2, 1),
+        row(4, 20, null, null, 0), row(4, null, null, null, 0)))
+  }
+
+  @Test
+  def testAggOnRightJoin(): Unit = {
+    checkResult("SELECT a1, b1, a2, b2, COUNT(c1) FROM " +
+      "(SELECT * FROM T1 RIGHT JOIN T2 ON a1 = b2) t GROUP BY a1, b1, a2, b2",
+      Seq(row(2, 1, 1, 2, 1), row(3, 2, 2, 3, 1),
+        row(null, null, 1, 1, 0), row(null, null, 2, 4, 0)))
+
+    checkResult("SELECT a1, b1, a3, b3, COUNT(c1) FROM " +
+      "(SELECT * FROM T1 RIGHT JOIN T3 ON a1 = a3) t GROUP BY a1, b1, a3, b3",
+      Seq(row(2, 1, 2, 20, 2), row(3, 2, 3, 10, 1), row(null, null, 1, 10, 0),
+        row(null, null, 4, 20, 0), row(null, null, 4, null, 0)))
+
+    checkResult("SELECT a3, b3, a1, b1, COUNT(c1) FROM " +
+      "(SELECT * FROM T3 RIGHT JOIN T1 ON a1 = a3) t GROUP BY a3, b3, a1, b1",
+      Seq(row(2, 20, 2, 1, 2), row(3, 10, 3, 2, 1),
+        row(null, null, 5, 2, 1), row(null, null, 6, 3, 1)))
+  }
+
+  @Test
+  def testAggOnFullJoin(): Unit = {
+    checkResult("SELECT a1, b1, a2, b2, COUNT(c1) FROM " +
+      "(SELECT * FROM T1 FULL OUTER JOIN T2 ON a1 = b2) t GROUP BY a1, b1, a2, b2",
+      Seq(row(2, 1, 1, 2, 1), row(3, 2, 2, 3, 1), row(5, 2, null, null, 1),
+        row(6, 3, null, null, 1), row(null, null, 1, 1, 0), row(null, null, 2, 4, 0)))
+
+    checkResult("SELECT a1, b1, a3, b3, COUNT(c1) FROM " +
+      "(SELECT * FROM T1 FULL OUTER JOIN T3 ON a1 = a3) t GROUP BY a1, b1, a3, b3",
+      Seq(row(2, 1, 2, 20, 2), row(3, 2, 3, 10, 1), row(5, 2, null, null, 1),
+        row(6, 3, null, null, 1), row(null, null, 1, 10, 0), row(null, null, 4, 20, 0),
+        row(null, null, 4, null, 0)))
+  }
+
+  @Test
+  def testAggOnOver(): Unit = {
+    checkResult("SELECT a1, b1, c, COUNT(d1) FROM " +
+      "(SELECT a1, b1, d1, COUNT(*) OVER (PARTITION BY c1) AS c FROM T1) t GROUP BY a1, b1, c",
+      Seq(row(2, 1, 2, 0), row(3, 2, 2, 1), row(5, 2, 1, 1), row(6, 3, 1, 1)))
+  }
+
+  @Test
+  def testAggOnWindow(): Unit = {
+    checkResult("SELECT a4, b4, COUNT(c4) FROM T4 " +
+      "GROUP BY a4, b4, TUMBLE(d4, INTERVAL '15' MINUTE)",
+      Seq(row(1, 1, 1), row(2, 1, 1), row(3, 2, 1), row(4, 3, 1)))
+
+    checkResult("SELECT a4, c4, COUNT(b4), AVG(b4) FROM T4 " +
+      "GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)",
+      Seq(row(1, "A", 1, 1.0), row(2, "B", 1, 1.0), row(3, "B", 1, 2.0), row(4, "C", 1, 3.0)))
+
+    checkResult("SELECT a4, e4, s, avg(ab), count(cb) FROM " +
+      "(SELECT a4, e4, avg(b4) as ab, count(b4) AS cb, " +
+      "TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, " +
+      "TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 " +
+      "GROUP BY a4, e4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, e4, s",
+      Seq(row(1, "Hi", UTCTimestamp("2018-06-01 10:00:00.0"), 1D, 1),
+        row(2, "Hello", UTCTimestamp("2018-06-01 10:00:00.0"), 1D, 1),
+        row(3, "Hello world", UTCTimestamp("2018-06-01 10:15:00.0"), 2D, 1),
+        row(4, "I am fine.", UTCTimestamp("2018-06-01 10:30:00.0"), 3D, 1)))
+
+    checkResult("SELECT a4, c4, s, COUNT(b4) FROM " +
+      "(SELECT a4, c4, avg(b4) AS b4, " +
+      "TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, " +
+      "TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 " +
+      "GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, c4, s",
+      Seq(row(1, "A", UTCTimestamp("2018-06-01 10:00:00.0"), 1),
+        row(2, "B", UTCTimestamp("2018-06-01 10:00:00.0"), 1),
+        row(3, "B", UTCTimestamp("2018-06-01 10:15:00.0"), 1),
+        row(4, "C", UTCTimestamp("2018-06-01 10:30:00.0"), 1)))
+
+    checkResult("SELECT a4, c4, e, COUNT(b4) FROM " +
+      "(SELECT a4, c4, VAR_POP(b4) AS b4, " +
+      "TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, " +
+      "TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 " +
+      "GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, c4, e",
+      Seq(row(1, "A", UTCTimestamp("2018-06-01 10:15:00.0"), 1),
+        row(2, "B", UTCTimestamp("2018-06-01 10:15:00.0"), 1),
+        row(3, "B", UTCTimestamp("2018-06-01 10:30:00.0"), 1),
+        row(4, "C", UTCTimestamp("2018-06-01 10:45:00.0"), 1)))
+
+    checkResult("SELECT a4, b4, c4, COUNT(*) FROM " +
+      "(SELECT a4, c4, SUM(b4) AS b4, " +
+      "TUMBLE_START(d4, INTERVAL '15' MINUTE) AS s, " +
+      "TUMBLE_END(d4, INTERVAL '15' MINUTE) AS e FROM T4 " +
+      "GROUP BY a4, c4, TUMBLE(d4, INTERVAL '15' MINUTE)) t GROUP BY a4, b4, c4",
+      Seq(row(1, 1, "A", 1), row(2, 1, "B", 1), row(3, 2, "B", 1), row(4, 3, "C", 1)))
+  }
+
+  @Test
+  def testAggWithGroupingSets(): Unit = {
+    checkResult("SELECT a1, b1, c1, COUNT(d1) FROM T1 " +
+      "GROUP BY GROUPING SETS ((a1, b1), (a1, c1))",
+      Seq(row(2, 1, null, 0), row(2, null, "A", 0), row(3, 2, null, 1),
+        row(3, null, "A", 1), row(5, 2, null, 1), row(5, null, "B", 1),
+        row(6, 3, null, 1), row(6, null, "C", 1)))
+
+    checkResult("SELECT a1, c1, COUNT(d1) FROM T1 " +
+      "GROUP BY GROUPING SETS ((a1, c1), (a1), ())",
+      Seq(row(2, "A", 0), row(2, null, 0), row(3, "A", 1), row(3, null, 1), row(5, "B", 1),
+        row(5, null, 1), row(6, "C", 1), row(6, null, 1), row(null, null, 3)))
+
+    checkResult("SELECT a1, b1, c1, COUNT(d1) FROM T1 " +
+      "GROUP BY GROUPING SETS ((a1, b1, c1), (a1, b1, d1))",
+      Seq(row(2, 1, "A", 0), row(2, 1, null, 0), row(3, 2, "A", 1), row(3, 2, null, 1),
+        row(5, 2, "B", 1), row(5, 2, null, 1), row(6, 3, "C", 1), row(6, 3, null, 1)))
+  }
+
+  @Test
+  def testAggWithRollup(): Unit = {
+    checkResult("SELECT a1, b1, c1, COUNT(d1) FROM T1 GROUP BY ROLLUP (a1, b1, c1)",
+      Seq(row(2, 1, "A", 0), row(2, 1, null, 0), row(2, null, null, 0), row(3, 2, "A", 1),
+        row(3, 2, null, 1), row(3, null, null, 1), row(5, 2, "B", 1), row(5, 2, null, 1),
+        row(5, null, null, 1), row(6, 3, "C", 1), row(6, 3, null, 1), row(6, null, null, 1),
+        row(null, null, null, 3)))
+  }
+
+  @Test
+  def testAggWithCube(): Unit = {
+    checkResult("SELECT a1, b1, c1, COUNT(d1) FROM T1 GROUP BY CUBE (a1, b1, c1)",
+      Seq(row(2, 1, "A", 0), row(2, 1, null, 0), row(2, null, "A", 0), row(2, null, null, 0),
+        row(3, 2, "A", 1), row(3, 2, null, 1), row(3, null, "A", 1), row(3, null, null, 1),
+        row(5, 2, "B", 1), row(5, 2, null, 1), row(5, null, "B", 1), row(5, null, null, 1),
+        row(6, 3, "C", 1), row(6, 3, null, 1), row(6, null, "C", 1), row(6, null, null, 1),
+        row(null, 1, "A", 0), row(null, 1, null, 0), row(null, 2, "A", 1), row(null, 2, "B", 1),
+        row(null, 2, null, 2), row(null, 3, "C", 1), row(null, 3, null, 1), row(null, null, "A", 1),
+        row(null, null, "B", 1), row(null, null, "C", 1), row(null, null, null, 3)))
+  }
+
+  @Test
+  def testSingleDistinctAgg(): Unit = {
+    checkResult("SELECT a1, COUNT(DISTINCT c1) FROM T1 GROUP BY a1",
+      Seq(row(2, 1), row(3, 1), row(5, 1), row(6, 1)))
+
+    checkResult("SELECT a1, b1, COUNT(DISTINCT c1) FROM T1 GROUP BY a1, b1",
+      Seq(row(2, 1, 1), row(3, 2, 1), row(5, 2, 1), row(6, 3, 1)))
+  }
+
+  @Test
+  def testSingleDistinctAgg_WithNonDistinctAgg(): Unit = {
+    checkResult("SELECT a1, COUNT(DISTINCT c1), SUM(b1) FROM T1 GROUP BY a1",
+      Seq(row(2, 1, 1), row(3, 1, 2), row(5, 1, 2), row(6, 1, 3)))
+
+    checkResult("SELECT a1, c1, COUNT(DISTINCT c1), SUM(b1) FROM T1 GROUP BY a1, c1",
+      Seq(row(2, "A", 1, 1), row(3, "A", 1, 2), row(5, "B", 1, 2), row(6, "C", 1, 3)))
+
+    checkResult("SELECT a1, COUNT(DISTINCT c1), SUM(b1) FROM T1 GROUP BY a1",
+      Seq(row(2, 1, 1), row(3, 1, 2), row(5, 1, 2), row(6, 1, 3)))
+
+    checkResult("SELECT a1, d1, COUNT(DISTINCT c1), SUM(b1) FROM T1 GROUP BY a1, d1",
+      Seq(row(2, null, 1, 1), row(3, "Hi", 1, 2),
+        row(5, "Hello", 1, 2), row(6, "Hello world", 1, 3)))
+  }
+
+  @Test
+  def testMultiDistinctAggs(): Unit = {
+    checkResult("SELECT a1, COUNT(DISTINCT b1), SUM(DISTINCT b1) FROM T1 GROUP BY a1", Seq(row(2,
+      1, 1), row(3, 1, 2), row(5, 1, 2), row(6, 1, 3)))
+
+    checkResult("SELECT a1, d1, COUNT(DISTINCT c1), SUM(DISTINCT b1) FROM T1 GROUP BY a1, d1",
+      Seq(row(2, null, 1, 1), row(3, "Hi", 1, 2),
+        row(5, "Hello", 1, 2), row(6, "Hello world", 1, 3)))
+
+    checkResult(
+      "SELECT a1, SUM(DISTINCT b1), MAX(DISTINCT b1), MIN(DISTINCT c1) FROM T1 GROUP BY a1",
+      Seq(row(2, 1, 1, "A"), row(3, 2, 2, "A"), row(5, 2, 2, "B"), row(6, 3, 3, "C")))
+
+    checkResult(
+      "SELECT a1, d1, COUNT(DISTINCT c1), MAX(DISTINCT b1), SUM(b1) FROM T1 GROUP BY a1, d1",
+      Seq(row(2, null, 1, 1, 1), row(3, "Hi", 1, 2, 2),
+        row(5, "Hello", 1, 2, 2), row(6, "Hello world", 1, 3, 3)))
+
+    checkResult("SELECT a1, b1, COUNT(DISTINCT c1), COUNT(DISTINCT d1) FROM T1 GROUP BY a1, b1",
+      Seq(row(2, 1, 1, 0), row(3, 2, 1, 1), row(5, 2, 1, 1), row(6, 3, 1, 1)))
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateRemoveITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateRemoveITCase.scala
new file mode 100644
index 0000000..929f49b1
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateRemoveITCase.scala
@@ -0,0 +1,214 @@
+/*
+ * 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.flink.table.runtime.batch.sql.agg
+
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.plan.stats.FlinkStatistic
+import org.apache.flink.table.runtime.utils.BatchTestBase
+import org.apache.flink.table.runtime.utils.BatchTestBase.row
+import org.apache.flink.table.runtime.utils.TestData._
+
+import org.junit.{Before, Test}
+
+import scala.collection.JavaConverters._
+import scala.collection.Seq
+
+class AggregateRemoveITCase extends BatchTestBase {
+
+  @Before
+  def before(): Unit = {
+    registerCollection("T1",
+      Seq(row(2, 1, "A", null),
+        row(3, 2, "A", "Hi"),
+        row(5, 2, "B", "Hello"),
+        row(6, 3, "C", "Hello world")),
+      new RowTypeInfo(Types.INT, Types.INT, Types.STRING, Types.STRING),
+      "a, b, c, d",
+      Array(true, true, true),
+      FlinkStatistic.builder().uniqueKeys(Set(Set("a").asJava).asJava).build()
+    )
+
+    registerCollection("T2", smallData3, type3, "a, b, c", nullablesOfSmallData3,
+      FlinkStatistic.builder().uniqueKeys(Set(Set("a").asJava).asJava).build())
+    registerCollection("T3", smallData5, type5, "a, b, c, d, e", nullablesOfSmallData5,
+      FlinkStatistic.builder().uniqueKeys(Set(Set("b").asJava).asJava).build())
+  }
+
+  @Test
+  def testSimple(): Unit = {
+    checkResult("SELECT a, b FROM T3 GROUP BY a, b",
+      Seq(row(1, 1), row(2, 2), row(2, 3)))
+
+    checkResult("SELECT a, b + 1, c, s FROM (" +
+      "SELECT a, MIN(b) AS b, SUM(b) AS s, MAX(c) AS c FROM T3 GROUP BY a)",
+      Seq(row(1, 2, 0, 1), row(2, 3, 2, 5)))
+
+    checkResult("SELECT a, SUM(b) AS s FROM T3 GROUP BY a",
+      Seq(row(1, 1), row(2, 5)))
+
+    checkResult("SELECT MAX(a), SUM(b), MIN(c) FROM (VALUES (1, 2, 3)) T(a, b, c)",
+      Seq(row(1, 2, 3)))
+
+    checkResult(
+      "SELECT a, b + 1, c, s FROM (" +
+        "SELECT a, MIN(b) AS b, SUM(b) AS s, MAX(c) AS c FROM T2 GROUP BY a)",
+      Seq(
+        row(1, 2L, "Hi", 1L),
+        row(2, 3L, "Hello", 2L),
+        row(3, 3L, "Hello world", 2L)
+      ))
+
+    checkResult(
+      "SELECT MAX(a), SUM(b), MIN(c) FROM (VALUES (1, 2, 3)) T(a, b, c)",
+      Seq(row(1, 2, 3))
+    )
+
+    // TODO enable this case after translateToPlanInternal method is implemented
+    //  in BatchExecNestedLoopJoin
+    // checkResult(
+    //   "SELECT * FROM T2 WHERE EXISTS (SELECT SUM(a) FROM T3 WHERE 1=2)",
+    //   Seq(row(1, 1, "Hi"), row(2, 2, "Hello"), row(3, 2, "Hello world"))
+    // )
+
+    checkResult(
+      """
+        |SELECT a, SUM(b), MAX(b) FROM
+        | (SELECT a, MAX(b) AS b FROM
+        | (VALUES (cast(null as BIGINT), cast(null as BIGINT))) T(a, b) GROUP BY a) t
+        | GROUP BY a
+      """.stripMargin,
+      Seq(row(null, null, null))
+    )
+  }
+
+  @Test
+  def testWithGroupingSets(): Unit = {
+    checkResult("SELECT a, b, c, COUNT(d) FROM T1 " +
+      "GROUP BY GROUPING SETS ((a, b), (a, c))",
+      Seq(row(2, 1, null, 0), row(2, null, "A", 0), row(3, 2, null, 1),
+        row(3, null, "A", 1), row(5, 2, null, 1), row(5, null, "B", 1),
+        row(6, 3, null, 1), row(6, null, "C", 1)))
+
+    checkResult("SELECT a, c, COUNT(d) FROM T1 " +
+      "GROUP BY GROUPING SETS ((a, c), (a), ())",
+      Seq(row(2, "A", 0), row(2, null, 0), row(3, "A", 1), row(3, null, 1), row(5, "B", 1),
+        row(5, null, 1), row(6, "C", 1), row(6, null, 1), row(null, null, 3)))
+
+    checkResult("SELECT a, b, c, COUNT(d) FROM T1 " +
+      "GROUP BY GROUPING SETS ((a, b, c), (a, b, d))",
+      Seq(row(2, 1, "A", 0), row(2, 1, null, 0), row(3, 2, "A", 1), row(3, 2, null, 1),
+        row(5, 2, "B", 1), row(5, 2, null, 1), row(6, 3, "C", 1), row(6, 3, null, 1)))
+  }
+
+
+  @Test
+  def testWithRollup(): Unit = {
+    checkResult("SELECT a, b, c, COUNT(d) FROM T1 GROUP BY ROLLUP (a, b, c)",
+      Seq(row(2, 1, "A", 0), row(2, 1, null, 0), row(2, null, null, 0), row(3, 2, "A", 1),
+        row(3, 2, null, 1), row(3, null, null, 1), row(5, 2, "B", 1), row(5, 2, null, 1),
+        row(5, null, null, 1), row(6, 3, "C", 1), row(6, 3, null, 1), row(6, null, null, 1),
+        row(null, null, null, 3)))
+  }
+
+  @Test
+  def testWithCube(): Unit = {
+    checkResult("SELECT a, b, c, COUNT(d) FROM T1 GROUP BY CUBE (a, b, c)",
+      Seq(row(2, 1, "A", 0), row(2, 1, null, 0), row(2, null, "A", 0), row(2, null, null, 0),
+        row(3, 2, "A", 1), row(3, 2, null, 1), row(3, null, "A", 1), row(3, null, null, 1),
+        row(5, 2, "B", 1), row(5, 2, null, 1), row(5, null, "B", 1), row(5, null, null, 1),
+        row(6, 3, "C", 1), row(6, 3, null, 1), row(6, null, "C", 1), row(6, null, null, 1),
+        row(null, 1, "A", 0), row(null, 1, null, 0), row(null, 2, "A", 1), row(null, 2, "B", 1),
+        row(null, 2, null, 2), row(null, 3, "C", 1), row(null, 3, null, 1), row(null, null, "A", 1),
+        row(null, null, "B", 1), row(null, null, "C", 1), row(null, null, null, 3)))
+
+    checkResult(
+      "SELECT b, c, e, SUM(a), MAX(d) FROM T3 GROUP BY CUBE (b, c, e)",
+      Seq(
+        row(null, null, null, 5, "Hallo Welt wie"),
+        row(null, null, 1, 3, "Hallo Welt wie"),
+        row(null, null, 2, 2, "Hallo Welt"),
+        row(null, 0, null, 1, "Hallo"),
+        row(null, 0, 1, 1, "Hallo"),
+        row(null, 1, null, 2, "Hallo Welt"),
+        row(null, 1, 2, 2, "Hallo Welt"),
+        row(null, 2, null, 2, "Hallo Welt wie"),
+        row(null, 2, 1, 2, "Hallo Welt wie"),
+        row(1, null, null, 1, "Hallo"),
+        row(1, null, 1, 1, "Hallo"),
+        row(1, 0, null, 1, "Hallo"),
+        row(1, 0, 1, 1, "Hallo"),
+        row(2, null, null, 2, "Hallo Welt"),
+        row(2, null, 2, 2, "Hallo Welt"),
+        row(2, 1, null, 2, "Hallo Welt"),
+        row(2, 1, 2, 2, "Hallo Welt"),
+        row(3, null, null, 2, "Hallo Welt wie"),
+        row(3, null, 1, 2, "Hallo Welt wie"),
+        row(3, 2, null, 2, "Hallo Welt wie"),
+        row(3, 2, 1, 2, "Hallo Welt wie")
+      ))
+  }
+
+  @Test
+  def testSingleDistinctAgg(): Unit = {
+    checkResult("SELECT a, COUNT(DISTINCT c) FROM T1 GROUP BY a",
+      Seq(row(2, 1), row(3, 1), row(5, 1), row(6, 1)))
+
+    checkResult("SELECT a, b, COUNT(DISTINCT c) FROM T1 GROUP BY a, b",
+      Seq(row(2, 1, 1), row(3, 2, 1), row(5, 2, 1), row(6, 3, 1)))
+
+    checkResult("SELECT a, b, COUNT(DISTINCT c), COUNT(DISTINCT d) FROM T1 GROUP BY a, b",
+      Seq(row(2, 1, 1, 0), row(3, 2, 1, 1), row(5, 2, 1, 1), row(6, 3, 1, 1)))
+  }
+
+  @Test
+  def testSingleDistinctAgg_WithNonDistinctAgg(): Unit = {
+    checkResult("SELECT a, COUNT(DISTINCT c), SUM(b) FROM T1 GROUP BY a",
+      Seq(row(2, 1, 1), row(3, 1, 2), row(5, 1, 2), row(6, 1, 3)))
+
+    checkResult("SELECT a, c, COUNT(DISTINCT c), SUM(b) FROM T1 GROUP BY a, c",
+      Seq(row(2, "A", 1, 1), row(3, "A", 1, 2), row(5, "B", 1, 2), row(6, "C", 1, 3)))
+
+    checkResult("SELECT a, COUNT(DISTINCT c), SUM(b) FROM T1 GROUP BY a",
+      Seq(row(2, 1, 1), row(3, 1, 2), row(5, 1, 2), row(6, 1, 3)))
+
+    checkResult("SELECT a, d, COUNT(DISTINCT c), SUM(b) FROM T1 GROUP BY a, d",
+      Seq(row(2, null, 1, 1), row(3, "Hi", 1, 2),
+        row(5, "Hello", 1, 2), row(6, "Hello world", 1, 3)))
+  }
+
+  @Test
+  def testMultiDistinctAggs(): Unit = {
+    checkResult("SELECT a, COUNT(DISTINCT b), SUM(DISTINCT b) FROM T1 GROUP BY a",
+      Seq(row(2, 1, 1), row(3, 1, 2), row(5, 1, 2), row(6, 1, 3)))
+
+    checkResult("SELECT a, d, COUNT(DISTINCT c), SUM(DISTINCT b) FROM T1 GROUP BY a, d",
+      Seq(row(2, null, 1, 1), row(3, "Hi", 1, 2),
+        row(5, "Hello", 1, 2), row(6, "Hello world", 1, 3)))
+
+    checkResult(
+      "SELECT a, SUM(DISTINCT b), MAX(DISTINCT b), MIN(DISTINCT c) FROM T1 GROUP BY a",
+      Seq(row(2, 1, 1, "A"), row(3, 2, 2, "A"), row(5, 2, 2, "B"), row(6, 3, 3, "C")))
+
+    checkResult(
+      "SELECT a, d, COUNT(DISTINCT c), MAX(DISTINCT b), SUM(b) FROM T1 GROUP BY a, d",
+      Seq(row(2, null, 1, 1, 1), row(3, "Hi", 1, 2, 2),
+        row(5, "Hello", 1, 2, 2), row(6, "Hello world", 1, 3, 3)))
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/PruneAggregateCallITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/PruneAggregateCallITCase.scala
new file mode 100644
index 0000000..9efecbc
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/PruneAggregateCallITCase.scala
@@ -0,0 +1,132 @@
+/*
+ * 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.flink.table.runtime.batch.sql.agg
+
+import org.apache.flink.table.plan.stats.FlinkStatistic
+import org.apache.flink.table.runtime.utils.BatchTestBase
+import org.apache.flink.table.runtime.utils.BatchTestBase.row
+import org.apache.flink.table.runtime.utils.TestData._
+
+import org.junit.{Before, Test}
+
+import scala.collection.JavaConverters._
+import scala.collection.Seq
+
+class PruneAggregateCallITCase extends BatchTestBase {
+
+  @Before
+  def before(): Unit = {
+    registerCollection("MyTable", smallData3, type3, "a, b, c", nullablesOfSmallData3)
+    registerCollection("MyTable2", smallData5, type5, "a, b, c, d, e", nullablesOfSmallData5,
+      FlinkStatistic.builder().uniqueKeys(Set(Set("b").asJava).asJava).build())
+  }
+
+  @Test
+  def testNoneEmptyGroupKey(): Unit = {
+    checkResult(
+      "SELECT a FROM (SELECT b, MAX(a) AS a, COUNT(*), MAX(c) FROM MyTable GROUP BY b) t",
+      Seq(row(1), row(3))
+    )
+    checkResult(
+      """
+        |SELECT c, a FROM
+        | (SELECT a, c, COUNT(b) as c, SUM(b) as s FROM MyTable GROUP BY a, c) t
+        |WHERE s > 1
+      """.stripMargin,
+      Seq(row("Hello world", 3), row("Hello", 2))
+    )
+    checkResult(
+      "SELECT a, c FROM (SELECT a, b, SUM(c) as c, COUNT(d) as d FROM MyTable2 GROUP BY a, b) t",
+      Seq(row(1, 0), row(2, 1), row(2, 2)))
+
+    checkResult(
+      "SELECT a FROM (SELECT a, b, SUM(c) as c, COUNT(d) as d FROM MyTable2 GROUP BY a, b) t",
+      Seq(row(1), row(2), row(2)))
+  }
+
+  @Test
+  def testEmptyGroupKey(): Unit = {
+    checkResult(
+      "SELECT 1 FROM (SELECT SUM(a) FROM MyTable) t",
+      Seq(row(1))
+    )
+
+    // TODO enable this case after translateToPlanInternal method is implemented
+    //  in BatchExecNestedLoopJoin
+    // checkResult(
+    //   "SELECT * FROM MyTable WHERE EXISTS (SELECT COUNT(*) FROM MyTable2)",
+    //   Seq(row(1, 1, "Hi"), row(2, 2, "Hello"), row(3, 2, "Hello world"))
+    // )
+
+    // TODO enable this case after translateToPlanInternal method is implemented
+    //  in BatchExecNestedLoopJoin
+    // checkResult(
+    //   "SELECT * FROM MyTable WHERE EXISTS (SELECT COUNT(*) FROM MyTable2 WHERE 1=2)",
+    //   Seq(row(1, 1, "Hi"), row(2, 2, "Hello"), row(3, 2, "Hello world"))
+    // )
+
+    checkResult(
+      "SELECT 1 FROM (SELECT SUM(a), COUNT(*) FROM MyTable) t",
+      Seq(row(1))
+    )
+
+    checkResult(
+      "SELECT 1 FROM (SELECT SUM(a), COUNT(*) FROM MyTable WHERE 1=2) t",
+      Seq(row(1))
+    )
+
+    checkResult(
+      "SELECT 1 FROM (SELECT COUNT(*), SUM(a) FROM MyTable) t",
+      Seq(row(1))
+    )
+
+    checkResult(
+      "SELECT 1 FROM (SELECT COUNT(*), SUM(a) FROM MyTable WHERE 1=2) t",
+      Seq(row(1))
+    )
+
+    // TODO enable this case after translateToPlanInternal method is implemented
+    //  in BatchExecNestedLoopJoin
+    // checkResult(
+    //   "SELECT * FROM MyTable WHERE EXISTS (SELECT SUM(a), COUNT(*) FROM MyTable2)",
+    //   Seq(row(1, 1, "Hi"), row(2, 2, "Hello"), row(3, 2, "Hello world"))
+    // )
+
+    // TODO enable this case after translateToPlanInternal method is implemented
+    //  in BatchExecNestedLoopJoin
+    // checkResult(
+    //   "SELECT * FROM MyTable WHERE EXISTS (SELECT COUNT(*), SUM(a) FROM MyTable2)",
+    //   Seq(row(1, 1, "Hi"), row(2, 2, "Hello"), row(3, 2, "Hello world"))
+    // )
+
+    // TODO enable this case after translateToPlanInternal method is implemented
+    //  in BatchExecNestedLoopJoin
+    // checkResult(
+    //   "SELECT * FROM MyTable WHERE EXISTS (SELECT SUM(a), COUNT(*) FROM MyTable2 WHERE 1=2)",
+    //   Seq(row(1, 1, "Hi"), row(2, 2, "Hello"), row(3, 2, "Hello world"))
+    // )
+
+    // TODO enable this case after translateToPlanInternal method is implemented
+    //  in BatchExecNestedLoopJoin
+    // checkResult(
+    //   "SELECT * FROM MyTable WHERE EXISTS (SELECT COUNT(*), SUM(a) FROM MyTable2 WHERE 1=2)",
+    //   Seq(row(1, 1, "Hi"), row(2, 2, "Hello"), row(3, 2, "Hello world"))
+    // )
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
index d8f22fa..cfc1828 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
@@ -78,10 +78,10 @@ class InnerJoinITCase extends BatchTestBase {
   @Before
   def before(): Unit = {
     tEnv.getConfig.getConf.setInteger(TableConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 3)
-    registerCollection("myUpperCaseData", myUpperCaseData, INT_STRING, Array(true, false), "N, L")
-    registerCollection("myLowerCaseData", myLowerCaseData, INT_STRING, Array(true, false), "n, l")
-    registerCollection("myTestData1", myTestData1, INT_INT, Array(false, false), "a, b")
-    registerCollection("myTestData2", myTestData2, INT_INT, Array(false, false), "a, b")
+    registerCollection("myUpperCaseData", myUpperCaseData, INT_STRING, "N, L", Array(true, false))
+    registerCollection("myLowerCaseData", myLowerCaseData, INT_STRING, "n, l", Array(true, false))
+    registerCollection("myTestData1", myTestData1, INT_INT, "a, b", Array(false, false))
+    registerCollection("myTestData2", myTestData2, INT_INT, "a, b", Array(false, false))
     disableOtherJoinOpForJoin(tEnv, expectedJoinType)
   }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinConditionTypeCoerceRuleITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinConditionTypeCoerceITCase.scala
similarity index 97%
rename from flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinConditionTypeCoerceRuleITCase.scala
rename to flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinConditionTypeCoerceITCase.scala
index 15a2ae2..ecb0b51 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinConditionTypeCoerceRuleITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinConditionTypeCoerceITCase.scala
@@ -27,7 +27,7 @@ import org.junit.{Before, Ignore, Test}
 
 // @RunWith(classOf[Parameterized]) TODO
 @Ignore // TODO support JoinConditionTypeCoerce
-class JoinConditionTypeCoerceRuleITCase extends BatchTestBase {
+class JoinConditionTypeCoerceITCase extends BatchTestBase {
   @Before
   def before(): Unit = {
     tEnv.getConfig.getConf.setInteger(TableConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 3)
@@ -35,14 +35,14 @@ class JoinConditionTypeCoerceRuleITCase extends BatchTestBase {
       "t1",
       numericData,
       numericType,
-      nullablesOfNumericData,
-      "a, b, c, d, e")
+      "a, b, c, d, e",
+      nullablesOfNumericData)
     registerCollection(
       "t2",
       numericData,
       numericType,
-      nullablesOfNumericData,
-      "a, b, c, d, e")
+      "a, b, c, d, e",
+      nullablesOfNumericData)
     // Disable NestedLoopJoin.
     JoinITCaseHelper.disableOtherJoinOpForJoin(tEnv, JoinType.SortMergeJoin)
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
index 908c0fd..d9ac9d0 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
@@ -44,14 +44,14 @@ class JoinITCase() extends BatchTestBase {
 
   @Before
   def before(): Unit = {
-    registerCollection("SmallTable3", smallData3, type3, nullablesOfSmallData3, "a, b, c")
-    registerCollection("Table3", data3, type3, nullablesOfData3, "a, b, c")
-    registerCollection("Table5", data5, type5, nullablesOfData5, "d, e, f, g, h")
-    registerCollection("NullTable3", nullData3, type3, nullablesOfNullData3, "a, b, c")
-    registerCollection("NullTable5", nullData5, type5, nullablesOfNullData5, "d, e, f, g, h")
+    registerCollection("SmallTable3", smallData3, type3, "a, b, c", nullablesOfSmallData3)
+    registerCollection("Table3", data3, type3, "a, b, c", nullablesOfData3)
+    registerCollection("Table5", data5, type5, "d, e, f, g, h", nullablesOfData5)
+    registerCollection("NullTable3", nullData3, type3, "a, b, c", nullablesOfNullData3)
+    registerCollection("NullTable5", nullData5, type5, "d, e, f, g, h", nullablesOfNullData5)
     registerCollection("l", data2_1, INT_DOUBLE, "a, b")
     registerCollection("r", data2_2, INT_DOUBLE, "c, d")
-    registerCollection("t", data2_3, INT_DOUBLE, nullablesOfData2_3, "c, d")
+    registerCollection("t", data2_3, INT_DOUBLE, "c, d", nullablesOfData2_3)
     JoinITCaseHelper.disableOtherJoinOpForJoin(tEnv, expectedJoinType)
   }
 
@@ -96,11 +96,11 @@ class JoinITCase() extends BatchTestBase {
       registerCollection("PojoSmallTable3", smallData3,
         new RowTypeInfo(INT_TYPE_INFO, LONG_TYPE_INFO,
           new GenericTypeInfoWithoutComparator[String](classOf[String])),
-        nullablesOfSmallData3, "a, b, c")
+        "a, b, c", nullablesOfSmallData3)
       registerCollection("PojoTable5", data5,
         new RowTypeInfo(INT_TYPE_INFO, LONG_TYPE_INFO, INT_TYPE_INFO,
           new GenericTypeInfoWithoutComparator[String](classOf[String]), LONG_TYPE_INFO),
-        nullablesOfData5, "d, e, f, g, h")
+        "d, e, f, g, h", nullablesOfData5)
 
       checkResult(
         "SELECT c, g FROM (SELECT h, g, f, e, d FROM PojoSmallTable3, PojoTable5 WHERE b = e)," +
@@ -667,7 +667,7 @@ class JoinITCase() extends BatchTestBase {
       ))
 
     registerCollection(
-      "NullT", Seq(row(null, null, "c")), type3, allNullablesOfNullData3, "a, b, c")
+      "NullT", Seq(row(null, null, "c")), type3, "a, b, c", allNullablesOfNullData3)
     checkResult(
       "SELECT T1.a, T1.b, T1.c FROM NullT T1, NullT T2 WHERE " +
         "(T1.a = T2.a OR (T1.a IS NULL AND T2.a IS NULL)) " +
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/OuterJoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/OuterJoinITCase.scala
index a16b88e..6c0debf 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/OuterJoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/OuterJoinITCase.scala
@@ -60,9 +60,9 @@ class OuterJoinITCase extends BatchTestBase {
   @Before
   def before(): Unit = {
     tEnv.getConfig.getConf.setInteger(TableConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 3)
-    registerCollection("uppercasedata", upperCaseData, INT_STRING, nullablesOfUpperCaseData, "N, L")
-    registerCollection("lowercasedata", lowerCaseData, INT_STRING, nullablesOfLowerCaseData, "n, l")
-    registerCollection("allnulls", allNulls, INT_ONLY, nullablesOfAllNulls, "a")
+    registerCollection("uppercasedata", upperCaseData, INT_STRING, "N, L", nullablesOfUpperCaseData)
+    registerCollection("lowercasedata", lowerCaseData, INT_STRING, "n, l", nullablesOfLowerCaseData)
+    registerCollection("allnulls", allNulls, INT_ONLY, "a", nullablesOfAllNulls)
     registerCollection("leftT", leftT, INT_DOUBLE, "a, b")
     registerCollection("rightT", rightT, INT_DOUBLE, "c, d")
     JoinITCaseHelper.disableOtherJoinOpForJoin(tEnv, expectedJoinType)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateITCase.scala
index 160680b..0e20799 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateITCase.scala
@@ -31,10 +31,11 @@ import org.apache.flink.table.runtime.utils.StreamingWithAggTestBase.AggMode
 import org.apache.flink.table.runtime.utils.StreamingWithMiniBatchTestBase.MiniBatchMode
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.StateBackendMode
 import org.apache.flink.table.runtime.utils.UserDefinedFunctionTestUtils._
-import org.apache.flink.table.runtime.utils.{StreamTestData, StreamingWithAggTestBase, TestingRetractSink}
+import org.apache.flink.table.runtime.utils.{StreamingWithAggTestBase, TestData, TestingRetractSink}
 import org.apache.flink.table.typeutils.BigDecimalTypeInfo
 import org.apache.flink.table.util.DateTimeTestUtil._
 import org.apache.flink.types.Row
+
 import org.junit.Assert.assertEquals
 import org.junit._
 import org.junit.runner.RunWith
@@ -159,7 +160,7 @@ class AggregateITCase(
         "FROM MyTable " +
         "GROUP BY b"
 
-    val t = failingDataSource(StreamTestData.get3TupleData).toTable(tEnv, 'a, 'b, 'c)
+    val t = failingDataSource(TestData.tupleData3).toTable(tEnv, 'a, 'b, 'c)
     tEnv.registerTable("MyTable", t)
 
     val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
@@ -543,7 +544,7 @@ class AggregateITCase(
   /** test unbounded groupBy (without window) **/
   @Test
   def testUnboundedGroupBy(): Unit = {
-    val t = failingDataSource(StreamTestData.get3TupleData).toTable(tEnv, 'a, 'b, 'c)
+    val t = failingDataSource(TestData.tupleData3).toTable(tEnv, 'a, 'b, 'c)
     tEnv.registerTable("MyTable", t)
 
     val sqlQuery = "SELECT b, COUNT(a) FROM MyTable GROUP BY b"
@@ -609,7 +610,7 @@ class AggregateITCase(
   def testUnboundedGroupByCollect(): Unit = {
     val sqlQuery = "SELECT b, COLLECT(a) FROM MyTable GROUP BY b"
 
-    val t = failingDataSource(StreamTestData.get3TupleData).toTable(tEnv, 'a, 'b, 'c)
+    val t = failingDataSource(TestData.tupleData3).toTable(tEnv, 'a, 'b, 'c)
     tEnv.registerTable("MyTable", t)
 
     val sink = new TestingRetractSink
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateRemoveITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateRemoveITCase.scala
new file mode 100644
index 0000000..5c7b59c
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateRemoveITCase.scala
@@ -0,0 +1,254 @@
+/*
+ * 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.flink.table.runtime.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.plan.stats.FlinkStatistic
+import org.apache.flink.table.runtime.utils.BatchTestBase.row
+import org.apache.flink.table.runtime.utils.StreamingWithAggTestBase.AggMode
+import org.apache.flink.table.runtime.utils.StreamingWithMiniBatchTestBase.MiniBatchMode
+import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.StateBackendMode
+import org.apache.flink.table.runtime.utils.{StreamTableEnvUtil, StreamingWithAggTestBase, TestData, TestingRetractSink}
+import org.apache.flink.types.Row
+
+import org.junit.Assert.assertEquals
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+import scala.collection.{Seq, mutable}
+
+@RunWith(classOf[Parameterized])
+class AggregateRemoveITCase(
+    aggMode: AggMode,
+    minibatch: MiniBatchMode,
+    backend: StateBackendMode)
+  extends StreamingWithAggTestBase(aggMode, minibatch, backend) {
+
+  @Test
+  def testSimple(): Unit = {
+    checkResult("SELECT a, b FROM T GROUP BY a, b",
+      Seq(row(2, 1), row(3, 2), row(5, 2), row(6, 3)))
+
+    checkResult("SELECT a, b + 1, c, s FROM (" +
+      "SELECT a, MIN(b) AS b, SUM(b) AS s, MAX(c) AS c FROM MyTable2 GROUP BY a)",
+      Seq(row(1, 2, 0, 1), row(2, 3, 2, 5)))
+
+    checkResult("SELECT a, SUM(b) AS s FROM MyTable2 GROUP BY a",
+      Seq(row(1, 1), row(2, 5)))
+
+    checkResult(
+      "SELECT a, b + 1, c, s FROM (" +
+        "SELECT a, MIN(b) AS b, SUM(b) AS s, MAX(c) AS c FROM MyTable GROUP BY a)",
+      Seq(
+        row(1, 2L, "Hi", 1L),
+        row(2, 3L, "Hello", 2L),
+        row(3, 3L, "Hello world", 2L)
+      ))
+  }
+
+  @Test
+  def testWithGroupingSets(): Unit = {
+    checkResult("SELECT a, b, c, COUNT(d) FROM T " +
+      "GROUP BY GROUPING SETS ((a, b), (a, c))",
+      Seq(row(2, 1, null, 0), row(2, null, "A", 0), row(3, 2, null, 1),
+        row(3, null, "A", 1), row(5, 2, null, 1), row(5, null, "B", 1),
+        row(6, 3, null, 1), row(6, null, "C", 1)))
+
+    checkResult("SELECT a, c, COUNT(d) FROM T " +
+      "GROUP BY GROUPING SETS ((a, c), (a), ())",
+      Seq(row(2, "A", 0), row(2, null, 0), row(3, "A", 1), row(3, null, 1), row(5, "B", 1),
+        row(5, null, 1), row(6, "C", 1), row(6, null, 1), row(null, null, 3)))
+
+    checkResult("SELECT a, b, c, COUNT(d) FROM T " +
+      "GROUP BY GROUPING SETS ((a, b, c), (a, b, d))",
+      Seq(row(2, 1, "A", 0), row(2, 1, null, 0), row(3, 2, "A", 1), row(3, 2, null, 1),
+        row(5, 2, "B", 1), row(5, 2, null, 1), row(6, 3, "C", 1), row(6, 3, null, 1)))
+  }
+
+  @Test
+  def testWithRollup(): Unit = {
+    checkResult("SELECT a, b, c, COUNT(d) FROM T GROUP BY ROLLUP (a, b, c)",
+      Seq(row(2, 1, "A", 0), row(2, 1, null, 0), row(2, null, null, 0), row(3, 2, "A", 1),
+        row(3, 2, null, 1), row(3, null, null, 1), row(5, 2, "B", 1), row(5, 2, null, 1),
+        row(5, null, null, 1), row(6, 3, "C", 1), row(6, 3, null, 1), row(6, null, null, 1),
+        row(null, null, null, 3)))
+  }
+
+  @Test
+  def testWithCube(): Unit = {
+    checkResult("SELECT a, b, c, COUNT(d) FROM T GROUP BY CUBE (a, b, c)",
+      Seq(row(2, 1, "A", 0), row(2, 1, null, 0), row(2, null, "A", 0), row(2, null, null, 0),
+        row(3, 2, "A", 1), row(3, 2, null, 1), row(3, null, "A", 1), row(3, null, null, 1),
+        row(5, 2, "B", 1), row(5, 2, null, 1), row(5, null, "B", 1), row(5, null, null, 1),
+        row(6, 3, "C", 1), row(6, 3, null, 1), row(6, null, "C", 1), row(6, null, null, 1),
+        row(null, 1, "A", 0), row(null, 1, null, 0), row(null, 2, "A", 1), row(null, 2, "B", 1),
+        row(null, 2, null, 2), row(null, 3, "C", 1), row(null, 3, null, 1), row(null, null, "A", 1),
+        row(null, null, "B", 1), row(null, null, "C", 1), row(null, null, null, 3)))
+
+    checkResult(
+      "SELECT b, c, e, SUM(a), MAX(d) FROM MyTable2 GROUP BY CUBE (b, c, e)",
+      Seq(
+        row(null, null, null, 5, "Hallo Welt wie"),
+        row(null, null, 1, 3, "Hallo Welt wie"),
+        row(null, null, 2, 2, "Hallo Welt"),
+        row(null, 0, null, 1, "Hallo"),
+        row(null, 0, 1, 1, "Hallo"),
+        row(null, 1, null, 2, "Hallo Welt"),
+        row(null, 1, 2, 2, "Hallo Welt"),
+        row(null, 2, null, 2, "Hallo Welt wie"),
+        row(null, 2, 1, 2, "Hallo Welt wie"),
+        row(1, null, null, 1, "Hallo"),
+        row(1, null, 1, 1, "Hallo"),
+        row(1, 0, null, 1, "Hallo"),
+        row(1, 0, 1, 1, "Hallo"),
+        row(2, null, null, 2, "Hallo Welt"),
+        row(2, null, 2, 2, "Hallo Welt"),
+        row(2, 1, null, 2, "Hallo Welt"),
+        row(2, 1, 2, 2, "Hallo Welt"),
+        row(3, null, null, 2, "Hallo Welt wie"),
+        row(3, null, 1, 2, "Hallo Welt wie"),
+        row(3, 2, null, 2, "Hallo Welt wie"),
+        row(3, 2, 1, 2, "Hallo Welt wie")
+      ))
+  }
+
+  @Test
+  def testSingleDistinctAgg(): Unit = {
+    checkResult("SELECT a, COUNT(DISTINCT c) FROM T GROUP BY a",
+      Seq(row(2, 1), row(3, 1), row(5, 1), row(6, 1)))
+
+    checkResult("SELECT a, b, COUNT(DISTINCT c) FROM T GROUP BY a, b",
+      Seq(row(2, 1, 1), row(3, 2, 1), row(5, 2, 1), row(6, 3, 1)))
+
+    checkResult("SELECT a, b, COUNT(DISTINCT c), COUNT(DISTINCT d) FROM T GROUP BY a, b",
+      Seq(row(2, 1, 1, 0), row(3, 2, 1, 1), row(5, 2, 1, 1), row(6, 3, 1, 1)))
+  }
+
+  @Test
+  def testSingleDistinctAgg_WithNonDistinctAgg(): Unit = {
+    checkResult("SELECT a, COUNT(DISTINCT c), SUM(b) FROM T GROUP BY a",
+      Seq(row(2, 1, 1), row(3, 1, 2), row(5, 1, 2), row(6, 1, 3)))
+
+    checkResult("SELECT a, c, COUNT(DISTINCT c), SUM(b) FROM T GROUP BY a, c",
+      Seq(row(2, "A", 1, 1), row(3, "A", 1, 2), row(5, "B", 1, 2), row(6, "C", 1, 3)))
+
+    checkResult("SELECT a, COUNT(DISTINCT c), SUM(b) FROM T GROUP BY a",
+      Seq(row(2, 1, 1), row(3, 1, 2), row(5, 1, 2), row(6, 1, 3)))
+
+    checkResult("SELECT a, d, COUNT(DISTINCT c), SUM(b) FROM T GROUP BY a, d",
+      Seq(row(2, null, 1, 1), row(3, "Hi", 1, 2),
+        row(5, "Hello", 1, 2), row(6, "Hello world", 1, 3)))
+  }
+
+  @Test
+  def testMultiDistinctAggs(): Unit = {
+    checkResult("SELECT a, COUNT(DISTINCT b), SUM(DISTINCT b) FROM T GROUP BY a", Seq(row(2,
+      1, 1), row(3, 1, 2), row(5, 1, 2), row(6, 1, 3)))
+
+    checkResult("SELECT a, d, COUNT(DISTINCT c), SUM(DISTINCT b) FROM T GROUP BY a, d",
+      Seq(row(2, null, 1, 1), row(3, "Hi", 1, 2),
+        row(5, "Hello", 1, 2), row(6, "Hello world", 1, 3)))
+
+    checkResult(
+      "SELECT a, SUM(DISTINCT b), MAX(DISTINCT b), MIN(DISTINCT c) FROM T GROUP BY a",
+      Seq(row(2, 1, 1, "A"), row(3, 2, 2, "A"), row(5, 2, 2, "B"), row(6, 3, 3, "C")))
+
+    checkResult(
+      "SELECT a, d, COUNT(DISTINCT c), MAX(DISTINCT b), SUM(b) FROM T GROUP BY a, d",
+      Seq(row(2, null, 1, 1, 1), row(3, "Hi", 1, 2, 2),
+        row(5, "Hello", 1, 2, 2), row(6, "Hello world", 1, 3, 3)))
+  }
+
+  @Test
+  def testAggregateRemove(): Unit = {
+    val data = new mutable.MutableList[(Int, Int)]
+    data.+=((1, 1))
+    data.+=((2, 2))
+    data.+=((3, 3))
+    data.+=((4, 2))
+    data.+=((4, 4))
+    data.+=((6, 2))
+
+    val t = failingDataSource(data).toTable(tEnv, 'a, 'b)
+    tEnv.registerTable("T1", t)
+
+    val t1 = tEnv.sqlQuery(
+      """
+        |select sum(b) from
+        | (select b from
+        |   (select b, sum(a) from
+        |     (select b, sum(a) as a from T1 group by b) t1
+        |   group by b) t2
+        | ) t3
+      """.stripMargin)
+    val sink = new TestingRetractSink
+    t1.toRetractStream[Row].addSink(sink).setParallelism(1)
+    env.execute()
+    val expected = List("10")
+    assertEquals(expected, sink.getRetractResults)
+  }
+
+  private def checkResult(str: String, rows: Seq[Row]): Unit = {
+    super.before()
+
+    val ds1 = env.fromCollection(Seq[(Int, Int, String, String)](
+      (2, 1, "A", null),
+      (3, 2, "A", "Hi"),
+      (5, 2, "B", "Hello"),
+      (6, 3, "C", "Hello world")))
+    StreamTableEnvUtil.registerDataStreamInternal[(Int, Int, String, String)](
+      tEnv,
+      "T",
+      ds1.javaStream,
+      Some(Array("a", "b", "c", "d")),
+      Some(Array(true, true, true, true)),
+      Some(FlinkStatistic.builder().uniqueKeys(Set(Set("a").asJava).asJava).build())
+    )
+
+    StreamTableEnvUtil.registerDataStreamInternal[(Int, Long, String)](
+      tEnv,
+      "MyTable",
+      env.fromCollection(TestData.smallTupleData3).javaStream,
+      Some(Array("a", "b", "c")),
+      Some(Array(true, true, true)),
+      Some(FlinkStatistic.builder().uniqueKeys(Set(Set("a").asJava).asJava).build())
+    )
+
+    StreamTableEnvUtil.registerDataStreamInternal[(Int, Long, Int, String, Long)](
+      tEnv,
+      "MyTable2",
+      env.fromCollection(TestData.smallTupleData5).javaStream,
+      Some(Array("a", "b", "c", "d", "e")),
+      Some(Array(true, true, true, true, true)),
+      Some(FlinkStatistic.builder().uniqueKeys(Set(Set("b").asJava).asJava).build())
+    )
+
+    val t = tEnv.sqlQuery(str)
+    val sink = new TestingRetractSink
+    env.setMaxParallelism(1)
+    env.setParallelism(1)
+    t.toRetractStream[Row].addSink(sink).setParallelism(1)
+    env.execute()
+    val expected = rows.map(_.toString)
+    assertEquals(expected.sorted, sink.getRetractResults.sorted)
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala
index d141c28..1506b77fe 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala
@@ -25,7 +25,7 @@ import org.apache.flink.api.scala.typeutils.Types
 import org.apache.flink.table.`type`.InternalTypes
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.dataformat.{BaseRow, GenericRow}
-import org.apache.flink.table.runtime.utils.{StreamTestData, StreamingTestBase, TestSinkUtil, TestingAppendBaseRowSink, TestingAppendSink, TestingAppendTableSink}
+import org.apache.flink.table.runtime.utils.{StreamingTestBase, TestData, TestSinkUtil, TestingAppendBaseRowSink, TestingAppendSink, TestingAppendTableSink}
 import org.apache.flink.table.typeutils.BaseRowTypeInfo
 import org.apache.flink.types.Row
 
@@ -166,7 +166,7 @@ class CalcITCase extends StreamingTestBase {
   def testPrimitiveMapType(): Unit = {
     val sqlQuery = "SELECT MAP[b, 30, 10, a] FROM MyTableRow"
 
-    val t = env.fromCollection(StreamTestData.getSmall3TupleData)
+    val t = env.fromCollection(TestData.smallTupleData3)
             .toTable(tEnv, 'a, 'b, 'c)
     tEnv.registerTable("MyTableRow", t)
 
@@ -186,7 +186,7 @@ class CalcITCase extends StreamingTestBase {
   def testNonPrimitiveMapType(): Unit = {
     val sqlQuery = "SELECT MAP[a, c] FROM MyTableRow"
 
-    val t = env.fromCollection(StreamTestData.getSmall3TupleData)
+    val t = env.fromCollection(TestData.smallTupleData3)
             .toTable(tEnv, 'a, 'b, 'c)
     tEnv.registerTable("MyTableRow", t)
 
@@ -228,7 +228,7 @@ class CalcITCase extends StreamingTestBase {
   def testIn(): Unit = {
     val sqlQuery = "SELECT * FROM MyTable WHERE b in (1,3,4,5,6)"
 
-    val t = env.fromCollection(StreamTestData.get3TupleData)
+    val t = env.fromCollection(TestData.tupleData3)
       .toTable(tEnv, 'a, 'b, 'c)
     tEnv.registerTable("MyTable", t)
 
@@ -250,7 +250,7 @@ class CalcITCase extends StreamingTestBase {
   def testNotIn(): Unit = {
     val sqlQuery = "SELECT * FROM MyTable WHERE b not in (1,3,4,5,6)"
 
-    val t = env.fromCollection(StreamTestData.get3TupleData)
+    val t = env.fromCollection(TestData.tupleData3)
       .toTable(tEnv, 'a, 'b, 'c)
     tEnv.registerTable("MyTable", t)
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/DeduplicateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/DeduplicateITCase.scala
index 8abe855..9f818b8 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/DeduplicateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/DeduplicateITCase.scala
@@ -21,8 +21,8 @@ package org.apache.flink.table.runtime.stream.sql
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.runtime.utils.StreamingWithMiniBatchTestBase.MiniBatchMode
-import org.apache.flink.table.runtime.utils._
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.StateBackendMode
+import org.apache.flink.table.runtime.utils._
 import org.apache.flink.types.Row
 
 import org.junit.Assert._
@@ -36,7 +36,7 @@ class DeduplicateITCase(miniBatch: MiniBatchMode, mode: StateBackendMode)
 
   @Test
   def testFirstRowOnProctime(): Unit = {
-    val t = failingDataSource(StreamTestData.get3TupleData)
+    val t = failingDataSource(TestData.tupleData3)
       .toTable(tEnv, 'a, 'b, 'c, 'proctime)
     tEnv.registerTable("T", t)
 
@@ -62,7 +62,7 @@ class DeduplicateITCase(miniBatch: MiniBatchMode, mode: StateBackendMode)
 
   @Test
   def testLastRowOnProctime(): Unit = {
-    val t = failingDataSource(StreamTestData.get3TupleData)
+    val t = failingDataSource(TestData.tupleData3)
       .toTable(tEnv, 'a, 'b, 'c, 'proctime)
     tEnv.registerTable("T", t)
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala
index 72ca720..4f57887 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala
@@ -21,11 +21,12 @@ package org.apache.flink.table.runtime.stream.sql
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.runtime.utils.TimeTestUtil.EventTimeProcessOperator
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.StateBackendMode
+import org.apache.flink.table.runtime.utils.TimeTestUtil.EventTimeProcessOperator
 import org.apache.flink.table.runtime.utils.UserDefinedFunctionTestUtils.{CountNullNonNull, CountPairs, LargerThanCount}
-import org.apache.flink.table.runtime.utils.{StreamTestData, StreamingWithStateTestBase, TestingAppendSink}
+import org.apache.flink.table.runtime.utils.{StreamingWithStateTestBase, TestData, TestingAppendSink}
 import org.apache.flink.types.Row
+
 import org.junit.Assert._
 import org.junit._
 import org.junit.runner.RunWith
@@ -49,7 +50,7 @@ class OverWindowITCase(mode: StateBackendMode) extends StreamingWithStateTestBas
 
   @Test
   def testProcTimeBoundedPartitionedRowsOver(): Unit = {
-    val t = failingDataSource(StreamTestData.get5TupleData)
+    val t = failingDataSource(TestData.tupleData5)
       .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime)
     tEnv.registerTable("MyTable", t)
 
@@ -85,7 +86,7 @@ class OverWindowITCase(mode: StateBackendMode) extends StreamingWithStateTestBas
 
   @Test
   def testProcTimeBoundedPartitionedRowsOverWithBultinProctime(): Unit = {
-    val t = failingDataSource(StreamTestData.get5TupleData)
+    val t = failingDataSource(TestData.tupleData5)
       .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime)
     tEnv.registerTable("MyTable", t)
 
@@ -121,7 +122,7 @@ class OverWindowITCase(mode: StateBackendMode) extends StreamingWithStateTestBas
 
   @Test
   def testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime(): Unit = {
-    val t = failingDataSource(StreamTestData.get5TupleData)
+    val t = failingDataSource(TestData.tupleData5)
       .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime)
     tEnv.registerTable("MyTable", t)
 
@@ -157,7 +158,7 @@ class OverWindowITCase(mode: StateBackendMode) extends StreamingWithStateTestBas
 
   @Test
   def testProcTimeBoundedNonPartitionedRowsOver(): Unit = {
-    val t = failingDataSource(StreamTestData.get5TupleData)
+    val t = failingDataSource(TestData.tupleData5)
       .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime)
     tEnv.registerTable("MyTable", t)
 
@@ -856,7 +857,7 @@ class OverWindowITCase(mode: StateBackendMode) extends StreamingWithStateTestBas
 
   @Test
   def testProcTimeDistinctUnboundedPartitionedRowsOver(): Unit = {
-    val t = failingDataSource(StreamTestData.get5TupleData)
+    val t = failingDataSource(TestData.tupleData5)
       .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime)
     tEnv.registerTable("MyTable", t)
 
@@ -941,7 +942,7 @@ class OverWindowITCase(mode: StateBackendMode) extends StreamingWithStateTestBas
 
   @Test
   def testProcTimeDistinctBoundedPartitionedRowsOver(): Unit = {
-    val t = failingDataSource(StreamTestData.get5TupleData)
+    val t = failingDataSource(TestData.tupleData5)
       .toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime)
     tEnv.registerTable("MyTable", t)
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/PruneAggregateCallITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/PruneAggregateCallITCase.scala
new file mode 100644
index 0000000..35d2365
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/PruneAggregateCallITCase.scala
@@ -0,0 +1,130 @@
+/*
+ * 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.flink.table.runtime.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.plan.stats.FlinkStatistic
+import org.apache.flink.table.runtime.utils.BatchTestBase.row
+import org.apache.flink.table.runtime.utils.StreamingWithAggTestBase.AggMode
+import org.apache.flink.table.runtime.utils.StreamingWithMiniBatchTestBase.MiniBatchMode
+import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.StateBackendMode
+import org.apache.flink.table.runtime.utils.{StreamTableEnvUtil, StreamingWithAggTestBase, TestData, TestingRetractSink}
+import org.apache.flink.types.Row
+
+import org.junit.Assert.assertEquals
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+import scala.collection.Seq
+
+@RunWith(classOf[Parameterized])
+class PruneAggregateCallITCase(
+    aggMode: AggMode,
+    minibatch: MiniBatchMode,
+    backend: StateBackendMode)
+  extends StreamingWithAggTestBase(aggMode, minibatch, backend) {
+
+  @Test
+  def testNoneEmptyGroupKey(): Unit = {
+    checkResult(
+      "SELECT a FROM (SELECT b, MAX(a) AS a, COUNT(*), MAX(c) FROM MyTable GROUP BY b) t",
+      Seq(row(1), row(3))
+    )
+    checkResult(
+      """
+        |SELECT c, a FROM
+        | (SELECT a, c, COUNT(b) as b, SUM(b) as s FROM MyTable GROUP BY a, c) t
+        |WHERE s > 1
+      """.stripMargin,
+      Seq(row("Hello world", 3), row("Hello", 2))
+    )
+  }
+
+  @Test
+  def testEmptyGroupKey(): Unit = {
+    checkResult(
+      "SELECT 1 FROM (SELECT SUM(a) FROM MyTable) t",
+      Seq(row(1))
+    )
+
+    // TODO enable this case after translateToPlanInternal method is implemented
+    //  in StreamExecJoin
+    // checkResult(
+    //   "SELECT * FROM MyTable WHERE EXISTS (SELECT COUNT(*) FROM MyTable2)",
+    //   Seq(row(1, 1, "Hi"), row(2, 2, "Hello"), row(3, 2, "Hello world"))
+    // )
+
+    checkResult(
+      "SELECT 1 FROM (SELECT SUM(a), COUNT(*) FROM MyTable) t",
+      Seq(row(1))
+    )
+
+    checkResult(
+      "SELECT 1 FROM (SELECT COUNT(*), SUM(a) FROM MyTable) t",
+      Seq(row(1))
+    )
+
+    // TODO enable this case after translateToPlanInternal method is implemented
+    //  in StreamExecJoin
+    // checkResult(
+    //    "SELECT * FROM MyTable WHERE EXISTS (SELECT SUM(a), COUNT(*) FROM MyTable2)",
+    //    Seq(row(1, 1, "Hi"), row(2, 2, "Hello"), row(3, 2, "Hello world"))
+    // )
+
+    // TODO enable this case after translateToPlanInternal method is implemented
+    //  in StreamExecJoin
+    // checkResult(
+    //    "SELECT * FROM MyTable WHERE EXISTS (SELECT COUNT(*), SUM(a) FROM MyTable2)",
+    //    Seq(row(1, 1, "Hi"), row(2, 2, "Hello"), row(3, 2, "Hello world"))
+    //  )
+  }
+
+  private def checkResult(str: String, rows: Seq[Row]): Unit = {
+    super.before()
+    StreamTableEnvUtil.registerDataStreamInternal[(Int, Long, String)](
+      tEnv,
+      "MyTable",
+      failingDataSource(TestData.smallTupleData3).javaStream,
+      Some(Array("a", "b", "c")),
+      Some(Array(true, true, true)),
+      Some(FlinkStatistic.UNKNOWN)
+    )
+
+    StreamTableEnvUtil.registerDataStreamInternal[(Int, Long, Int, String, Long)](
+      tEnv,
+      "MyTable2",
+      failingDataSource(TestData.smallTupleData5).javaStream,
+      Some(Array("a", "b", "c", "d", "e")),
+      Some(Array(true, true, true, true, true)),
+      Some(FlinkStatistic.builder().uniqueKeys(Set(Set("b").asJava).asJava).build())
+    )
+
+    val t = tEnv.sqlQuery(str)
+    val sink = new TestingRetractSink
+    env.setMaxParallelism(1)
+    env.setParallelism(1)
+    t.toRetractStream[Row].addSink(sink).setParallelism(1)
+    env.execute()
+    val expected = rows.map(_.toString)
+    assertEquals(expected.sorted, sink.getRetractResults.sorted)
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalSortITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalSortITCase.scala
index eccc276..acee2ce 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalSortITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalSortITCase.scala
@@ -123,7 +123,7 @@ class TemporalSortITCase(mode: StateBackendMode) extends StreamingWithStateTestB
 
   @Test
   def testProcTimeOrderBy(): Unit = {
-    val t = failingDataSource(StreamTestData.get3TupleData)
+    val t = failingDataSource(TestData.tupleData3)
       .toTable(tEnv, 'a, 'b, 'c, 'proctime)
     tEnv.registerTable("T", t)
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchScalaTableEnvUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchScalaTableEnvUtil.scala
index b77056f..9b40562 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchScalaTableEnvUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchScalaTableEnvUtil.scala
@@ -20,6 +20,7 @@ package org.apache.flink.table.runtime.utils
 
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.api.{BatchTableEnvironment, Table, TableEnvironment}
+import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.runtime.utils.BatchTableEnvUtil.parseFieldNames
 
 import scala.reflect.ClassTag
@@ -40,7 +41,7 @@ object BatchScalaTableEnvUtil {
       tableName: String, data: Iterable[T], fieldNames: String): Unit = {
     val typeInfo = implicitly[TypeInformation[T]]
     BatchTableEnvUtil.registerCollection(
-      tEnv, tableName, data, typeInfo, Some(parseFieldNames(fieldNames)), None)
+      tEnv, tableName, data, typeInfo, Some(parseFieldNames(fieldNames)), None, None)
   }
 
   /**
@@ -51,15 +52,20 @@ object BatchScalaTableEnvUtil {
     * @param data The [[Iterable]] to be converted.
     * @param fieldNames field names, eg: "a, b, c"
     * @param fieldNullables The field isNullables attributes of data.
+    * @param statistic statistics of current Table
     * @tparam T The type of the [[Iterable]].
     * @return The converted [[Table]].
     */
-  def registerCollection[T : ClassTag : TypeInformation](
-      tEnv: BatchTableEnvironment, tableName: String, data: Iterable[T],
-      fieldNames: String, fieldNullables: Array[Boolean]): Unit = {
+  def registerCollection[T: ClassTag : TypeInformation](
+      tEnv: BatchTableEnvironment,
+      tableName: String,
+      data: Iterable[T],
+      fieldNames: String,
+      fieldNullables: Array[Boolean],
+      statistic: Option[FlinkStatistic]): Unit = {
     val typeInfo = implicitly[TypeInformation[T]]
-    BatchTableEnvUtil.registerCollection(
-      tEnv, tableName, data, typeInfo, Some(parseFieldNames(fieldNames)), Option(fieldNullables))
+    BatchTableEnvUtil.registerCollection(tEnv, tableName, data, typeInfo,
+      Some(parseFieldNames(fieldNames)), Option(fieldNullables), statistic)
   }
 
   /**
@@ -91,7 +97,7 @@ object BatchScalaTableEnvUtil {
   def fromCollection[T: ClassTag : TypeInformation](
       tEnv: BatchTableEnvironment, data: Iterable[T]): Table = {
     val typeInfo = implicitly[TypeInformation[T]]
-    BatchTableEnvUtil.fromCollection(tEnv, null, data, typeInfo, null)
+    BatchTableEnvUtil.fromCollection(tEnv, null, data, typeInfo, null, None)
   }
 
   /**
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTableEnvUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTableEnvUtil.scala
index be32bff..fe93155 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTableEnvUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTableEnvUtil.scala
@@ -26,6 +26,7 @@ import org.apache.flink.api.java.io.CollectionInputFormat
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.{BatchTableEnvironment, Table, TableEnvironment}
 import org.apache.flink.table.plan.schema.DataStreamTable
+import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.sinks.CollectTableSink
 import org.apache.flink.util.AbstractID
 
@@ -70,7 +71,7 @@ object BatchTableEnvUtil {
       tableName: String, data: Iterable[T], typeInfo: TypeInformation[T],
       fieldNames: String): Unit = {
     registerCollection(
-      tEnv, tableName, data, typeInfo, Some(parseFieldNames(fieldNames)), None)
+      tEnv, tableName, data, typeInfo, Some(parseFieldNames(fieldNames)), None, None)
   }
 
   /**
@@ -82,6 +83,7 @@ object BatchTableEnvUtil {
     * @param typeInfo information of [[Iterable]].
     * @param fieldNames field names, eg: "a, b, c"
     * @param fieldNullables The field isNullables attributes of data.
+    * @param statistic statistics of current Table
     * @tparam T The type of the [[Iterable]].
     * @return The converted [[Table]].
     */
@@ -91,9 +93,10 @@ object BatchTableEnvUtil {
       data: Iterable[T],
       typeInfo: TypeInformation[T],
       fieldNames: String,
-      fieldNullables: Array[Boolean]): Unit = {
-    registerCollection(
-      tEnv, tableName, data, typeInfo, Some(parseFieldNames(fieldNames)), Option(fieldNullables))
+      fieldNullables: Array[Boolean],
+      statistic: Option[FlinkStatistic]): Unit = {
+    registerCollection(tEnv, tableName, data, typeInfo,
+      Some(parseFieldNames(fieldNames)), Option(fieldNullables), statistic)
   }
 
   /**
@@ -105,6 +108,7 @@ object BatchTableEnvUtil {
     * @param typeInfo information of [[Iterable]].
     * @param fieldNames field names.
     * @param fieldNullables The field isNullables attributes of data.
+    * @param statistic statistics of current Table
     * @tparam T The type of the [[Iterable]].
     * @return The converted [[Table]].
     */
@@ -115,13 +119,15 @@ object BatchTableEnvUtil {
       data: Iterable[T],
       typeInfo: TypeInformation[T],
       fieldNames: Option[Array[String]],
-      fieldNullables: Option[Array[Boolean]]): Unit = {
+      fieldNullables: Option[Array[Boolean]],
+      statistic: Option[FlinkStatistic]): Unit = {
     val boundedStream = tEnv.streamEnv.createInput(new CollectionInputFormat[T](
       data.asJavaCollection,
       typeInfo.createSerializer(tEnv.streamEnv.getConfig)),
       typeInfo)
     boundedStream.forceNonParallel()
-    registerBoundedStreamInternal(tEnv, tableName, boundedStream, fieldNames, fieldNullables)
+    registerBoundedStreamInternal(
+      tEnv, tableName, boundedStream, fieldNames, fieldNullables, statistic)
   }
 
   /**
@@ -137,12 +143,14 @@ object BatchTableEnvUtil {
       name: String,
       boundedStream: DataStream[T],
       fieldNames: Option[Array[String]],
-      fieldNullables: Option[Array[Boolean]]): Unit = {
+      fieldNullables: Option[Array[Boolean]],
+      statistic: Option[FlinkStatistic]): Unit = {
     val (typeFieldNames, fieldIdxs) =
       tEnv.getFieldInfo(boundedStream.getTransformation.getOutputType)
     val boundedStreamTable = new DataStreamTable[T](
       boundedStream, fieldIdxs, fieldNames.getOrElse(typeFieldNames), fieldNullables)
-    tEnv.registerTableInternal(name, boundedStreamTable)
+    val withStatistic = boundedStreamTable.copy(statistic.getOrElse(FlinkStatistic.UNKNOWN))
+    tEnv.registerTableInternal(name, withStatistic)
   }
 
   /**
@@ -154,7 +162,8 @@ object BatchTableEnvUtil {
       tableName: String,
       data: Iterable[T],
       typeInfo: TypeInformation[T],
-      fieldNames: Array[String]): Table = {
+      fieldNames: Array[String],
+      statistic: Option[FlinkStatistic]): Table = {
     CollectionInputFormat.checkCollection(data.asJavaCollection, typeInfo.getTypeClass)
     val boundedStream = tEnv.streamEnv.createInput(new CollectionInputFormat[T](
       data.asJavaCollection,
@@ -162,7 +171,7 @@ object BatchTableEnvUtil {
       typeInfo)
     boundedStream.setParallelism(1)
     val name = if (tableName == null) tEnv.createUniqueTableName() else tableName
-    registerBoundedStreamInternal(tEnv, name, boundedStream, Option(fieldNames), None)
+    registerBoundedStreamInternal(tEnv, name, boundedStream, Option(fieldNames), None, statistic)
     tEnv.scan(name)
   }
 
@@ -172,6 +181,6 @@ object BatchTableEnvUtil {
     */
   def fromCollection[T](tEnv: BatchTableEnvironment,
       data: Iterable[T], typeInfo: TypeInformation[T], fields: String): Table = {
-    fromCollection(tEnv, null, data, typeInfo, parseFieldNames(fields))
+    fromCollection(tEnv, null, data, typeInfo, parseFieldNames(fields), None)
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
index 0202793..8d032cd 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
@@ -29,6 +29,7 @@ import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTabl
 import org.apache.flink.table.api.{SqlParserException, Table, TableConfig, TableConfigOptions, TableEnvironment, TableImpl}
 import org.apache.flink.table.dataformat.{BinaryRow, BinaryRowWriter}
 import org.apache.flink.table.functions.AggregateFunction
+import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.plan.util.FlinkRelOptUtil
 import org.apache.flink.table.runtime.utils.BatchAbstractTestBase.DEFAULT_PARALLELISM
 import org.apache.flink.table.util.{BaseRowTestUtil, DiffRepository}
@@ -388,18 +389,21 @@ class BatchTestBase extends BatchAbstractTestBase {
       tableName: String,
       data: Iterable[T],
       typeInfo: TypeInformation[T],
-      fieldNullables: Array[Boolean],
-      fields: String): Unit = {
-    BatchTableEnvUtil.registerCollection(tEnv, tableName, data, typeInfo, fields, fieldNullables)
+      fields: String,
+      fieldNullables: Array[Boolean]): Unit = {
+    BatchTableEnvUtil.registerCollection(
+      tEnv, tableName, data, typeInfo, fields, fieldNullables, None)
   }
 
-  def registerCollection(
+  def registerCollection[T](
       tableName: String,
-      data: Iterable[Row],
-      typeInfo: TypeInformation[Row],
+      data: Iterable[T],
+      typeInfo: TypeInformation[T],
       fields: String,
-      fieldNullables: Array[Boolean]): Unit = {
-    BatchTableEnvUtil.registerCollection(tEnv, tableName, data, typeInfo, fields, fieldNullables)
+      fieldNullables: Array[Boolean],
+      statistic: FlinkStatistic): Unit = {
+    BatchTableEnvUtil.registerCollection(
+      tEnv, tableName, data, typeInfo, fields, fieldNullables, Some(statistic))
   }
 
   def registerFunction[T: TypeInformation, ACC: TypeInformation](
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamTableEnvUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamTableEnvUtil.scala
new file mode 100644
index 0000000..9731409
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamTableEnvUtil.scala
@@ -0,0 +1,52 @@
+/*
+ * 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.flink.table.runtime.utils
+
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamTableEnvironment, TableEnvironment}
+import org.apache.flink.table.plan.schema.DataStreamTable
+import org.apache.flink.table.plan.stats.FlinkStatistic
+
+object StreamTableEnvUtil {
+
+  //  TODO unify BatchTableEnvUtil and StreamTableEnvUtil
+  /**
+    * Registers a [[DataStream]] as a table under a given name in the [[TableEnvironment]]'s
+    * catalog.
+    *
+    * @param name     The name under which the table is registered in the catalog.
+    * @param dataStream The [[DataStream]] to register as table in the catalog.
+    * @tparam T the type of the [[DataStream]].
+    */
+  def registerDataStreamInternal[T](
+      tEnv: StreamTableEnvironment,
+      name: String,
+      dataStream: DataStream[T],
+      fieldNames: Option[Array[String]],
+      fieldNullables: Option[Array[Boolean]],
+      statistic: Option[FlinkStatistic]): Unit = {
+    val (typeFieldNames, fieldIdxs) =
+      tEnv.getFieldInfo(dataStream.getTransformation.getOutputType)
+    val boundedStreamTable = new DataStreamTable[T](
+      dataStream, fieldIdxs, fieldNames.getOrElse(typeFieldNames), fieldNullables)
+    val withStatistic = boundedStreamTable.copy(statistic.getOrElse(FlinkStatistic.UNKNOWN))
+    tEnv.registerTableInternal(name, withStatistic)
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestData.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestData.scala
deleted file mode 100644
index f437ab8..0000000
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestData.scala
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.runtime.utils
-
-import org.apache.flink.api.scala._
-import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
-
-import java.sql.{Date, Time, Timestamp}
-
-import scala.collection.mutable
-
-object StreamTestData {
-
-  def getSmall3TupleData: Seq[(Int, Long, String)] = {
-    val data = new mutable.MutableList[(Int, Long, String)]
-    data.+=((1, 1L, "Hi"))
-    data.+=((2, 2L, "Hello"))
-    data.+=((3, 2L, "Hello world"))
-    data
-  }
-
-  def get3TupleData: Seq[(Int, Long, String)] = {
-    val data = new mutable.MutableList[(Int, Long, String)]
-    data.+=((1, 1L, "Hi"))
-    data.+=((2, 2L, "Hello"))
-    data.+=((3, 2L, "Hello world"))
-    data.+=((4, 3L, "Hello world, how are you?"))
-    data.+=((5, 3L, "I am fine."))
-    data.+=((6, 3L, "Luke Skywalker"))
-    data.+=((7, 4L, "Comment#1"))
-    data.+=((8, 4L, "Comment#2"))
-    data.+=((9, 4L, "Comment#3"))
-    data.+=((10, 4L, "Comment#4"))
-    data.+=((11, 5L, "Comment#5"))
-    data.+=((12, 5L, "Comment#6"))
-    data.+=((13, 5L, "Comment#7"))
-    data.+=((14, 5L, "Comment#8"))
-    data.+=((15, 5L, "Comment#9"))
-    data.+=((16, 6L, "Comment#10"))
-    data.+=((17, 6L, "Comment#11"))
-    data.+=((18, 6L, "Comment#12"))
-    data.+=((19, 6L, "Comment#13"))
-    data.+=((20, 6L, "Comment#14"))
-    data.+=((21, 6L, "Comment#15"))
-    data
-  }
-
-  def get5TupleData: Seq[(Int, Long, Int, String, Long)] = {
-    val data = new mutable.MutableList[(Int, Long, Int, String, Long)]
-    data.+=((1, 1L, 0, "Hallo", 1L))
-    data.+=((2, 2L, 1, "Hallo Welt", 2L))
-    data.+=((2, 3L, 2, "Hallo Welt wie", 1L))
-    data.+=((3, 4L, 3, "Hallo Welt wie gehts?", 2L))
-    data.+=((3, 5L, 4, "ABC", 2L))
-    data.+=((3, 6L, 5, "BCD", 3L))
-    data.+=((4, 7L, 6, "CDE", 2L))
-    data.+=((4, 8L, 7, "DEF", 1L))
-    data.+=((4, 9L, 8, "EFG", 1L))
-    data.+=((4, 10L, 9, "FGH", 2L))
-    data.+=((5, 11L, 10, "GHI", 1L))
-    data.+=((5, 12L, 11, "HIJ", 3L))
-    data.+=((5, 13L, 12, "IJK", 3L))
-    data.+=((5, 14L, 13, "JKL", 2L))
-    data.+=((5, 15L, 14, "KLM", 2L))
-    data
-  }
-
-  def getSmall3TupleDataStream(env: StreamExecutionEnvironment): DataStream[(Int, Long, String)] = {
-    env.fromCollection(getSmall3TupleData)
-  }
-
-  def get3TupleDataStream(env: StreamExecutionEnvironment): DataStream[(Int, Long, String)] = {
-    env.fromCollection(get3TupleData)
-  }
-
-  def get5TupleDataStream(env: StreamExecutionEnvironment):
-      DataStream[(Int, Long, Int, String, Long)] = {
-    env.fromCollection(get5TupleData)
-  }
-
-
-  def getTimeZoneTestData(env: StreamExecutionEnvironment):
-      DataStream[(Int, Date, Time, Timestamp)] = {
-    val MiLLIS_PER_DAY  = 24 * 3600 * 1000
-    val MiLLIS_PER_HOUR  = 3600 * 1000
-    val data = new mutable.MutableList[(Int, Date, Time, Timestamp)]
-    data.+=((1, new Date(0), new Time(0), new Timestamp(1)))
-    data.+=((2, new Date(1*MiLLIS_PER_DAY), new Time(MiLLIS_PER_HOUR), new Timestamp(2)))
-    data.+=((3, new Date(2*MiLLIS_PER_DAY), new Time(2*MiLLIS_PER_HOUR), new Timestamp(3)))
-
-    env.fromCollection(data)
-  }
-}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestData.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestData.scala
index 9409e73..8fb0a2b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestData.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestData.scala
@@ -29,7 +29,7 @@ import org.apache.flink.types.Row
 import java.math.{BigDecimal => JBigDecimal}
 import java.sql.Timestamp
 
-import scala.collection.Seq
+import scala.collection.{Seq, mutable}
 
 object TestData {
 
@@ -72,23 +72,31 @@ object TestData {
 
   val nullablesOfNullData5 = Array(true, false, false, false, false)
 
-  lazy val smallData3 = Seq(
-    row(1, 1L, "Hi"),
-    row(2, 2L, "Hello"),
-    row(3, 2L, "Hello world")
-  )
+  lazy val smallTupleData3: Seq[(Int, Long, String)] = {
+    val data = new mutable.MutableList[(Int, Long, String)]
+    data.+=((1, 1L, "Hi"))
+    data.+=((2, 2L, "Hello"))
+    data.+=((3, 2L, "Hello world"))
+    data
+  }
+
+  lazy val smallData3: Seq[Row] = smallTupleData3.map(d => row(d.productIterator.toList: _*))
 
   val nullablesOfSmallData3 = Array(false, false, false)
 
-  lazy val smallData5 = Seq(
-    row(1, 1L, 0, "Hallo", 1L),
-    row(2, 2L, 1, "Hallo Welt", 2L),
-    row(2, 3L, 2, "Hallo Welt wie", 1L)
-  )
+  lazy val smallTupleData5: Seq[(Int, Long, Int, String, Long)] = {
+    val data = new mutable.MutableList[(Int, Long, Int, String, Long)]
+    data.+=((1, 1L, 0, "Hallo", 1L))
+    data.+=((2, 2L, 1, "Hallo Welt", 2L))
+    data.+=((2, 3L, 2, "Hallo Welt wie", 1L))
+    data
+  }
+
+  lazy val smallData5: Seq[Row] = smallTupleData5.map(d => row(d.productIterator.toList: _*))
 
   val nullablesOfSmallData5 = Array(false, false, false, false, false)
 
-  lazy val buildInData = Seq(
+  lazy val buildInData: Seq[Row] = Seq(
     row(false, 1.toByte, 2, 3L, 2.56, "abcd", "f%g", UTCDate("2017-12-12"),
       UTCTime("10:08:09"), UTCTimestamp("2017-11-11 20:32:19")),
 
@@ -99,33 +107,37 @@ object TestData {
       UTCTime("10:08:09"), UTCTimestamp("2015-05-20 10:00:00.887"))
   )
 
-  lazy val data3 = Seq(
-    row(1, 1L, "Hi"),
-    row(2, 2L, "Hello"),
-    row(3, 2L, "Hello world"),
-    row(4, 3L, "Hello world, how are you?"),
-    row(5, 3L, "I am fine."),
-    row(6, 3L, "Luke Skywalker"),
-    row(7, 4L, "Comment#1"),
-    row(8, 4L, "Comment#2"),
-    row(9, 4L, "Comment#3"),
-    row(10, 4L, "Comment#4"),
-    row(11, 5L, "Comment#5"),
-    row(12, 5L, "Comment#6"),
-    row(13, 5L, "Comment#7"),
-    row(14, 5L, "Comment#8"),
-    row(15, 5L, "Comment#9"),
-    row(16, 6L, "Comment#10"),
-    row(17, 6L, "Comment#11"),
-    row(18, 6L, "Comment#12"),
-    row(19, 6L, "Comment#13"),
-    row(20, 6L, "Comment#14"),
-    row(21, 6L, "Comment#15")
-  )
+  lazy val tupleData3: Seq[(Int, Long, String)] = {
+    val data = new mutable.MutableList[(Int, Long, String)]
+    data.+=((1, 1L, "Hi"))
+    data.+=((2, 2L, "Hello"))
+    data.+=((3, 2L, "Hello world"))
+    data.+=((4, 3L, "Hello world, how are you?"))
+    data.+=((5, 3L, "I am fine."))
+    data.+=((6, 3L, "Luke Skywalker"))
+    data.+=((7, 4L, "Comment#1"))
+    data.+=((8, 4L, "Comment#2"))
+    data.+=((9, 4L, "Comment#3"))
+    data.+=((10, 4L, "Comment#4"))
+    data.+=((11, 5L, "Comment#5"))
+    data.+=((12, 5L, "Comment#6"))
+    data.+=((13, 5L, "Comment#7"))
+    data.+=((14, 5L, "Comment#8"))
+    data.+=((15, 5L, "Comment#9"))
+    data.+=((16, 6L, "Comment#10"))
+    data.+=((17, 6L, "Comment#11"))
+    data.+=((18, 6L, "Comment#12"))
+    data.+=((19, 6L, "Comment#13"))
+    data.+=((20, 6L, "Comment#14"))
+    data.+=((21, 6L, "Comment#15"))
+    data
+  }
+
+  lazy val data3: Seq[Row] = tupleData3.map(d => row(d.productIterator.toList: _*))
 
   val nullablesOfData3 = Array(false, false, false)
 
-  lazy val genericData3 = Seq(
+  lazy val genericData3: Seq[Row] = Seq(
     row(new JTuple2("1", 1), new JTuple2(1, 1), 1),
     row(new JTuple2("2", 1), new JTuple2(1, 1), 2),
     row(new JTuple2("1", 1), new JTuple2(1, 1), 1),
@@ -134,7 +146,7 @@ object TestData {
 
   val nullablesOfData3WithTimestamp = Array(true, false, false, false)
 
-  lazy val data3WithTimestamp = Seq(
+  lazy val data3WithTimestamp: Seq[Row] = Seq(
     row(2, 2L, "Hello", new Timestamp(2000L)),
     row(1, 1L, "Hi", new Timestamp(1000L)),
     row(3, 2L, "Hello world", new Timestamp(3000L)),
@@ -158,27 +170,31 @@ object TestData {
     row(21, 6L, "Comment#15", new Timestamp(21000L))
   )
 
-  lazy val data5 = Seq(
-    row(1, 1L, 0, "Hallo", 1L),
-    row(2, 2L, 1, "Hallo Welt", 2L),
-    row(2, 3L, 2, "Hallo Welt wie", 1L),
-    row(3, 4L, 3, "Hallo Welt wie gehts?", 2L),
-    row(3, 5L, 4, "ABC", 2L),
-    row(3, 6L, 5, "BCD", 3L),
-    row(4, 7L, 6, "CDE", 2L),
-    row(4, 8L, 7, "DEF", 1L),
-    row(4, 9L, 8, "EFG", 1L),
-    row(4, 10L, 9, "FGH", 2L),
-    row(5, 11L, 10, "GHI", 1L),
-    row(5, 12L, 11, "HIJ", 3L),
-    row(5, 13L, 12, "IJK", 3L),
-    row(5, 14L, 13, "JKL", 2L),
-    row(5, 15L, 14, "KLM", 2L)
-  )
+  lazy val tupleData5: Seq[(Int, Long, Int, String, Long)] = {
+    val data = new mutable.MutableList[(Int, Long, Int, String, Long)]
+    data.+=((1, 1L, 0, "Hallo", 1L))
+    data.+=((2, 2L, 1, "Hallo Welt", 2L))
+    data.+=((2, 3L, 2, "Hallo Welt wie", 1L))
+    data.+=((3, 4L, 3, "Hallo Welt wie gehts?", 2L))
+    data.+=((3, 5L, 4, "ABC", 2L))
+    data.+=((3, 6L, 5, "BCD", 3L))
+    data.+=((4, 7L, 6, "CDE", 2L))
+    data.+=((4, 8L, 7, "DEF", 1L))
+    data.+=((4, 9L, 8, "EFG", 1L))
+    data.+=((4, 10L, 9, "FGH", 2L))
+    data.+=((5, 11L, 10, "GHI", 1L))
+    data.+=((5, 12L, 11, "HIJ", 3L))
+    data.+=((5, 13L, 12, "IJK", 3L))
+    data.+=((5, 14L, 13, "JKL", 2L))
+    data.+=((5, 15L, 14, "KLM", 2L))
+    data
+  }
+
+  lazy val data5: Seq[Row] = tupleData5.map(d => row(d.productIterator.toList: _*))
 
   val nullablesOfData5 = Array(false, false, false, false, false)
 
-  lazy val data6 = Seq(
+  lazy val data6: Seq[Row] = Seq(
     row(1,   1.1, "a",    UTCDate("2017-04-08"), UTCTime("12:00:59"),
       UTCTimestamp("2015-05-20 10:00:00")),
     row(2,   2.5, "abc",  UTCDate("2017-04-09"), UTCTime("12:00:59"),
@@ -213,7 +229,7 @@ object TestData {
 
   val nullablesOfData6 = Array(false, false, false, false, false, false)
 
-  lazy val duplicateData5 = Seq(
+  lazy val duplicateData5: Seq[Row] = Seq(
     row(1, 1L, 10, "Hallo", 1L),
     row(2, 2L, 11, "Hallo Welt", 2L),
     row(2, 3L, 12, "Hallo Welt wie", 1L),
@@ -233,7 +249,7 @@ object TestData {
 
   val nullablesOfDuplicateData5 = Array(false, false, false, false, false)
 
-  lazy val numericData = Seq(
+  lazy val numericData: Seq[Row] = Seq(
     row(1, 1L, 1.0f, 1.0d, JBigDecimal.valueOf(1)),
     row(2, 2L, 2.0f, 2.0d, JBigDecimal.valueOf(2)),
     row(3, 3L, 3.0f, 3.0d, JBigDecimal.valueOf(3))
@@ -242,7 +258,7 @@ object TestData {
   val nullablesOfNumericData = Array(false, false, false, false, false)
 
   // person test data
-  lazy val personData = Seq(
+  lazy val personData: Seq[Row] = Seq(
     row(1, 23, "tom", 172, "m"),
     row(2, 21, "mary", 161, "f"),
     row(3, 18, "jack", 182, "m"),
@@ -265,7 +281,7 @@ object TestData {
   val INT_ONLY = new RowTypeInfo(INT_TYPE_INFO)
   val INT_INT = new RowTypeInfo(INT_TYPE_INFO, INT_TYPE_INFO)
 
-  lazy val data2_1 = Seq(
+  lazy val data2_1: Seq[Row] = Seq(
     row(1, 2.0),
     row(1, 2.0),
     row(2, 1.0),
@@ -276,7 +292,7 @@ object TestData {
     row(6, null)
   )
 
-  lazy val data2_2 = Seq(
+  lazy val data2_2: Seq[Row] = Seq(
     row(2, 3.0),
     row(2, 3.0),
     row(3, 2.0),
@@ -286,7 +302,7 @@ object TestData {
     row(6, null)
   )
 
-  lazy val data2_3 = Seq(
+  lazy val data2_3: Seq[Row] = Seq(
     row(2, 3.0),
     row(2, 3.0),
     row(3, 2.0),
@@ -323,7 +339,7 @@ object TestData {
 
   val nullablesOfIntIntData3 = Array(false, false)
 
-  lazy val upperCaseData = Seq(
+  lazy val upperCaseData: Seq[Row] = Seq(
     row(1, "A"),
     row(2, "B"),
     row(3, "C"),
@@ -333,7 +349,7 @@ object TestData {
 
   val nullablesOfUpperCaseData = Array(false, false)
 
-  lazy val lowerCaseData = Seq(
+  lazy val lowerCaseData: Seq[Row] = Seq(
     row(1, "a"),
     row(2, "b"),
     row(3, "c"),
@@ -341,7 +357,7 @@ object TestData {
 
   val nullablesOfLowerCaseData = Array(false, false)
 
-  lazy val allNulls = Seq(
+  lazy val allNulls: Seq[Row] = Seq(
     row(null),
     row(null),
     row(null),
@@ -349,7 +365,7 @@ object TestData {
 
   val nullablesOfAllNulls = Array(true)
 
-  lazy val projectionTestData = Seq(
+  lazy val projectionTestData: Seq[Row] = Seq(
     row(1, 10, 100, "1", "10", "100", 1000, "1000"),
     row(2, 20, 200, "2", "20", "200", 2000, "2000"),
     row(3, 30, 300, "3", "30", "300", 3000, "3000"))
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
index fedd7e8..be85b3c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
@@ -144,16 +144,14 @@ abstract class TableTestUtil(test: TableTestBase) {
     * @param name table name
     * @param types field types
     * @param names field names
-    * @param tableStats table stats
-    * @param uniqueKeys unique keys
+    * @param statistic statistic of current table
     * @return returns the registered [[Table]].
     */
   def addTableSource(
       name: String,
       types: Array[TypeInformation[_]],
       names: Array[String],
-      tableStats: Option[TableStats] = None,
-      uniqueKeys: Option[JSet[_ <: JSet[String]]] = None): Table
+      statistic: FlinkStatistic = FlinkStatistic.UNKNOWN): Table
 
   /**
     * Create a [[DataStream]] with the given schema,
@@ -478,15 +476,10 @@ case class StreamTableTestUtil(test: TableTestBase) extends TableTestUtil(test)
       name: String,
       types: Array[TypeInformation[_]],
       names: Array[String],
-      tableStats: Option[TableStats] = None,
-      uniqueKeys: Option[JSet[_ <: JSet[String]]] = None): Table = {
+      statistic: FlinkStatistic = FlinkStatistic.UNKNOWN): Table = {
     val tableEnv = getTableEnv
     val schema = new TableSchema(names, types)
     val tableSource = new TestTableSource(schema)
-    val statistic = FlinkStatistic.builder()
-      .tableStats(tableStats.orNull)
-      .uniqueKeys(uniqueKeys.orNull)
-      .build()
     val table = new StreamTableSourceTable[BaseRow](tableSource, statistic)
     tableEnv.registerTableInternal(name, table)
     tableEnv.scan(name)
@@ -596,15 +589,10 @@ case class BatchTableTestUtil(test: TableTestBase) extends TableTestUtil(test) {
       name: String,
       types: Array[TypeInformation[_]],
       names: Array[String],
-      tableStats: Option[TableStats] = None,
-      uniqueKeys: Option[JSet[_ <: JSet[String]]] = None): Table = {
+      statistic: FlinkStatistic = FlinkStatistic.UNKNOWN): Table = {
     val tableEnv = getTableEnv
     val schema = new TableSchema(names, types)
     val tableSource = new TestTableSource(schema)
-    val statistic = FlinkStatistic.builder()
-      .tableStats(tableStats.orNull)
-      .uniqueKeys(uniqueKeys.orNull)
-      .build()
     val table = new BatchTableSourceTable[BaseRow](tableSource, statistic)
     tableEnv.registerTableInternal(name, table)
     tableEnv.scan(name)