cloud-fan commented on a change in pull request #33352:
URL: https://github.com/apache/spark/pull/33352#discussion_r670676063



##########
File path: 
sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownAggregates.java
##########
@@ -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.spark.sql.connector.read;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.connector.expressions.Aggregation;
+
+/**
+ * A mix-in interface for {@link ScanBuilder}. Data source can implement this 
interface to
+ * push down aggregates. Depends on the data source implementation, the 
aggregates may not

Review comment:
       ```
   A mix-in interface for {@link ScanBuilder}. Data sources can implement this 
interface to
   push down aggregates. Spark assumes that the data source can't fully 
complete the
   grouping work, and will group the data source output again. For queries like
   "SELECT min(value) AS m FROM t GROUP BY key", after pushing down the 
aggregate
   to the data source, the data source can still output data with duplicated 
keys, which is OK
   as Spark will do GROUP BY key again. The final query plan can be something 
like this:
   {{{
     Aggregate [key#1], [min(min(value)#2) AS m#3]
       +- RelationV2[key#1, min(value)#2]
   }}}
   
   Similarly, if there is no grouping expression, the data source can still 
output more than one rows.
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to