You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/02/10 16:38:43 UTC

[GitHub] [spark] amandeep-sharma opened a new pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

amandeep-sharma opened a new pull request #31545:
URL: https://github.com/apache/spark/pull/31545


   [SPARK-34417] Fixed dataframe.na.fillMap() for column having a dot in name 
   
   Solution - Use resolved column name for comparison in ```case (k, _) => columnEquals(k, f.name)```
   
   Added unit test for the same
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788752800






----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-783531368


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39930/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788522629


   **[Test build #135618 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135618/testReport)** for PR 31545 at commit [`896652c`](https://github.com/apache/spark/commit/896652c34aaf8ecc9bcaaa4f81ccb149dd0c4a76).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574248882



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,12 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       No, it does fail. Since column is successfully resolved at ```df.resolve(colName)```, it will not go to 424.
   Only for the data frame columns not present in the na-fill-map, it will go to line 424.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787054860


   **[Test build #135540 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135540/testReport)** for PR 31545 at commit [`5ee8bee`](https://github.com/apache/spark/commit/5ee8beed8ca0fec367ef3c4c3e34b58419def47b).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574257299



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,12 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       @imback82 i think your point is that if data frame has a column having dot in the name, but it is not part of the null fill map then it will fail.
   Yes, it will fail, example below.
   ```import org.apache.spark.sql.SparkSession
   
   object ColumnNameWithDot {
   
     def main(args: Array[String]): Unit = {
   
       val spark = SparkSession.builder.appName("Simple Application")
         .config("spark.master", "local").getOrCreate()
   
       spark.sparkContext.setLogLevel("OFF")
   
       import spark.implicits._
       val df = Seq(("abc", 23), ("def", 44), (null, 0)).toDF("ColWith.Dot", "Col.2")
       df.na.fill(Map("`ColWith.Dot`" -> "na"))
         .show()
     }
   }




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788657417


   **[Test build #135618 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135618/testReport)** for PR 31545 at commit [`896652c`](https://github.com/apache/spark/commit/896652c34aaf8ecc9bcaaa4f81ccb149dd0c4a76).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781906014


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39840/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584716356



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,42 +395,42 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val attrToValue = AttributeMap(values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
-
+      val attr = df.resolve(colName) match {
+        case a: Attribute => a
+        case _ => throw new IllegalArgumentException("Nested field is not supported.")

Review comment:
       think about it again, probably `UnsupportedException` is better.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-783505966


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39930/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-783646664


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135349/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787066257


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40121/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r578925797



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -394,10 +395,11 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
   }
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
+    val resolved = mutable.Map[String, Any]()

Review comment:
       Done.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584935658



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,42 +395,42 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val attrToValue = AttributeMap(values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
-
+      val attr = df.resolve(colName) match {
+        case a: Attribute => a
+        case _ => throw new IllegalArgumentException("Nested field is not supported.")

Review comment:
       Would be nicer if we can add the nested field name to the error message.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781876771


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135260/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584049545



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       Above mentioned approach will not work if the null-fill map has qualified column name(dfAlias.columnName) as below.
   Eg: 
   `def payment = spark.read.format("csv").option("header", "true")
         .schema(paymentSchema)
         .load("Payment.csv").as("payment")
   
       payment.na.fill(Map("payment.`Customer.Id`" -> -1))
         .show()
   `
   Also, column name quoted with back-tick(`) will also not work.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574255693



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,22 +395,24 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
+    var resolved: Map[String, Any] = Map()
     values.foreach { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)
 
       // Check data type
       replaceValue match {
         case _: jl.Double | _: jl.Float | _: jl.Integer | _: jl.Long | _: jl.Boolean | _: String =>
-          // This is good
+        // This is good

Review comment:
       Taken care of.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574247925



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,22 +395,24 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
+    var resolved: Map[String, Any] = Map()
     values.foreach { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)
 
       // Check data type
       replaceValue match {
         case _: jl.Double | _: jl.Float | _: jl.Integer | _: jl.Long | _: jl.Boolean | _: String =>
-          // This is good
+        // This is good

Review comment:
       Thanks for catching, I shall fix it and also add comments for using additional map.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574257299



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,12 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       @imback82 i think your point is that if data frame has a column having dot in the name, but it is not part of the null fill map then it will fail.
   Yes, it will fail, example below. It will need fix in the catalyst layer.
   ```import org.apache.spark.sql.SparkSession
   
   object ColumnNameWithDot {
   
     def main(args: Array[String]): Unit = {
   
       val spark = SparkSession.builder.appName("Simple Application")
         .config("spark.master", "local").getOrCreate()
   
       spark.sparkContext.setLogLevel("OFF")
   
       import spark.implicits._
       val df = Seq(("abc", 23), ("def", 44), (null, 0)).toDF("ColWith.Dot", "Col.2")
       df.na.fill(Map("`ColWith.Dot`" -> "na"))
         .show()
     }
   }




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584049545



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       Above mentioned approach will not work if the null-fill map has qualified column name as below.
   Eg: 
   `def payment = spark.read.format("csv").option("header", "true")
         .schema(paymentSchema)
         .load("Payment.csv").as("payment")
   
       payment.na.fill(Map("payment.`Customer.Id`" -> -1))
         .show() `
   Also, column name quoted with back-tick(`) will also not work.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-783467571


   **[Test build #135349 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135349/testReport)** for PR 31545 at commit [`24302b9`](https://github.com/apache/spark/commit/24302b9b7e1bb5568c7918369599a43123912f88).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781335758


   **[Test build #135228 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135228/testReport)** for PR 31545 at commit [`20b8db7`](https://github.com/apache/spark/commit/20b8db78d657eec066265395fd53465f66f8a220).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781832030


   **[Test build #135257 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135257/testReport)** for PR 31545 at commit [`daf7c8a`](https://github.com/apache/spark/commit/daf7c8a97aa10721cdcb181059b6cf1c2134a295).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787956326


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40177/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r578717656



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -394,10 +395,11 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
   }
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
+    val resolved = mutable.Map[String, Any]()

Review comment:
       instead of having mutable map, you can just do the following:
   ```scala
   val resolved = values.map { case (colName, replaceValue) =>
     ...
     ...
     (resolvedColumn.name, replaceValue)
   }  
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788757957


   @cloud-fan sure, i will create JIRA and work on it.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781530445


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135228/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787054860


   **[Test build #135540 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135540/testReport)** for PR 31545 at commit [`5ee8bee`](https://github.com/apache/spark/commit/5ee8beed8ca0fec367ef3c4c3e34b58419def47b).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787091050


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135540/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787899512


   **[Test build #135596 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135596/testReport)** for PR 31545 at commit [`d0cbbe4`](https://github.com/apache/spark/commit/d0cbbe46cbf8ffbb12eb5bb2feb55d3a549a196c).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781876771


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135260/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781829338


   **[Test build #135257 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135257/testReport)** for PR 31545 at commit [`daf7c8a`](https://github.com/apache/spark/commit/daf7c8a97aa10721cdcb181059b6cf1c2134a295).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787091050


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135540/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787062597


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40121/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787047885


   **[Test build #135539 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135539/testReport)** for PR 31545 at commit [`f49c742`](https://github.com/apache/spark/commit/f49c742d53f2c08ce49e11f070a142457776c04b).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787958293


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40177/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781297626


   ok to test


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584049545



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       Above mentioned approach will not work if the null-fill map has qualified column name(dfAlias.columnName) as below.
   Eg: 
   
   _def payment = spark.read.format("csv").option("header", "true")
         .schema(paymentSchema)
         .load("Payment.csv").as("payment")
       payment.na.fill(Map("payment.`Customer.Id`" -> -1))
         .show()_
   
   Also, column name quoted with back-tick(`) will also not work.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       Above mentioned approach will not work if the null-fill map has qualified column name(dfAlias.columnName) as below.
   Eg: 
   
   _def payment = spark.read.format("csv").option("header", "true")
         .schema(paymentSchema)
         .load("Payment.csv").as("payment")
   
       payment.na.fill(Map("payment.`Customer.Id`" -> -1))
         .show()_
   
   Also, column name quoted with back-tick(`) will also not work.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya edited a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
viirya edited a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788150408


   I updated the PR title/description a bit to fit the template more. Feel free to edit it.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781362843


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39809/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
imback82 commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-777185887


   cc @cloud-fan @viirya 


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r580341615



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       Since the input string is the plain column name, we should call neither `df.resolve` nor `df.col`. We should follow `Dataset.drop/withColumn` to get the top-level attribute directly:
   ```
   val columnEquals = df.sparkSession.sessionState.analyzer.resolver
   val output = df.queryExecution.analyzed.output
   val attrToValue = AttributeMap(values.map { case (colName, replaceValue) =>
     output.find(a => columnEquals(a.name, colName)).getOrElse(throw ...) -> replaceValue
   })
   val projections = output.map {  attr =>
     attrToValue.get(attr).map {
        case v: jl.Float => fillCol[Float](attr, v) // add an overload of fillCol that takes Attribute
        ...
     }.getOrElse(Column(attr))
   }
   ...
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781829338


   **[Test build #135257 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135257/testReport)** for PR 31545 at commit [`daf7c8a`](https://github.com/apache/spark/commit/daf7c8a97aa10721cdcb181059b6cf1c2134a295).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788549611


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40197/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781877375


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39840/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787035354


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40118/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788095928


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135596/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787032851


   **[Test build #135537 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135537/testReport)** for PR 31545 at commit [`cd0baa1`](https://github.com/apache/spark/commit/cd0baa14e585be285142da68d0e13f77d3d338d4).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781322906


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39809/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787036391


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40118/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584049545



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       Above mentioned approach will not work if the null-fill map has qualified column name(dfAlias.columnName) as below.
   Eg: 
   `def payment = spark.read.format("csv").option("header", "true")
         .schema(paymentSchema)
         .load("Payment.csv").as("payment")
   `
       payment.na.fill(Map("payment.`Customer.Id`" -> -1))
         .show()
   `
   Also, column name quoted with back-tick(`) will also not work.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584049545



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       Above mentioned approach will not work if the null-fill map has qualified column name(dfAlias.columnName) as below.
   Eg: 
   `def payment = spark.read.format("csv").option("header", "true")
         .schema(paymentSchema)
         .load("Payment.csv").as("payment")
   
       payment.na.fill(Map("payment.`Customer.Id`" -> -1))
         .show() `
   Also, column name quoted with back-tick(`) will also not work.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584721566



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,29 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       It seems `Map("ColWith.Dot" -> na)` worked before but doesn't work now. I think it's a reasonable change but we should add an item in the migration guide `docs/sql-migration-guide.md`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r575641562



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -406,11 +408,18 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
         case _ => throw new IllegalArgumentException(
           s"Unsupported value type ${replaceValue.getClass.getName} ($replaceValue).")
       }
+      resolved += (resolvedColumn.name -> replaceValue)
+      // Use resolved column name onwards for filling the null values.
+      // It is needed for the column names having a dot and quoted with back-tick.
+      // Eg: "`ColWith.Dot`" will be resolved to the column with name "ColWith.Dot"
+      // in a dataframe having columns ("ColWith.Dot", "Col").
+      // If resolved name is not used, while filling null values "`ColWith.Dot`" will
+      // not match "ColWith.Dot".

Review comment:
       Usually put the comment before the code you want to comment?
   
   ```
   // Use resolved column name onwards for filling the null
   // ...
   resolved += (resolvedColumn.name -> replaceValue)
   ```
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574320712



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,12 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       Handled the above failure.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787051386


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135539/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SQL] [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r575649132



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -406,11 +408,18 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
         case _ => throw new IllegalArgumentException(
           s"Unsupported value type ${replaceValue.getClass.getName} ($replaceValue).")
       }
+      resolved += (resolvedColumn.name -> replaceValue)
+      // Use resolved column name onwards for filling the null values.
+      // It is needed for the column names having a dot and quoted with back-tick.
+      // Eg: "`ColWith.Dot`" will be resolved to the column with name "ColWith.Dot"
+      // in a dataframe having columns ("ColWith.Dot", "Col").
+      // If resolved name is not used, while filling null values "`ColWith.Dot`" will
+      // not match "ColWith.Dot".

Review comment:
       @viirya  Moved comments ahead of the statement.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788080539


   **[Test build #135596 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135596/testReport)** for PR 31545 at commit [`d0cbbe4`](https://github.com/apache/spark/commit/d0cbbe46cbf8ffbb12eb5bb2feb55d3a549a196c).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787053895


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40120/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787090772


   **[Test build #135540 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135540/testReport)** for PR 31545 at commit [`5ee8bee`](https://github.com/apache/spark/commit/5ee8beed8ca0fec367ef3c4c3e34b58419def47b).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r585185552



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,42 +395,42 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val attrToValue = AttributeMap(values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
-
+      val attr = df.resolve(colName) match {
+        case a: Attribute => a
+        case _ => throw new IllegalArgumentException("Nested field is not supported.")

Review comment:
       Incorporated the feedback. Thanks!




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781848151






----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787051386


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135539/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574236037



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,12 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       I think this will fail if you have `Map("Col" -> na)` because of the same reason due to Line 424 in `DataFrameNaFunctions.scala`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-783531368


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39930/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787032851


   **[Test build #135537 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135537/testReport)** for PR 31545 at commit [`cd0baa1`](https://github.com/apache/spark/commit/cd0baa14e585be285142da68d0e13f77d3d338d4).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781335758


   **[Test build #135228 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135228/testReport)** for PR 31545 at commit [`20b8db7`](https://github.com/apache/spark/commit/20b8db78d657eec066265395fd53465f66f8a220).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
viirya commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-778587076


   Please add `[SQL]` to the PR title, e.g. `[SPARK-34417] [SQL] ...`


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781848151






----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788095928


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135596/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r578366008



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -419,7 +428,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
           case v: jl.Boolean => fillCol[Boolean](f, v.booleanValue())
           case v: String => fillCol[String](f, v)
         }
-      }.getOrElse(df.col(f.name))
+      }.getOrElse(df.col(s"`${f.name}`"))

Review comment:
       does `replace0` have the same issue?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan closed pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #31545:
URL: https://github.com/apache/spark/pull/31545


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
viirya commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788150408


   I updated the PR description a bit to fit the template more. Feel free to edit it.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787958293


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40177/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787036791


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40118/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574257299



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,12 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       @imback82 i think your point is that if data frame has a column having dot in the name, but it is not part of the null fill map then it will fail.
   Yes, it will fail. It will need fix in the catalyst layer.
   ```import org.apache.spark.sql.SparkSession
   
   object ColumnNameWithDot {
   
     def main(args: Array[String]): Unit = {
   
       val spark = SparkSession.builder.appName("Simple Application")
         .config("spark.master", "local").getOrCreate()
   
       spark.sparkContext.setLogLevel("OFF")
   
       import spark.implicits._
       val df = Seq(("abc", 23), ("def", 44), (null, 0)).toDF("ColWith.Dot", "Col.2")
       df.na.fill(Map("`ColWith.Dot`" -> "na"))
         .show()
     }
   }```

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,12 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       @imback82 i think your point is that if data frame has a column having dot in the name, but it is not part of the null fill map then it will fail.
   Yes, it will fail. It will need fix in the catalyst layer.
   ```import org.apache.spark.sql.SparkSession
   
   object ColumnNameWithDot {
   
     def main(args: Array[String]): Unit = {
   
       val spark = SparkSession.builder.appName("Simple Application")
         .config("spark.master", "local").getOrCreate()
   
       spark.sparkContext.setLogLevel("OFF")
   
       import spark.implicits._
       val df = Seq(("abc", 23), ("def", 44), (null, 0)).toDF("ColWith.Dot", "Col.2")
       df.na.fill(Map("`ColWith.Dot`" -> "na"))
         .show()
     }
   }`

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,12 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       @imback82 i think your point is that if data frame has a column having dot in the name, but it is not part of the null fill map then it will fail.
   Yes, it will fail. It will need fix in the catalyst layer.
   ```import org.apache.spark.sql.SparkSession
   
   object ColumnNameWithDot {
   
     def main(args: Array[String]): Unit = {
   
       val spark = SparkSession.builder.appName("Simple Application")
         .config("spark.master", "local").getOrCreate()
   
       spark.sparkContext.setLogLevel("OFF")
   
       import spark.implicits._
       val df = Seq(("abc", 23), ("def", 44), (null, 0)).toDF("ColWith.Dot", "Col.2")
       df.na.fill(Map("`ColWith.Dot`" -> "na"))
         .show()
     }
   }




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788557421


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40197/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781362843


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39809/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781906014


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39840/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r580342149



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -419,7 +428,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
           case v: jl.Boolean => fillCol[Boolean](f, v.booleanValue())
           case v: String => fillCol[String](f, v)
         }
-      }.getOrElse(df.col(f.name))
+      }.getOrElse(df.col(s"`${f.name}`"))

Review comment:
       let's fix it with a separated JIRA




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788660563


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135618/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787057762


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40120/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787066257


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40121/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SQL] [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r575649193



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -419,7 +428,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
           case v: jl.Boolean => fillCol[Boolean](f, v.booleanValue())
           case v: String => fillCol[String](f, v)
         }
-      }.getOrElse(df.col(f.name))
+      }.getOrElse(df.col(if (f.name.contains('.')) s"`${f.name}`" else f.name))

Review comment:
       @viirya done.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-783646664


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135349/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787059295


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40120/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788557421


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40197/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787047885


   **[Test build #135539 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135539/testReport)** for PR 31545 at commit [`f49c742`](https://github.com/apache/spark/commit/f49c742d53f2c08ce49e11f070a142457776c04b).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584484106



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       This is a good point, but we should forbid nested fields as we only support top-level columns here. How about
   ```
   val attrToValue = AttributeMap(values.map { case (colName, replaceValue) =>
     val attr = df.resolve(colName) match {
       case a: Attribute => a
       case _ => throw new IllegalArgumentException("Nested field is not supported")
     }
     attr -> replaceValue
   })
   val projections = output.map {  attr =>
     attrToValue.get(attr).map {
        case v: jl.Float => fillCol[Float](attr, v) // add an overload of fillCol that takes Attribute
        ...
     }.getOrElse(Column(attr))
   }
   ```
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574257299



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,12 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       @imback82 i think your point is that if data frame has a column having dot in the name, but it is not part of the null fill map then it will fail.
   Yes, it will fail. It will need fix in the catalyst layer.
   `import org.apache.spark.sql.SparkSession
   
   object ColumnNameWithDot {
   
     def main(args: Array[String]): Unit = {
   
       val spark = SparkSession.builder.appName("Simple Application")
         .config("spark.master", "local").getOrCreate()
   
       spark.sparkContext.setLogLevel("OFF")
   
       import spark.implicits._
       val df = Seq(("abc", 23), ("def", 44), (null, 0)).toDF("ColWith.Dot", "Col.2")
       df.na.fill(Map("`ColWith.Dot`" -> "na"))
         .show()
     }
   }`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] viirya commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r575642271



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -419,7 +428,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
           case v: jl.Boolean => fillCol[Boolean](f, v.booleanValue())
           case v: String => fillCol[String](f, v)
         }
-      }.getOrElse(df.col(f.name))
+      }.getOrElse(df.col(if (f.name.contains('.')) s"`${f.name}`" else f.name))

Review comment:
       Just always quote the `f.name`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r578403123



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -419,7 +428,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
           case v: jl.Boolean => fillCol[Boolean](f, v.booleanValue())
           case v: String => fillCol[String](f, v)
         }
-      }.getOrElse(df.col(f.name))
+      }.getOrElse(df.col(s"`${f.name}`"))

Review comment:
       Yes, should be fixed in the same PR or separate JIRA-PR? 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787051624


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135537/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-783467571


   **[Test build #135349 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135349/testReport)** for PR 31545 at commit [`24302b9`](https://github.com/apache/spark/commit/24302b9b7e1bb5568c7918369599a43123912f88).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574257803



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,12 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       Can you run this test after changing it to `Map("Col" -> na)`? I see the following failure, which should be handled:
   ```
   sbt:spark-sql> testOnly *DataFrameNaFunctionsSuite -- -z "SPARK-34417"
   [info] - SPARK-34417 - test fillMap() for column with a dot in the name *** FAILED *** (2 seconds, 274 milliseconds)
   [info]   org.apache.spark.sql.AnalysisException: Cannot resolve column name "ColWith.Dot" among (ColWith.Dot, Col); did you mean to quote the `ColWith.Dot` column?
   [info]   at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$resolveException(Dataset.scala:272)
   [info]   at org.apache.spark.sql.Dataset.$anonfun$resolve$1(Dataset.scala:263)
   ...
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] srowen commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
srowen commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574226018



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,22 +395,24 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
+    var resolved: Map[String, Any] = Map()
     values.foreach { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)
 
       // Check data type
       replaceValue match {
         case _: jl.Double | _: jl.Float | _: jl.Integer | _: jl.Long | _: jl.Boolean | _: String =>
-          // This is good
+        // This is good

Review comment:
       Nit: this indent was correct




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787928567


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40177/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-783635016


   **[Test build #135349 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135349/testReport)** for PR 31545 at commit [`24302b9`](https://github.com/apache/spark/commit/24302b9b7e1bb5568c7918369599a43123912f88).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] srowen commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
srowen commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-783449324


   Jenkins retest this please


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574248882



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,12 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       No, it does not fail. Since column is successfully resolved at ```df.resolve(colName)```, it will not go to 424.
   Only for the data frame columns not present in the na-fill-map, it will go to line 424.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787036791


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40118/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787051624


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135537/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787059248


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40121/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584049545



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       Above mentioned approach will not work if the null-fill map has qualified column name(dfAlias.columnName) as below.
   Eg: 
   `
   def payment = spark.read.format("csv").option("header", "true")
         .schema(paymentSchema)
         .load("Payment.csv").as("payment")
       payment.na.fill(Map("payment.`Customer.Id`" -> -1))
         .show()
   `
   Also, column name quoted with back-tick(`) will also not work.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781510412


   **[Test build #135228 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135228/testReport)** for PR 31545 at commit [`20b8db7`](https://github.com/apache/spark/commit/20b8db78d657eec066265395fd53465f66f8a220).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781893314


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39840/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r575619987



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -394,10 +394,11 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
   }
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
+    var resolved: Map[String, Any] = Map()

Review comment:
       I would you a mutable map assigned to a `val`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787059295


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40120/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584706117



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       Incorporated the suggested changes. Please review.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788522629


   **[Test build #135618 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135618/testReport)** for PR 31545 at commit [`896652c`](https://github.com/apache/spark/commit/896652c34aaf8ecc9bcaaa4f81ccb149dd0c4a76).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781339985


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39809/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787899512


   **[Test build #135596 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135596/testReport)** for PR 31545 at commit [`d0cbbe4`](https://github.com/apache/spark/commit/d0cbbe46cbf8ffbb12eb5bb2feb55d3a549a196c).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-783485075


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39930/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781857670


   **[Test build #135260 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135260/testReport)** for PR 31545 at commit [`24302b9`](https://github.com/apache/spark/commit/24302b9b7e1bb5568c7918369599a43123912f88).


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan edited a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
cloud-fan edited a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788752800


   thanks, merging to master/3.1!


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r575639671



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -394,10 +394,11 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
   }
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
+    var resolved: Map[String, Any] = Map()

Review comment:
       @HyukjinKwon done.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788660563


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135618/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787048159


   **[Test build #135539 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135539/testReport)** for PR 31545 at commit [`f49c742`](https://github.com/apache/spark/commit/f49c742d53f2c08ce49e11f070a142457776c04b).
    * This patch **fails Scala style tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584049545



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       Above mentioned approach will not work if the null-fill map has qualified column name(dfAlias.columnName) as below.
   Eg: 
   `def payment = spark.read.format("csv").option("header", "true")
         .schema(paymentSchema)
         .load("Payment.csv").as("payment")
   `
   
       payment.na.fill(Map("payment.`Customer.Id`" -> -1))
         .show()
   `
   Also, column name quoted with back-tick(`) will also not work.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-776843446


   Can one of the admins verify this patch?


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r584049545



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       Above mentioned approach will not work if the null-fill map has qualified column name(dfAlias.columnName) as below.
   Eg: 
   
       _def payment = spark.read.format("csv").option("header", "true")
         .schema(paymentSchema)
         .load("Payment.csv").as("payment")
   
       payment.na.fill(Map("payment.`Customer.Id`" -> -1))
         .show()_
   
   Also, column name quoted with back-tick(`) will also not work.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala
##########
@@ -395,9 +395,9 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) {
 
   private def fillMap(values: Seq[(String, Any)]): DataFrame = {
     // Error handling
-    values.foreach { case (colName, replaceValue) =>
+    val resolved = values.map { case (colName, replaceValue) =>
       // Check column name exists
-      df.resolve(colName)
+      val resolvedColumn = df.resolve(colName)

Review comment:
       Above mentioned approach will not work if the null-fill map has qualified column name(dfAlias.columnName) as below.
   Eg: 
   
       def payment = spark.read.format("csv").option("header", "true")
         .schema(paymentSchema)
         .load("Payment.csv").as("payment")
   
       payment.na.fill(Map("payment.`Customer.Id`" -> -1))
         .show()
   
   Also, column name quoted with back-tick(`) will also not work.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-776843446


   Can one of the admins verify this patch?


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-781530445


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135228/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-788534786


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/40197/
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] amandeep-sharma commented on a change in pull request #31545: [SPARK-34417] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
amandeep-sharma commented on a change in pull request #31545:
URL: https://github.com/apache/spark/pull/31545#discussion_r574257299



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
##########
@@ -460,4 +460,12 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) ::
       Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil)
   }
+
+  test("SPARK-34417 - test fillMap() for column with a dot in the name") {
+    val na = "n/a"
+    checkAnswer(
+      Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col")
+        .na.fill(Map("`ColWith.Dot`" -> na)),

Review comment:
       @imback82 i think your point is that if data frame has a column having dot in the name, but it is not part of the null fill map then it will fail.
   Yes, it will fail, example below. It will need fix in the catalyst layer or line 424.
   ```import org.apache.spark.sql.SparkSession
   
   object ColumnNameWithDot {
   
     def main(args: Array[String]): Unit = {
   
       val spark = SparkSession.builder.appName("Simple Application")
         .config("spark.master", "local").getOrCreate()
   
       spark.sparkContext.setLogLevel("OFF")
   
       import spark.implicits._
       val df = Seq(("abc", 23), ("def", 44), (null, 0)).toDF("ColWith.Dot", "Col.2")
       df.na.fill(Map("`ColWith.Dot`" -> "na"))
         .show()
     }
   }




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31545: [SPARK-34417] [SQL] org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) fails for column name having a dot

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31545:
URL: https://github.com/apache/spark/pull/31545#issuecomment-787051391


   **[Test build #135537 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135537/testReport)** for PR 31545 at commit [`cd0baa1`](https://github.com/apache/spark/commit/cd0baa14e585be285142da68d0e13f77d3d338d4).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org