Skip to content

Conversation

@maryannxue
Copy link
Contributor

@maryannxue maryannxue commented Jul 30, 2018

What changes were proposed in this pull request?

When the pivot column is of a complex type, the eval() result will be an UnsafeRow, while the keys of the HashMap for column value matching is a GenericInternalRow. As a result, there will be no match and the result will always be empty.
So for a pivot column of complex-types, we should:

  1. If the complex-type is not comparable (orderable), throw an Exception. It cannot be a pivot column.
  2. Otherwise, if it goes through the PivotFirst code path, PivotFirst should use a TreeMap instead of HashMap for such columns.

This PR has also reverted the walk-around in Analyzer that had been introduced to avoid this PivotFirst issue.

How was this patch tested?

Added UT.

@holdensmagicalunicorn
Copy link

@maryannxue, thanks! I am a bot who has found some folks who might be able to help with the review:@gatorsmile, @yhuai and @marmbrus

// alternate plan that instead uses two steps of aggregation.
val namedAggExps: Seq[NamedExpression] = aggregates.map(a => Alias(a, a.sql)())
val bigGroup = groupByExprs ++ pivotColumn.references
val namedPivotCol = pivotColumn match {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is to revert the original walk-around aimed to avoid the PivotFirst issue. Now that we have PivotFirst working alright for complex types, we can revert it.

@SparkQA
Copy link

SparkQA commented Jul 31, 2018

Test build #93815 has finished for PR 21926 at commit b41a45c.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@maryannxue
Copy link
Contributor Author

retest this please

@SparkQA
Copy link

SparkQA commented Jul 31, 2018

Test build #93818 has finished for PR 21926 at commit b41a45c.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

case Pivot(groupByExprsOpt, pivotColumn, pivotValues, aggregates, child) =>
if (!RowOrdering.isOrderable(pivotColumn.dataType)) {
throw new AnalysisException(
s"Invalid pivot column '${pivotColumn}'. Pivot columns must be comparable.")
Copy link
Member

@gatorsmile gatorsmile Jul 31, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To the other reviewers: this is consistent with the requirements of group-by columns.

@gatorsmile
Copy link
Member

LGTM Thanks! Merged to master.

@asfgit asfgit closed this in b4fd75f Jul 31, 2018
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants