fqaiser94 commented on a change in pull request #27066:
URL: https://github.com/apache/spark/pull/27066#discussion_r448418602
##########
File path:
sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
##########
@@ -923,4 +923,501 @@ class ColumnExpressionSuite extends QueryTest with
SharedSparkSession {
val inSet = InSet(Literal("a"), Set("a", "b").map(UTF8String.fromString))
assert(inSet.sql === "('a' IN ('a', 'b'))")
}
+
+ def checkAnswerAndSchema(
+ df: => DataFrame,
+ expectedAnswer: Seq[Row],
+ expectedSchema: StructType): Unit = {
+
+ checkAnswer(df, expectedAnswer)
+ assert(df.schema == expectedSchema)
+ }
+
+ private lazy val structType = StructType(Seq(
+ StructField("a", IntegerType, nullable = false),
+ StructField("b", IntegerType, nullable = true),
+ StructField("c", IntegerType, nullable = false)))
+
+ private lazy val structLevel1: DataFrame = spark.createDataFrame(
+ sparkContext.parallelize(Row(Row(1, null, 3)) :: Nil),
+ StructType(Seq(StructField("a", structType, nullable = false))))
+
+ private lazy val nullStructLevel1: DataFrame = spark.createDataFrame(
+ sparkContext.parallelize(Row(null) :: Nil),
+ StructType(Seq(StructField("a", structType, nullable = true))))
+
+ private lazy val structLevel2: DataFrame = spark.createDataFrame(
+ sparkContext.parallelize(Row(Row(Row(1, null, 3))) :: Nil),
+ StructType(Seq(
+ StructField("a", StructType(Seq(
+ StructField("a", structType, nullable = false))),
+ nullable = false))))
+
+ private lazy val nullStructLevel2: DataFrame = spark.createDataFrame(
+ sparkContext.parallelize(Row(Row(null)) :: Nil),
+ StructType(Seq(
+ StructField("a", StructType(Seq(
+ StructField("a", structType, nullable = true))),
+ nullable = false))))
+
+ private lazy val structLevel3: DataFrame = spark.createDataFrame(
+ sparkContext.parallelize(Row(Row(Row(Row(1, null, 3)))) :: Nil),
+ StructType(Seq(
+ StructField("a", StructType(Seq(
+ StructField("a", StructType(Seq(
+ StructField("a", structType, nullable = false))),
+ nullable = false))),
+ nullable = false))))
+
+ test("withField should throw an exception if called on a non-StructType
column") {
+ intercept[AnalysisException] {
+ testData.withColumn("key", $"key".withField("a", lit(2)))
+ }.getMessage should include("struct argument should be struct type, got:
int")
+ }
+
+ test("withField should throw an exception if either fieldName or col
argument are null") {
+ an[java.lang.NullPointerException] should be thrownBy {
+ structLevel1.withColumn("a", $"a".withField(null, lit(2)))
+ }
+
+ an[java.lang.NullPointerException] should be thrownBy {
+ structLevel1.withColumn("a", $"a".withField("b", null))
+ }
+
+ an[java.lang.NullPointerException] should be thrownBy {
+ structLevel1.withColumn("a", $"a".withField(null, null))
+ }
+ }
+
+ test("withField should throw an exception if any intermediate structs don't
exist") {
+ intercept[AnalysisException] {
+ structLevel2.withColumn("a", 'a.withField("x.b", lit(2)))
+ }.getMessage should include("No such struct field x in a")
+
+ intercept[AnalysisException] {
+ structLevel3.withColumn("a", 'a.withField("a.x.b", lit(2)))
+ }.getMessage should include("No such struct field x in a")
+ }
+
+ test("withField should throw an exception if any intermediate field is not a
struct") {
+ intercept[AnalysisException] {
+ structLevel1.withColumn("a", 'a.withField("b.a", lit(2)))
+ }.getMessage should include("struct argument should be struct type, got:
int")
+
+ intercept[AnalysisException] {
+ val structLevel2: DataFrame = spark.createDataFrame(
+ sparkContext.parallelize(Row(Row(Row(1, null, 3), 4)) :: Nil),
+ StructType(Seq(
+ StructField("a", StructType(Seq(
+ StructField("a", structType, nullable = false),
+ StructField("a", IntegerType, nullable = false))),
+ nullable = false))))
+
+ structLevel2.withColumn("a", 'a.withField("a.b", lit(2)))
+ }.getMessage should include("Ambiguous reference to fields")
+ }
+
+ test("withField should add field with no name") {
Review comment:
I don't have a use case to support this behaviour.
The main reason I decided to allow this behaviour is to keep in line with
the behaviour of other similar functions.
For example, `Dataset.withColumn` allows you to create columns with empty
string as a name.
```
scala> Seq(1).toDF("col1").withColumn("", lit(2)).printSchema
root
|-- col1: integer (nullable = false)
|-- : integer (nullable = false)
```
Similarly, `org.apache.spark.sql.functions.struct` allows you to create a
struct with fields which have empty string as a name.
```
scala> Seq(1).toDF("col1").withColumn("col2", struct(lit(1).as("a"),
lit(2).as(""))).printSchema
root
|-- col1: integer (nullable = false)
|-- col2: struct (nullable = false)
| |-- a: integer (nullable = false)
| |-- : integer (nullable = false)
```
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]