Skip to content

Commit

Permalink
[SPARK-6948] [MLLIB] compress vectors in VectorAssembler
Browse files Browse the repository at this point in the history
The compression is based on storage. brkyvz

Author: Xiangrui Meng <[email protected]>

Closes #5985 from mengxr/SPARK-6948 and squashes the following commits:

df56a00 [Xiangrui Meng] update python tests
6d90d45 [Xiangrui Meng] compress vectors in VectorAssembler
  • Loading branch information
mengxr committed May 7, 2015
1 parent 658a478 commit e43803b
Show file tree
Hide file tree
Showing 3 changed files with 13 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,6 @@ object VectorAssembler {
case o =>
throw new SparkException(s"$o of type ${o.getClass.getName} is not supported.")
}
Vectors.sparse(cur, indices.result(), values.result())
Vectors.sparse(cur, indices.result(), values.result()).compressed
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature
import org.scalatest.FunSuite

import org.apache.spark.SparkException
import org.apache.spark.mllib.linalg.{Vector, Vectors}
import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors}
import org.apache.spark.mllib.util.MLlibTestSparkContext
import org.apache.spark.sql.{Row, SQLContext}

Expand Down Expand Up @@ -48,6 +48,14 @@ class VectorAssemblerSuite extends FunSuite with MLlibTestSparkContext {
}
}

test("assemble should compress vectors") {
import org.apache.spark.ml.feature.VectorAssembler.assemble
val v1 = assemble(0.0, 0.0, 0.0, Vectors.dense(4.0))
assert(v1.isInstanceOf[SparseVector])
val v2 = assemble(1.0, 2.0, 3.0, Vectors.sparse(1, Array(0), Array(4.0)))
assert(v2.isInstanceOf[DenseVector])
}

test("VectorAssembler") {
val df = sqlContext.createDataFrame(Seq(
(0, 0.0, Vectors.dense(1.0, 2.0), "a", Vectors.sparse(2, Array(1), Array(3.0)), 10L)
Expand Down
6 changes: 3 additions & 3 deletions python/pyspark/ml/feature.py
Original file line number Diff line number Diff line change
Expand Up @@ -121,12 +121,12 @@ class VectorAssembler(JavaTransformer, HasInputCols, HasOutputCol):
>>> df = sc.parallelize([Row(a=1, b=0, c=3)]).toDF()
>>> vecAssembler = VectorAssembler(inputCols=["a", "b", "c"], outputCol="features")
>>> vecAssembler.transform(df).head().features
SparseVector(3, {0: 1.0, 2: 3.0})
DenseVector([1.0, 0.0, 3.0])
>>> vecAssembler.setParams(outputCol="freqs").transform(df).head().freqs
SparseVector(3, {0: 1.0, 2: 3.0})
DenseVector([1.0, 0.0, 3.0])
>>> params = {vecAssembler.inputCols: ["b", "a"], vecAssembler.outputCol: "vector"}
>>> vecAssembler.transform(df, params).head().vector
SparseVector(2, {1: 1.0})
DenseVector([0.0, 1.0])
"""

_java_class = "org.apache.spark.ml.feature.VectorAssembler"
Expand Down

0 comments on commit e43803b

Please sign in to comment.