[ https://issues.apache.org/jira/browse/SPARK-29890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17266063#comment-17266063 ]
Peter Toth commented on SPARK-29890: ------------------------------------ [~imback82], [~cloud_fan], due this change `fill` started to throw exception when `cols` contains a column that can't be resolved. I wonder if this behaviour change of `fill` is desired or it is more like bug. I'm happy to open fix PR if you think this side effect is unintended. > Unable to fill na with 0 with duplicate columns > ----------------------------------------------- > > Key: SPARK-29890 > URL: https://issues.apache.org/jira/browse/SPARK-29890 > Project: Spark > Issue Type: Bug > Components: Spark Shell > Affects Versions: 2.0.2, 2.1.3, 2.2.3, 2.3.3, 2.4.3 > Reporter: sandeshyapuram > Assignee: Terry Kim > Priority: Major > Fix For: 2.4.5, 3.0.0 > > > Trying to fill out na values with 0. > {noformat} > scala> :paste > // Entering paste mode (ctrl-D to finish) > val parent = > spark.sparkContext.parallelize(Seq((1,2),(3,4),(5,6))).toDF("nums", "abc") > val c1 = parent.filter(lit(true)) > val c2 = parent.filter(lit(true)) > c1.join(c2, Seq("nums"), "left") > .na.fill(0).show{noformat} > {noformat} > 9/11/14 04:24:24 ERROR org.apache.hadoop.security.JniBasedUnixGroupsMapping: > error looking up the name of group 820818257: No such file or directory > org.apache.spark.sql.AnalysisException: Reference 'abc' is ambiguous, could > be: abc, abc.; > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:213) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveQuoted(LogicalPlan.scala:117) > at org.apache.spark.sql.Dataset.resolve(Dataset.scala:220) > at org.apache.spark.sql.Dataset.col(Dataset.scala:1246) > at > org.apache.spark.sql.DataFrameNaFunctions.org$apache$spark$sql$DataFrameNaFunctions$$fillCol(DataFrameNaFunctions.scala:443) > at > org.apache.spark.sql.DataFrameNaFunctions$$anonfun$7.apply(DataFrameNaFunctions.scala:500) > at > org.apache.spark.sql.DataFrameNaFunctions$$anonfun$7.apply(DataFrameNaFunctions.scala:492) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) > at > scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) > at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) > at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186) > at > org.apache.spark.sql.DataFrameNaFunctions.fillValue(DataFrameNaFunctions.scala:492) > at > org.apache.spark.sql.DataFrameNaFunctions.fill(DataFrameNaFunctions.scala:171) > at > org.apache.spark.sql.DataFrameNaFunctions.fill(DataFrameNaFunctions.scala:155) > at > org.apache.spark.sql.DataFrameNaFunctions.fill(DataFrameNaFunctions.scala:134) > ... 54 elided{noformat} > -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org