spark-instrumented-optimizer/sql/core
Liang-Chi Hsieh 160c144baa [SPARK-30590][SQL] Untyped select API cannot take typed column expression 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>
2020-02-27 14:09:07 +08:00
..
benchmarks [SPARK-30843][SQL] Fix getting of time components before 1582 year 2020-02-17 13:59:21 +08:00
src [SPARK-30590][SQL] Untyped select API cannot take typed column expression that needs input type 2020-02-27 14:09:07 +08:00
v1.2/src [SPARK-30578][SQL][TEST] Explicitly set conf to use DSv2 for orc in OrcFilterSuite 2020-01-20 21:42:33 +08:00
v2.3/src [SPARK-30578][SQL][TEST] Explicitly set conf to use DSv2 for orc in OrcFilterSuite 2020-01-20 21:42:33 +08:00
pom.xml [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00