Skip to content
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

Closed
wants to merge 13 commits into from

Conversation

viirya
Copy link
Member

@viirya viirya commented Feb 8, 2020

What changes were proposed in this pull request?

This patch proposes to throw clear analysis exception if untyped Dataset.select takes typed column expression that needs input type.

Why are the changes needed?

Dataset provides few typed select helper functions to select typed column expressions. The maximum number of typed columns supported is 5. If wanting to select more than 5 typed columns, it silently calls untyped Dataset.select and can causes weird unresolved error, like:

org.apache.spark.sql.AnalysisException: unresolved operator 'Aggregate [fooagg(FooAgg(1), None, None, None, input[0, int, false] AS value#114, assertnotnull(cast(value#114 as int)), input[0, int, false] AS value#113, IntegerType, IntegerType, false) AS foo_agg_1#116, fooagg(FooAgg(2), None, None, None, input[0, int, false] AS value#119, assertnotnull(cast(value#119 as int)), input[0, int, false] AS value#118, IntegerType, IntegerType, false) AS foo_agg_2#121, fooagg(FooAgg(3), None, None, None, input[0, int, false] AS value#124, assertnotnull(cast(value#124 as int)), input[0, int, false] AS value#123, IntegerType, IntegerType, false) AS foo_agg_3#126, fooagg(FooAgg(4), None, None, None, input[0, int, false] AS value#129, assertnotnull(cast(value#129 as int)), input[0, int, false] AS value#128, IntegerType, IntegerType, false) AS foo_agg_4#131, fooagg(FooAgg(5), None, None, None, input[0, int, false] AS value#134, assertnotnull(cast(value#134 as int)), input[0, int, false] AS value#133, IntegerType, IntegerType, false) AS foo_agg_5#136, fooagg(FooAgg(6), None, None, None, input[0, int, false] AS value#139, assertnotnull(cast(value#139 as int)), input[0, int, false] AS value#138, IntegerType, IntegerType, false) AS foo_agg_6#141];;
'Aggregate [fooagg(FooAgg(1), None, None, None, input[0, int, false] AS value#114, assertnotnull(cast(value#114 as int)), input[0, int, false] AS value#113, IntegerType, IntegerType, false) AS foo_agg_1#116, fooagg(FooAgg(2), None, None, None, input[0, int, false] AS value#119, assertnotnull(cast(value#119 as int)), input[0, int, false] AS value#118, IntegerType, IntegerType, false) AS foo_agg_2#121, fooagg(FooAgg(3), None, None, None, input[0, int, false] AS value#124, assertnotnull(cast(value#124 as int)), input[0, int, false] AS value#123, IntegerType, IntegerType, false) AS foo_agg_3#126, fooagg(FooAgg(4), None, None, None, input[0, int, false] AS value#129, assertnotnull(cast(value#129 as int)), input[0, int, false] AS value#128, IntegerType, IntegerType, false) AS foo_agg_4#131, fooagg(FooAgg(5), None, None, None, input[0, int, false] AS value#134, assertnotnull(cast(value#134 as int)), input[0, int, false] AS value#133, IntegerType, IntegerType, false) AS foo_agg_5#136, fooagg(FooAgg(6), None, None, None, input[0, int, false] AS value#139, assertnotnull(cast(value#139 as int)), input[0, int, false] AS value#138, IntegerType, IntegerType, false) AS foo_agg_6#141]
+- Project [_1#6 AS a#13, _2#7 AS b#14, _3#8 AS c#15, _4#9 AS d#16, _5#10 AS e#17, _6#11 AS F#18]
 +- LocalRelation [_1#6, _2#7, _3#8, _4#9, _5#10, _6#11]

at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:43)
 at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:95)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$3.apply(CheckAnalysis.scala:431)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$3.apply(CheckAnalysis.scala:430)
 at org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:127)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.checkAnalysis(CheckAnalysis.scala:430)

However, to fully disallow typed columns as input to untyped select API will break current usage like count that is a TypedColumn in functions. In order to keep compatibility, we should allow current usage of certain TypedColumns as input to untyped select API. For the TypedColumns that will cause unresolved exception, we should explicitly let users know that they are incorrectly calling untyped select with typed columns which need input type.

Does this PR introduce any user-facing change?

Yes, but this PR only refines the error message.

When users call Dataset.select API with typed column that needs input type, an analysis exception will be thrown. Previously an unresolved error will be thrown.

How was this patch tested?

Unit tests.

@SparkQA
Copy link

SparkQA commented Feb 8, 2020

Test build #118060 has finished for PR 27499 at commit 8aafa57.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya
Copy link
Member Author

viirya commented Feb 8, 2020

retest this please

@SparkQA
Copy link

SparkQA commented Feb 8, 2020

Test build #118062 has finished for PR 27499 at commit 8aafa57.

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

@SparkQA
Copy link

SparkQA commented Feb 8, 2020

Test build #118077 has finished for PR 27499 at commit ab7060e.

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

@@ -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))
Copy link
Member Author

Choose a reason for hiding this comment

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

This seems to me it is wrongly being a TypedColumn. Count is a DeclarativeAggregate.

Copy link
Contributor

@cloud-fan cloud-fan Feb 21, 2020

Choose a reason for hiding this comment

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

Copy link
Member

Choose a reason for hiding this comment

The 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 ..

Copy link
Member Author

Choose a reason for hiding this comment

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

Ok. :)

ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStartedEvent.this"),

// [SPARK-30590][SQL] Untyped select API cannot take typed column expression
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.functions.count")
Copy link
Member Author

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.

@SparkQA
Copy link

SparkQA commented Feb 9, 2020

Test build #118080 has finished for PR 27499 at commit c9d3cd3.

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

@viirya
Copy link
Member Author

viirya commented Feb 9, 2020

@SparkQA
Copy link

SparkQA commented Feb 11, 2020

Test build #118189 has finished for PR 27499 at commit b784ba5.

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

@viirya
Copy link
Member Author

viirya commented Feb 11, 2020

retest this please

@SparkQA
Copy link

SparkQA commented Feb 11, 2020

Test build #118196 has finished for PR 27499 at commit b784ba5.

  • This patch fails build dependency tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya
Copy link
Member Author

viirya commented Feb 11, 2020

retest this please

@SparkQA
Copy link

SparkQA commented Feb 11, 2020

Test build #118197 has finished for PR 27499 at commit b784ba5.

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

@viirya
Copy link
Member Author

viirya commented Feb 11, 2020

retest this please.

@SparkQA
Copy link

SparkQA commented Feb 11, 2020

Test build #118209 has finished for PR 27499 at commit b784ba5.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

Seems fine except #27499 (comment). Might need to update title and PR description too.

@SparkQA
Copy link

SparkQA commented Feb 21, 2020

Test build #118777 has finished for PR 27499 at commit 45feb5c.

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

@viirya
Copy link
Member Author

viirya commented Feb 21, 2020

Updated the description. I think the title is still ok?

@SparkQA
Copy link

SparkQA commented Feb 21, 2020

Test build #118800 has finished for PR 27499 at commit 53ba69c.

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

@SparkQA
Copy link

SparkQA commented Feb 22, 2020

Test build #118813 has finished for PR 27499 at commit 096ce42.

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

@viirya
Copy link
Member Author

viirya commented Feb 22, 2020

also cc @dongjoon-hyun

}
if (isSimpleEncoder) {
// This typed column produces simple type output that can be fit into untyped `DataFrame`.
typedCol.withInputType(exprEnc, logicalPlan.output)
Copy link
Contributor

Choose a reason for hiding this comment

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

Previously we didn't call withInputType for count, right?

Copy link
Member Author

Choose a reason for hiding this comment

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

count has no TypedAggregateExpression. withInputType only works on TypedAggregateExpression.

Copy link
Contributor

Choose a reason for hiding this comment

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

I mean, df.select(count("*")) works without calling withInputType, right?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yes, for TypedColumn that doesn't contain TypedAggregateExpression, withInputType is no-op, so you don't need to call withInputType for df.select(count("*")).

Copy link
Contributor

Choose a reason for hiding this comment

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

So here we are supporting more cases than before?

Copy link
Member Author

Choose a reason for hiding this comment

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

Oh, I get your point now. Yea, we should not allow more cases than before.

@viirya viirya changed the title [SPARK-30590][SQL] Untyped select API cannot take typed column expression [SPARK-30590][SQL] Untyped select API cannot take typed column expression without input type Feb 25, 2020
@viirya viirya changed the title [SPARK-30590][SQL] Untyped select API cannot take typed column expression without input type [SPARK-30590][SQL] Untyped select API cannot take typed column expression that needs input type Feb 25, 2020
@SparkQA
Copy link

SparkQA commented Feb 25, 2020

Test build #118897 has finished for PR 27499 at commit 83958fb.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya
Copy link
Member Author

viirya commented Feb 25, 2020

retest this please.

*/
private[sql] def needInputType: Boolean = {
expr.find {
case ta: TypedAggregateExpression if ta.inputDeserializer.isEmpty => true
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: case ta: TypedAggregateExpression => ta.inputDeserializer.isEmpty

Copy link
Member Author

Choose a reason for hiding this comment

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

ok.

@cloud-fan
Copy link
Contributor

LGTM, let's highlight that it only refines the error message in the Does this PR introduce any user-facing change? section.


// Passes typed columns to untyped `Dataset.select` API.
val err = intercept[AnalysisException] {
df.select(fooAgg(1), fooAgg(2), fooAgg(3), fooAgg(4), fooAgg(5), fooAgg(6))
Copy link
Contributor

Choose a reason for hiding this comment

The 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 select, and typed count is supported in both typed and untyped select. That said, if we add a 6th overload of typed select, it can break queries that call the untyped select with 6 typed counts.

I'm not sure what's the best way to move forward. Maybe we should add new methods typedSelect to disambiguate the untyped version.

Copy link
Member Author

Choose a reason for hiding this comment

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

Yea, to be clear, if we add a 6th overload of typed select, a call to the untyped select with 6 typed count could return Dataset[(Long, Long, ...)] instead of DataFrame.

I think you meant something like existing selectUntyped? Although its naming is confusing.

@SparkQA
Copy link

SparkQA commented Feb 25, 2020

Test build #118909 has finished for PR 27499 at commit 83958fb.

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

@SparkQA
Copy link

SparkQA commented Feb 26, 2020

Test build #118931 has finished for PR 27499 at commit 68d17f7.

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

@viirya
Copy link
Member Author

viirya commented Feb 26, 2020

retest this please

@SparkQA
Copy link

SparkQA commented Feb 26, 2020

Test build #118944 has finished for PR 27499 at commit 68d17f7.

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

Project(cols.map(_.named), logicalPlan)
val untypedCols = cols.map {
case typedCol: TypedColumn[_, _] =>
if (!typedCol.needInputType) {
Copy link
Contributor

@cloud-fan cloud-fan Feb 26, 2020

Choose a reason for hiding this comment

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

just noticed: why don't we inline this method? Then we can centralize the changes here. The methods in TypedColumn can still be accessed by java users who ignore "private[spark]", so better to avoid adding if we can.

Copy link
Member Author

Choose a reason for hiding this comment

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

Oh ok. I previously don't want to make select looks complicated. Inlined it now.

@SparkQA
Copy link

SparkQA commented Feb 26, 2020

Test build #118984 has finished for PR 27499 at commit 7d045fb.

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

@cloud-fan
Copy link
Contributor

thanks, merging to master/3.0!

@cloud-fan cloud-fan closed this in 160c144 Feb 27, 2020
@viirya
Copy link
Member Author

viirya commented Feb 27, 2020

Thanks! I will open a JIRA for discussion of typed select API.

@HyukjinKwon
Copy link
Member

+1 LGTM too

cloud-fan pushed a commit that referenced this pull request Feb 27, 2020
…sion that needs input type

### What changes were proposed in this pull request?

This patch proposes to throw clear analysis exception if untyped `Dataset.select` takes typed column expression that needs input type.

### Why are the changes needed?

`Dataset` provides few typed `select` helper functions to select typed column expressions. The maximum number of typed columns supported is 5. If wanting to select more than 5 typed columns, it silently calls untyped `Dataset.select` and can causes weird unresolved error, like:

```
org.apache.spark.sql.AnalysisException: unresolved operator 'Aggregate [fooagg(FooAgg(1), None, None, None, input[0, int, false] AS value#114, assertnotnull(cast(value#114 as int)), input[0, int, false] AS value#113, IntegerType, IntegerType, false) AS foo_agg_1#116, fooagg(FooAgg(2), None, None, None, input[0, int, false] AS value#119, assertnotnull(cast(value#119 as int)), input[0, int, false] AS value#118, IntegerType, IntegerType, false) AS foo_agg_2#121, fooagg(FooAgg(3), None, None, None, input[0, int, false] AS value#124, assertnotnull(cast(value#124 as int)), input[0, int, false] AS value#123, IntegerType, IntegerType, false) AS foo_agg_3#126, fooagg(FooAgg(4), None, None, None, input[0, int, false] AS value#129, assertnotnull(cast(value#129 as int)), input[0, int, false] AS value#128, IntegerType, IntegerType, false) AS foo_agg_4#131, fooagg(FooAgg(5), None, None, None, input[0, int, false] AS value#134, assertnotnull(cast(value#134 as int)), input[0, int, false] AS value#133, IntegerType, IntegerType, false) AS foo_agg_5#136, fooagg(FooAgg(6), None, None, None, input[0, int, false] AS value#139, assertnotnull(cast(value#139 as int)), input[0, int, false] AS value#138, IntegerType, IntegerType, false) AS foo_agg_6#141];;
'Aggregate [fooagg(FooAgg(1), None, None, None, input[0, int, false] AS value#114, assertnotnull(cast(value#114 as int)), input[0, int, false] AS value#113, IntegerType, IntegerType, false) AS foo_agg_1#116, fooagg(FooAgg(2), None, None, None, input[0, int, false] AS value#119, assertnotnull(cast(value#119 as int)), input[0, int, false] AS value#118, IntegerType, IntegerType, false) AS foo_agg_2#121, fooagg(FooAgg(3), None, None, None, input[0, int, false] AS value#124, assertnotnull(cast(value#124 as int)), input[0, int, false] AS value#123, IntegerType, IntegerType, false) AS foo_agg_3#126, fooagg(FooAgg(4), None, None, None, input[0, int, false] AS value#129, assertnotnull(cast(value#129 as int)), input[0, int, false] AS value#128, IntegerType, IntegerType, false) AS foo_agg_4#131, fooagg(FooAgg(5), None, None, None, input[0, int, false] AS value#134, assertnotnull(cast(value#134 as int)), input[0, int, false] AS value#133, IntegerType, IntegerType, false) AS foo_agg_5#136, fooagg(FooAgg(6), None, None, None, input[0, int, false] AS value#139, assertnotnull(cast(value#139 as int)), input[0, int, false] AS value#138, IntegerType, IntegerType, false) AS foo_agg_6#141]
+- Project [_1#6 AS a#13, _2#7 AS b#14, _3#8 AS c#15, _4#9 AS d#16, _5#10 AS e#17, _6#11 AS F#18]
 +- LocalRelation [_1#6, _2#7, _3#8, _4#9, _5#10, _6#11]

at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:43)
 at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:95)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$3.apply(CheckAnalysis.scala:431)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$3.apply(CheckAnalysis.scala:430)
 at org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:127)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.checkAnalysis(CheckAnalysis.scala:430)
```

However, to fully disallow typed columns as input to untyped `select` API will break current usage like `count` that is a `TypedColumn` in `functions`. In order to keep compatibility, we should allow current usage of certain `TypedColumn`s as input to untyped `select` API. For the `TypedColumn`s that will cause unresolved exception, we should explicitly let users know that they are incorrectly calling untyped `select` with typed columns which need input type.

### Does this PR introduce any user-facing change?

Yes, but this PR only refines the error message.

When users call `Dataset.select` API with typed column that needs input type, an analysis exception will be thrown. Previously an unresolved error will be thrown.

### How was this patch tested?

Unit tests.

Closes #27499 from viirya/SPARK-30590.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit 160c144)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
@viirya
Copy link
Member Author

viirya commented Feb 28, 2020

Created SPARK-30983 for discussion of typed select API.

sjincho pushed a commit to sjincho/spark that referenced this pull request Apr 15, 2020
…sion that needs input type

### What changes were proposed in this pull request?

This patch proposes to throw clear analysis exception if untyped `Dataset.select` takes typed column expression that needs input type.

### Why are the changes needed?

`Dataset` provides few typed `select` helper functions to select typed column expressions. The maximum number of typed columns supported is 5. If wanting to select more than 5 typed columns, it silently calls untyped `Dataset.select` and can causes weird unresolved error, like:

```
org.apache.spark.sql.AnalysisException: unresolved operator 'Aggregate [fooagg(FooAgg(1), None, None, None, input[0, int, false] AS value#114, assertnotnull(cast(value#114 as int)), input[0, int, false] AS value#113, IntegerType, IntegerType, false) AS foo_agg_1#116, fooagg(FooAgg(2), None, None, None, input[0, int, false] AS value#119, assertnotnull(cast(value#119 as int)), input[0, int, false] AS value#118, IntegerType, IntegerType, false) AS foo_agg_2#121, fooagg(FooAgg(3), None, None, None, input[0, int, false] AS value#124, assertnotnull(cast(value#124 as int)), input[0, int, false] AS value#123, IntegerType, IntegerType, false) AS foo_agg_3#126, fooagg(FooAgg(4), None, None, None, input[0, int, false] AS value#129, assertnotnull(cast(value#129 as int)), input[0, int, false] AS value#128, IntegerType, IntegerType, false) AS foo_agg_4#131, fooagg(FooAgg(5), None, None, None, input[0, int, false] AS value#134, assertnotnull(cast(value#134 as int)), input[0, int, false] AS value#133, IntegerType, IntegerType, false) AS foo_agg_5#136, fooagg(FooAgg(6), None, None, None, input[0, int, false] AS value#139, assertnotnull(cast(value#139 as int)), input[0, int, false] AS value#138, IntegerType, IntegerType, false) AS foo_agg_6#141];;
'Aggregate [fooagg(FooAgg(1), None, None, None, input[0, int, false] AS value#114, assertnotnull(cast(value#114 as int)), input[0, int, false] AS value#113, IntegerType, IntegerType, false) AS foo_agg_1#116, fooagg(FooAgg(2), None, None, None, input[0, int, false] AS value#119, assertnotnull(cast(value#119 as int)), input[0, int, false] AS value#118, IntegerType, IntegerType, false) AS foo_agg_2#121, fooagg(FooAgg(3), None, None, None, input[0, int, false] AS value#124, assertnotnull(cast(value#124 as int)), input[0, int, false] AS value#123, IntegerType, IntegerType, false) AS foo_agg_3#126, fooagg(FooAgg(4), None, None, None, input[0, int, false] AS value#129, assertnotnull(cast(value#129 as int)), input[0, int, false] AS value#128, IntegerType, IntegerType, false) AS foo_agg_4#131, fooagg(FooAgg(5), None, None, None, input[0, int, false] AS value#134, assertnotnull(cast(value#134 as int)), input[0, int, false] AS value#133, IntegerType, IntegerType, false) AS foo_agg_5#136, fooagg(FooAgg(6), None, None, None, input[0, int, false] AS value#139, assertnotnull(cast(value#139 as int)), input[0, int, false] AS value#138, IntegerType, IntegerType, false) AS foo_agg_6#141]
+- Project [_1#6 AS a#13, _2#7 AS b#14, _3#8 AS c#15, _4#9 AS d#16, _5#10 AS e#17, _6#11 AS F#18]
 +- LocalRelation [_1#6, _2#7, _3#8, _4#9, _5#10, _6#11]

at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:43)
 at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:95)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$3.apply(CheckAnalysis.scala:431)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$3.apply(CheckAnalysis.scala:430)
 at org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:127)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.checkAnalysis(CheckAnalysis.scala:430)
```

However, to fully disallow typed columns as input to untyped `select` API will break current usage like `count` that is a `TypedColumn` in `functions`. In order to keep compatibility, we should allow current usage of certain `TypedColumn`s as input to untyped `select` API. For the `TypedColumn`s that will cause unresolved exception, we should explicitly let users know that they are incorrectly calling untyped `select` with typed columns which need input type.

### Does this PR introduce any user-facing change?

Yes, but this PR only refines the error message.

When users call `Dataset.select` API with typed column that needs input type, an analysis exception will be thrown. Previously an unresolved error will be thrown.

### How was this patch tested?

Unit tests.

Closes apache#27499 from viirya/SPARK-30590.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
@viirya viirya deleted the SPARK-30590 branch December 27, 2023 18:38
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
5 participants