New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-30590][SQL] Untyped select API cannot take typed column expression that needs input type #27499
[SPARK-30590][SQL] Untyped select API cannot take typed column expression that needs input type #27499
Changes from 3 commits
8aafa57
ab7060e
c9d3cd3
b784ba5
4103413
45feb5c
53ba69c
096ce42
83958fb
0ef18e8
68d17f7
c50a357
7d045fb
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -352,8 +352,7 @@ object functions { | |
* @group agg_funcs | ||
* @since 1.3.0 | ||
*/ | ||
def count(columnName: String): TypedColumn[Any, Long] = | ||
count(Column(columnName)).as(ExpressionEncoder[Long]()) | ||
def count(columnName: String): Column = count(Column(columnName)) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This seems to me it is wrongly being a There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is a breaking change, right? At least https://github.com/apache/spark/pull/27499/files#diff-2c67e6ae3d5115b5521681f6ef871b1dR43 is broken. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It seems a right change but let's revert this line considering it's code freeze period .. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ok. :) |
||
|
||
/** | ||
* Aggregate function: returns the number of distinct items in a group. | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -219,6 +219,15 @@ case class OptionBooleanIntAggregator(colName: String) | |
def OptionalBoolIntEncoder: Encoder[Option[(Boolean, Int)]] = ExpressionEncoder() | ||
} | ||
|
||
case class FooAgg(s: Int) extends Aggregator[Row, Int, Int] { | ||
def zero: Int = s | ||
def reduce(b: Int, r: Row): Int = b + r.getAs[Int](0) | ||
def merge(b1: Int, b2: Int): Int = b1 + b2 | ||
def finish(b: Int): Int = b | ||
def bufferEncoder: Encoder[Int] = Encoders.scalaInt | ||
def outputEncoder: Encoder[Int] = Encoders.scalaInt | ||
} | ||
|
||
class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { | ||
import testImplicits._ | ||
|
||
|
@@ -394,4 +403,21 @@ class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { | |
checkAnswer(group, Row("bob", Row(true, 3)) :: Nil) | ||
checkDataset(group.as[OptionBooleanIntData], OptionBooleanIntData("bob", Some((true, 3)))) | ||
} | ||
|
||
test("SPARK-30590: select multiple typed column expressions") { | ||
val df = Seq((1, 2, 3, 4, 5, 6)).toDF("a", "b", "c", "d", "e", "f") | ||
val fooAgg = (i: Int) => FooAgg(i).toColumn.name(s"foo_agg_$i") | ||
|
||
val agg1 = df.select(fooAgg(1), fooAgg(2), fooAgg(3), fooAgg(4), fooAgg(5)) | ||
checkDataset(agg1, (3, 5, 7, 9, 11)) | ||
|
||
val agg2 = df.selectUntyped(fooAgg(1), fooAgg(2), fooAgg(3), fooAgg(4), fooAgg(5), fooAgg(6)) | ||
.asInstanceOf[Dataset[(Int, Int, Int, Int, Int, Int)]] | ||
HyukjinKwon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
checkDataset(agg2, (3, 5, 7, 9, 11, 13)) | ||
|
||
val err = intercept[AnalysisException] { | ||
df.select(fooAgg(1), fooAgg(2), fooAgg(3), fooAgg(4), fooAgg(5), fooAgg(6)) | ||
HyukjinKwon marked this conversation as resolved.
Show resolved
Hide resolved
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not related to this PR, just a note: We have 5 overloads of typed I'm not sure what's the best way to move forward. Maybe we should add new methods There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yea, to be clear, if we add a 6th overload of typed I think you meant something like existing |
||
}.getMessage | ||
assert(err.contains("a typed column that cannot be passed in untyped `select` API")) | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Put it under 3.0 exclude rules temporarily. The version number in the master branch is still 3.0.0.