Skip to content

Commit

Permalink
[SPARK-33556][ML] Add array_to_vector function for dataframe column
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

Add array_to_vector function for dataframe column

### Why are the changes needed?
Utility function for array to vector conversion.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
scala unit test & doctest.

Closes apache#30498 from WeichenXu123/array_to_vec.

Lead-authored-by: Weichen Xu <weichen.xu@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
  • Loading branch information
WeichenXu123 and HyukjinKwon committed Dec 1, 2020
1 parent f5d2165 commit 596fbc1
Show file tree
Hide file tree
Showing 5 changed files with 68 additions and 3 deletions.
16 changes: 15 additions & 1 deletion mllib/src/main/scala/org/apache/spark/ml/functions.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.ml

import org.apache.spark.annotation.Since
import org.apache.spark.ml.linalg.{SparseVector, Vector}
import org.apache.spark.ml.linalg.{SparseVector, Vector, Vectors}
import org.apache.spark.mllib.linalg.{Vector => OldVector}
import org.apache.spark.sql.Column
import org.apache.spark.sql.functions.udf
Expand Down Expand Up @@ -72,6 +72,20 @@ object functions {
}
}

private val arrayToVectorUdf = udf { array: Seq[Double] =>
Vectors.dense(array.toArray)
}

/**
* Converts a column of array of numeric type into a column of dense vectors in MLlib.
* @param v: the column of array&lt;NumericType&gt type
* @return a column of type `org.apache.spark.ml.linalg.Vector`
* @since 3.1.0
*/
def array_to_vector(v: Column): Column = {
arrayToVectorUdf(v)
}

private[ml] def checkNonNegativeWeight = udf {
value: Double =>
require(value >= 0, s"illegal weight value: $value. weight must be >= 0.0.")
Expand Down
18 changes: 16 additions & 2 deletions mllib/src/test/scala/org/apache/spark/ml/FunctionsSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@
package org.apache.spark.ml

import org.apache.spark.SparkException
import org.apache.spark.ml.functions.vector_to_array
import org.apache.spark.ml.linalg.Vectors
import org.apache.spark.ml.functions.{array_to_vector, vector_to_array}
import org.apache.spark.ml.linalg.{Vector, Vectors}
import org.apache.spark.ml.util.MLTest
import org.apache.spark.mllib.linalg.{Vectors => OldVectors}
import org.apache.spark.sql.functions.col
Expand Down Expand Up @@ -87,4 +87,18 @@ class FunctionsSuite extends MLTest {
assert(thrown2.getMessage.contains(
s"Unsupported dtype: float16. Valid values: float64, float32."))
}

test("test array_to_vector") {
val df1 = Seq(Tuple1(Array(0.5, 1.5))).toDF("c1")
val resultVec = df1.select(array_to_vector(col("c1"))).collect()(0)(0).asInstanceOf[Vector]
assert(resultVec === Vectors.dense(Array(0.5, 1.5)))

val df2 = Seq(Tuple1(Array(1.5f, 2.5f))).toDF("c1")
val resultVec2 = df2.select(array_to_vector(col("c1"))).collect()(0)(0).asInstanceOf[Vector]
assert(resultVec2 === Vectors.dense(Array(1.5, 2.5)))

val df3 = Seq(Tuple1(Array(1, 2))).toDF("c1")
val resultVec3 = df3.select(array_to_vector(col("c1"))).collect()(0)(0).asInstanceOf[Vector]
assert(resultVec3 === Vectors.dense(Array(1.0, 2.0)))
}
}
1 change: 1 addition & 0 deletions python/docs/source/reference/pyspark.ml.rst
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,7 @@ ML Functions
.. autosummary::
:toctree: api/

array_to_vector
vector_to_array


Expand Down
34 changes: 34 additions & 0 deletions python/pyspark/ml/functions.py
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,40 @@ def vector_to_array(col, dtype="float64"):
sc._jvm.org.apache.spark.ml.functions.vector_to_array(_to_java_column(col), dtype))


def array_to_vector(col):
"""
Converts a column of array of numeric type into a column of dense vectors in MLlib
.. versionadded:: 3.1.0
Parameters
----------
col : :py:class:`pyspark.sql.Column` or str
Input column
Returns
-------
:py:class:`pyspark.sql.Column`
The converted column of MLlib dense vectors.
Examples
--------
>>> from pyspark.ml.functions import array_to_vector
>>> df1 = spark.createDataFrame([([1.5, 2.5],),], schema='v1 array<double>')
>>> df1.select(array_to_vector('v1').alias('vec1')).collect()
[Row(vec1=DenseVector([1.5, 2.5]))]
>>> df2 = spark.createDataFrame([([1.5, 3.5],),], schema='v1 array<float>')
>>> df2.select(array_to_vector('v1').alias('vec1')).collect()
[Row(vec1=DenseVector([1.5, 3.5]))]
>>> df3 = spark.createDataFrame([([1, 3],),], schema='v1 array<int>')
>>> df3.select(array_to_vector('v1').alias('vec1')).collect()
[Row(vec1=DenseVector([1.0, 3.0]))]
"""
sc = SparkContext._active_spark_context
return Column(
sc._jvm.org.apache.spark.ml.functions.array_to_vector(_to_java_column(col)))


def _test():
import doctest
from pyspark.sql import SparkSession
Expand Down
2 changes: 2 additions & 0 deletions python/pyspark/ml/functions.pyi
Original file line number Diff line number Diff line change
Expand Up @@ -20,3 +20,5 @@ from pyspark import SparkContext as SparkContext, since as since # noqa: F401
from pyspark.sql.column import Column as Column

def vector_to_array(col: Column) -> Column: ...

def array_to_vector(col: Column) -> Column: ...

0 comments on commit 596fbc1

Please sign in to comment.