[SPARK-34952][SQL][FOLLOWUP] Move aggregates to a separate package

### What changes were proposed in this pull request?
Add `aggregate` package under `sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions` and move all the aggregates (e.g. `Count`, `Max`, `Min`, etc.) there.

### Why are the changes needed?
Right now these aggregates are under `sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions`. It looks OK now, but we plan to add a new `filter` package under `expressions` for all the DSV2 filters. It will look strange that filters have their own package, but aggregates don't.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing tests

Closes #33815 from huaxingao/agg_package.

Authored-by: Huaxin Gao <huaxin_gao@apple.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
This commit is contained in:
Huaxin Gao 2021-08-23 15:31:13 -07:00 committed by Liang-Chi Hsieh
parent 9f595c4ce3
commit cd2342691d
14 changed files with 26 additions and 18 deletions

View file

@ -15,12 +15,13 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.spark.sql.connector.expressions; package org.apache.spark.sql.connector.expressions.aggregate;
import org.apache.spark.annotation.Evolving;
import java.io.Serializable; import java.io.Serializable;
import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.Expression;
/** /**
* Base class of the Aggregate Functions. * Base class of the Aggregate Functions.
* *

View file

@ -15,12 +15,13 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.spark.sql.connector.expressions; package org.apache.spark.sql.connector.expressions.aggregate;
import org.apache.spark.annotation.Evolving;
import java.io.Serializable; import java.io.Serializable;
import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.FieldReference;
/** /**
* Aggregation in SQL statement. * Aggregation in SQL statement.
* *

View file

@ -15,9 +15,10 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.spark.sql.connector.expressions; package org.apache.spark.sql.connector.expressions.aggregate;
import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.FieldReference;
/** /**
* An aggregate function that returns the number of the specific row in a group. * An aggregate function that returns the number of the specific row in a group.

View file

@ -15,7 +15,7 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.spark.sql.connector.expressions; package org.apache.spark.sql.connector.expressions.aggregate;
import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Evolving;

View file

@ -15,9 +15,10 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.spark.sql.connector.expressions; package org.apache.spark.sql.connector.expressions.aggregate;
import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.FieldReference;
/** /**
* An aggregate function that returns the maximum value in a group. * An aggregate function that returns the maximum value in a group.

View file

@ -15,9 +15,10 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.spark.sql.connector.expressions; package org.apache.spark.sql.connector.expressions.aggregate;
import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.FieldReference;
/** /**
* An aggregate function that returns the minimum value in a group. * An aggregate function that returns the minimum value in a group.

View file

@ -15,9 +15,10 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.spark.sql.connector.expressions; package org.apache.spark.sql.connector.expressions.aggregate;
import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.FieldReference;
/** /**
* An aggregate function that returns the summation of all the values in a group. * An aggregate function that returns the summation of all the values in a group.

View file

@ -18,7 +18,7 @@
package org.apache.spark.sql.connector.read; package org.apache.spark.sql.connector.read;
import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.Aggregation; import org.apache.spark.sql.connector.expressions.aggregate.Aggregation;
/** /**
* A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to

View file

@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.catalyst.util.truncatedString
import org.apache.spark.sql.connector.expressions.Aggregation import org.apache.spark.sql.connector.expressions.aggregate.Aggregation
import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource}
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}

View file

@ -40,7 +40,8 @@ import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2
import org.apache.spark.sql.connector.catalog.SupportsRead import org.apache.spark.sql.connector.catalog.SupportsRead
import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.catalog.TableCapability._
import org.apache.spark.sql.connector.expressions.{AggregateFunc, Count, CountStar, FieldReference, Max, Min, Sum} import org.apache.spark.sql.connector.expressions.FieldReference
import org.apache.spark.sql.connector.expressions.aggregate.{AggregateFunc, Count, CountStar, Max, Min, Sum}
import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.{InSubqueryExec, RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.{InSubqueryExec, RowDataSourceScanExec, SparkPlan}
import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.command._

View file

@ -25,7 +25,7 @@ import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskCon
import org.apache.spark.internal.Logging import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.connector.expressions.{AggregateFunc, Count, CountStar, Max, Min, Sum} import org.apache.spark.sql.connector.expressions.aggregate.{AggregateFunc, Count, CountStar, Max, Min, Sum}
import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects} import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects}
import org.apache.spark.sql.sources._ import org.apache.spark.sql.sources._
import org.apache.spark.sql.types._ import org.apache.spark.sql.types._

View file

@ -22,7 +22,8 @@ import scala.collection.mutable
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, NamedExpression, PredicateHelper, SchemaPruning} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, NamedExpression, PredicateHelper, SchemaPruning}
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.catalyst.util.CharVarcharUtils
import org.apache.spark.sql.connector.expressions.{Aggregation, FieldReference} import org.apache.spark.sql.connector.expressions.FieldReference
import org.apache.spark.sql.connector.expressions.aggregate.Aggregation
import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, SupportsPushDownRequiredColumns}
import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns}
import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.datasources.DataSourceStrategy

View file

@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.planning.ScanOperation import org.apache.spark.sql.catalyst.planning.ScanOperation
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LeafNode, LogicalPlan, Project} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LeafNode, LogicalPlan, Project}
import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.connector.expressions.Aggregation import org.apache.spark.sql.connector.expressions.aggregate.Aggregation
import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, V1Scan} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, V1Scan}
import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.sources import org.apache.spark.sql.sources

View file

@ -20,7 +20,7 @@ import scala.util.control.NonFatal
import org.apache.spark.internal.Logging import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.connector.expressions.Aggregation import org.apache.spark.sql.connector.expressions.aggregate.Aggregation
import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, SupportsPushDownRequiredColumns}
import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.execution.datasources.PartitioningUtils
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCRDD, JDBCRelation} import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCRDD, JDBCRelation}