[ 
https://issues.apache.org/jira/browse/SPARK-11569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14995580#comment-14995580
 ] 

Maciej Szymkiewicz commented on SPARK-11569:
--------------------------------------------

It looks this problem affects Scala after all:

{code}

val df = sqlContext.createDataFrame(
  Seq(("asd2s","1e1e",1.1,0), ("asd2s","1e1e",0.1,0), 
      (null,"1e3e",1.2,0), ("bd34t","1e1e",5.1,1), 
      ("asd2s","1e3e",0.2,0), ("bd34t","1e2e",4.3,1))
).toDF("x0","x1","x2","x3")
val indexer = new StringIndexer().setInputCol("x0").setOutputCol("x0idx")

indexer.fit(df).transform(df).show
// java.lang.NullPointerException
//      at 
org.apache.spark.sql.types.Metadata$.org$apache$spark$sql$types$Metadata$$hash(Metadata.scala:208)
//      at 
org.apache.spark.sql.types.Metadata$$anonfun$org$apache$spark$sql$types$Metadata$$hash$2.apply(Metadata.scala:196)
//      at 
org.apache.spark.sql.types.Metadata$$anonfun$org$apache$spark$sql$types$Metadata$$hash$2.apply(Metadata.scala:196)
//      at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
{code}

Source: http://stackoverflow.com/q/33574807/1560062


> StringIndexer transform fails when column contains nulls
> --------------------------------------------------------
>
>                 Key: SPARK-11569
>                 URL: https://issues.apache.org/jira/browse/SPARK-11569
>             Project: Spark
>          Issue Type: Bug
>          Components: ML, PySpark
>    Affects Versions: 1.4.0, 1.5.0, 1.6.0
>            Reporter: Maciej Szymkiewicz
>
> Transforming column containing {{null}} values using {{StringIndexer}} 
> results in {{java.lang.NullPointerException}}
> {code}
> from pyspark.ml.feature import StringIndexer
> df = sqlContext.createDataFrame([("a", 1), (None, 2)], ("k", "v"))
> df.printSchema()
> ## root
> ##  |-- k: string (nullable = true)
> ##  |-- v: long (nullable = true)
> indexer = StringIndexer(inputCol="k", outputCol="kIdx")
> indexer.fit(df).transform(df)
> ## <repr(<pyspark.sql.dataframe.DataFrame at 0x7f4b0d8e7110>) failed: 
> py4j.protocol.Py4JJavaError: An error occurred while calling o75.json.
> ## : java.lang.NullPointerException
> {code}
> Problem disappears when we drop 
> {code}
> df1 = df.na.drop()
> indexer.fit(df1).transform(df1)
> {code}
> or replace {{nulls}}
> {code}
> from pyspark.sql.functions import col, when
> k = col("k")
> df2 = df.withColumn("k", when(k.isNull(), "__NA__").otherwise(k))
> indexer.fit(df2).transform(df2)
> {code}
> and cannot be reproduced using Scala API
> {code}
> import org.apache.spark.ml.feature.StringIndexer
> val df = sc.parallelize(Seq(("a", 1), (null, 2))).toDF("k", "v")
> df.printSchema
> // root
> //  |-- k: string (nullable = true)
> //  |-- v: integer (nullable = false)
> val indexer = new StringIndexer().setInputCol("k").setOutputCol("kIdx")
> indexer.fit(df).transform(df).count
> // 2
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to