Object

org.apache.spark.sql.catalyst.optimizer

RewriteDistinctAggregates

Related Doc: package optimizer

Permalink

object RewriteDistinctAggregates extends Rule[LogicalPlan]

This rule rewrites an aggregate query with distinct aggregations into an expanded double aggregation in which the regular aggregation expressions and every distinct clause is aggregated in a separate group. The results are then combined in a second aggregate.

For example (in scala):

val data = Seq(
  ("a", "ca1", "cb1", 10),
  ("a", "ca1", "cb2", 5),
  ("b", "ca1", "cb1", 13))
  .toDF("key", "cat1", "cat2", "value")
data.createOrReplaceTempView("data")

val agg = data.groupBy($"key")
  .agg(
    countDistinct($"cat1").as("cat1_cnt"),
    countDistinct($"cat2").as("cat2_cnt"),
    sum($"value").as("total"))

This translates to the following (pseudo) logical plan:

Aggregate(
   key = ['key]
   functions = [COUNT(DISTINCT 'cat1),
                COUNT(DISTINCT 'cat2),
                sum('value)]
   output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
  LocalTableScan [...]

This rule rewrites this logical plan to the following (pseudo) logical plan:

Aggregate(
   key = ['key]
   functions = [count(if (('gid = 1)) 'cat1 else null),
                count(if (('gid = 2)) 'cat2 else null),
                first(if (('gid = 0)) 'total else null) ignore nulls]
   output = ['key, 'cat1_cnt, 'cat2_cnt, 'total])
  Aggregate(
     key = ['key, 'cat1, 'cat2, 'gid]
     functions = [sum('value)]
     output = ['key, 'cat1, 'cat2, 'gid, 'total])
    Expand(
       projections = [('key, null, null, 0, cast('value as bigint)),
                      ('key, 'cat1, null, 1, null),
                      ('key, null, 'cat2, 2, null)]
       output = ['key, 'cat1, 'cat2, 'gid, 'value])
      LocalTableScan [...]

The rule does the following things here: 1. Expand the data. There are three aggregation groups in this query:

  1. the non-distinct group; ii. the distinct 'cat1 group; iii. the distinct 'cat2 group. An expand operator is inserted to expand the child data for each group. The expand will null out all unused columns for the given group; this must be done in order to ensure correctness later on. Groups can by identified by a group id (gid) column added by the expand operator. 2. De-duplicate the distinct paths and aggregate the non-aggregate path. The group by clause of this aggregate consists of the original group by clause, all the requested distinct columns and the group id. Both de-duplication of distinct column and the aggregation of the non-distinct group take advantage of the fact that we group by the group id (gid) and that we have nulled out all non-relevant columns the given group. 3. Aggregating the distinct groups and combining this with the results of the non-distinct aggregation. In this step we use the group id to filter the inputs for the aggregate functions. The result of the non-distinct group are 'aggregated' by using the first operator, it might be more elegant to use the native UDAF merge mechanism for this in the future.

This rule duplicates the input data by two or more times (# distinct groups + an optional non-distinct group). This will put quite a bit of memory pressure of the used aggregate and exchange operators. Keeping the number of distinct groups as low a possible should be priority, we could improve this in the current rule by applying more advanced expression canonicalization techniques.

Linear Supertypes
Rule[LogicalPlan], Logging, AnyRef, Any
Ordering
  1. Alphabetic
  2. By Inheritance
Inherited
  1. RewriteDistinctAggregates
  2. Rule
  3. Logging
  4. AnyRef
  5. Any
  1. Hide All
  2. Show All
Visibility
  1. Public
  2. All

Value Members

  1. final def !=(arg0: Any): Boolean

    Permalink
    Definition Classes
    AnyRef → Any
  2. final def ##(): Int

    Permalink
    Definition Classes
    AnyRef → Any
  3. final def ==(arg0: Any): Boolean

    Permalink
    Definition Classes
    AnyRef → Any
  4. def apply(plan: LogicalPlan): LogicalPlan

    Permalink
    Definition Classes
    RewriteDistinctAggregatesRule
  5. final def asInstanceOf[T0]: T0

    Permalink
    Definition Classes
    Any
  6. def clone(): AnyRef

    Permalink
    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  7. final def eq(arg0: AnyRef): Boolean

    Permalink
    Definition Classes
    AnyRef
  8. def equals(arg0: Any): Boolean

    Permalink
    Definition Classes
    AnyRef → Any
  9. def finalize(): Unit

    Permalink
    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( classOf[java.lang.Throwable] )
  10. final def getClass(): Class[_]

    Permalink
    Definition Classes
    AnyRef → Any
  11. def hashCode(): Int

    Permalink
    Definition Classes
    AnyRef → Any
  12. def initializeLogIfNecessary(isInterpreter: Boolean): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  13. final def isInstanceOf[T0]: Boolean

    Permalink
    Definition Classes
    Any
  14. def isTraceEnabled(): Boolean

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  15. def log: Logger

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  16. def logDebug(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  17. def logDebug(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  18. def logError(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  19. def logError(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  20. def logInfo(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  21. def logInfo(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  22. def logName: String

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  23. def logTrace(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  24. def logTrace(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  25. def logWarning(msg: ⇒ String, throwable: Throwable): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  26. def logWarning(msg: ⇒ String): Unit

    Permalink
    Attributes
    protected
    Definition Classes
    Logging
  27. final def ne(arg0: AnyRef): Boolean

    Permalink
    Definition Classes
    AnyRef
  28. final def notify(): Unit

    Permalink
    Definition Classes
    AnyRef
  29. final def notifyAll(): Unit

    Permalink
    Definition Classes
    AnyRef
  30. def rewrite(a: Aggregate): Aggregate

    Permalink
  31. val ruleName: String

    Permalink

    Name for this rule, automatically inferred based on class name.

    Name for this rule, automatically inferred based on class name.

    Definition Classes
    Rule
  32. final def synchronized[T0](arg0: ⇒ T0): T0

    Permalink
    Definition Classes
    AnyRef
  33. def toString(): String

    Permalink
    Definition Classes
    AnyRef → Any
  34. final def wait(): Unit

    Permalink
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  35. final def wait(arg0: Long, arg1: Int): Unit

    Permalink
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  36. final def wait(arg0: Long): Unit

    Permalink
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )

Inherited from Rule[LogicalPlan]

Inherited from Logging

Inherited from AnyRef

Inherited from Any

Ungrouped