Hi guys, Apologies for the long mail. I am running below code snippet
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, 9)).toDF("ColWith.Dot", "Col") df.na.fill(Map("`ColWith.Dot`" -> "n/a")).show() } } and it is failing with error Exception in thread "main" org.apache.spark.sql.AnalysisException: Cannot resolve column name "ColWith.Dot" among (ColWith.Dot, Col); I checked that code-fix were made for the similar issue, found https://issues.apache.org/jira/browse/SPARK-19473; but none of them fixed all cases. I debugged the code below are the observations 1. In org.apache.spark.sql.DataFrameNaFunctions.fillMap(values: Seq[(String, Any)]) the df.resolve(colName) call succeeds, since column name is quoted with back tick it resolves the column 2. val projections = df.schema.fields.map { ... ... }.getOrElse(df.col(f.name)) fails since resolved column name is not quoted with backtick Problem lies in the org.apache.spark.sql.catalyst.expressions resolve(nameParts: Seq[String], resolver: Resolver): Option[NamedExpression] where the comment says we try to resolve it as a column. // If none of attributes match database.table.column pattern or // `table.column` pattern, we try to resolve it as a column. val (candidates, nestedFields) = matches match { case (Seq(), _) => val name = nameParts.head val attributes = collectMatches(name, direct.get(name.toLowerCase(Locale.ROOT))) (attributes, nameParts.tail) case _ => matches } should be changed to // If none of attributes match database.table.column pattern or // `table.column` pattern, we try to resolve it as a column. val (candidates, nestedFields) = matches match { case (Seq(), _) => val name = nameParts.mkString(".") val attributes = collectMatches(name, direct.get(name.toLowerCase(Locale.ROOT))) (attributes, Seq.empty) case _ => matches } git diff is as below - val name = nameParts.head + val name = nameParts.mkString(".") val attributes = collectMatches(name, direct.get(name.toLowerCase(Locale.ROOT))) - (attributes, nameParts.tail) + (attributes, Seq.empty) I tested this change, there is no longer need to use backtick with columns having dot in the name. Can this change be merged? Regards, Amandeep