spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Takeshi Yamamuro (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (SPARK-19217) Offer easy cast from vector to array
Date Sun, 21 Jan 2018 05:32:00 GMT

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

Takeshi Yamamuro commented on SPARK-19217:
------------------------------------------

If it still makes some sense and nobody takes this, I'll do. How about adding a new method
for supported cast types in UserDefinedType ( [https://github.com/apache/spark/compare/master...maropu:CastUDF).]
 In this example we could cast from VectorUDT to Array and _vice versa_.
{code:java}
scala> val df1 = Seq((1, Vectors.dense(Array(1.0, 2.0, 3.0)))).toDF("a", "b")
scala> val df2 = df1.selectExpr("CAST(b AS ARRAY<DOUBLE>)")
scala> df2.printSchema
root
 |-- b: array (nullable = true)
 |    |-- element: double (containsNull = true)

scala> df2.show
+---------------+
|              b|
+---------------+
|[1.0, 2.0, 3.0]|
+---------------+

scala> val df3 = Seq((1, Seq(1.0, 2.0, 3.0))).toDF("a", "b")
scala> val df4 = df3.select(df3("b").cast(new VectorUDT()))
scala> df4.printSchema
root
 |-- b: vector (nullable = true)

scala> df4.show
+-------------+
|            b|
+-------------+
|[1.0,2.0,3.0]|
+-------------+
{code}

WDYT cc: [~cloud_fan]
 

> Offer easy cast from vector to array
> ------------------------------------
>
>                 Key: SPARK-19217
>                 URL: https://issues.apache.org/jira/browse/SPARK-19217
>             Project: Spark
>          Issue Type: Improvement
>          Components: ML, PySpark, SQL
>    Affects Versions: 2.1.0
>            Reporter: Nicholas Chammas
>            Priority: Minor
>
> Working with ML often means working with DataFrames with vector columns. You can't save
these DataFrames to storage (edit: at least as ORC) without converting the vector columns
to array columns, and there doesn't appear to an easy way to make that conversion.
> This is a common enough problem that it is [documented on Stack Overflow|http://stackoverflow.com/q/35855382/877069].
The current solutions to making the conversion from a vector column to an array column are:
> # Convert the DataFrame to an RDD and back
> # Use a UDF
> Both approaches work fine, but it really seems like you should be able to do something
like this instead:
> {code}
> (le_data
>     .select(
>         col('features').cast('array').alias('features')
>     ))
> {code}
> We already have an {{ArrayType}} in {{pyspark.sql.types}}, but it appears that {{cast()}}
doesn't support this conversion.
> Would this be an appropriate thing to add?



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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


Mime
View raw message