Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
53 changes: 45 additions & 8 deletions mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ package org.apache.spark.ml.clustering

import java.util.Locale

import breeze.linalg.normalize
import breeze.numerics.exp
import org.apache.hadoop.fs.Path
import org.json4s.DefaultFormats
import org.json4s.JsonAST.JObject
Expand All @@ -27,23 +29,23 @@ import org.json4s.jackson.JsonMethods._
import org.apache.spark.annotation.{DeveloperApi, Since}
import org.apache.spark.internal.Logging
import org.apache.spark.ml.{Estimator, Model}
import org.apache.spark.ml.linalg.{Matrix, Vector, Vectors, VectorUDT}
import org.apache.spark.ml.linalg._
import org.apache.spark.ml.param._
import org.apache.spark.ml.param.shared.{HasCheckpointInterval, HasFeaturesCol, HasMaxIter, HasSeed}
import org.apache.spark.ml.util._
import org.apache.spark.ml.util.DefaultParamsReader.Metadata
import org.apache.spark.ml.util.Instrumentation.instrumented
import org.apache.spark.mllib.clustering.{DistributedLDAModel => OldDistributedLDAModel,
EMLDAOptimizer => OldEMLDAOptimizer, LDA => OldLDA, LDAModel => OldLDAModel,
LDAOptimizer => OldLDAOptimizer, LocalLDAModel => OldLocalLDAModel,
LDAOptimizer => OldLDAOptimizer, LDAUtils => OldLDAUtils, LocalLDAModel => OldLocalLDAModel,
OnlineLDAOptimizer => OldOnlineLDAOptimizer}
import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors}
import org.apache.spark.mllib.linalg.MatrixImplicits._
import org.apache.spark.mllib.linalg.VectorImplicits._
import org.apache.spark.mllib.util.MLUtils
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession}
import org.apache.spark.sql.functions.{col, monotonically_increasing_id, udf}
import org.apache.spark.sql.functions.{monotonically_increasing_id, udf}
import org.apache.spark.sql.types.StructType
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.PeriodicCheckpointer
Expand Down Expand Up @@ -457,21 +459,56 @@ abstract class LDAModel private[ml] (
*/
@Since("2.0.0")
override def transform(dataset: Dataset[_]): DataFrame = {
if ($(topicDistributionCol).nonEmpty) {
transformSchema(dataset.schema, logging = true)

// TODO: Make the transformer natively in ml framework to avoid extra conversion.
val transformer = oldLocalModel.getTopicDistributionMethod
if ($(topicDistributionCol).nonEmpty) {
val func = getTopicDistributionMethod
val transformer = udf(func)

val t = udf { (v: Vector) => transformer(OldVectors.fromML(v)).asML }
dataset.withColumn($(topicDistributionCol),
t(DatasetUtils.columnToVector(dataset, getFeaturesCol))).toDF()
transformer(DatasetUtils.columnToVector(dataset, getFeaturesCol)))
} else {
logWarning("LDAModel.transform was called without any output columns. Set an output column" +
" such as topicDistributionCol to produce results.")
dataset.toDF()
}
}

/**
* Get a method usable as a UDF for `topicDistributions()`
*/
private def getTopicDistributionMethod: Vector => Vector = {
val expElogbeta = exp(OldLDAUtils.dirichletExpectation(topicsMatrix.asBreeze.toDenseMatrix.t).t)
val oldModel = oldLocalModel
val docConcentrationBrz = oldModel.docConcentration.asBreeze
val gammaShape = oldModel.gammaShape
val k = oldModel.k
val gammaSeed = oldModel.seed

vector: Vector =>
if (vector.numNonzeros == 0) {
Vectors.zeros(k)
} else {
val (ids: List[Int], cts: Array[Double]) = vector match {
case v: DenseVector => ((0 until v.size).toList, v.values)
case v: SparseVector => (v.indices.toList, v.values)
case other =>
throw new UnsupportedOperationException(
s"Only sparse and dense vectors are supported but got ${other.getClass}.")
}

val (gamma, _, _) = OldOnlineLDAOptimizer.variationalTopicInference(
ids,
cts,
expElogbeta,
docConcentrationBrz,
gammaShape,
k,
gammaSeed)
Vectors.dense(normalize(gamma, 1.0).toArray)
}
}

@Since("1.6.0")
override def transformSchema(schema: StructType): StructType = {
validateAndTransformSchema(schema)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,6 @@ package org.apache.spark.ml.feature

import java.{util => ju}

import org.json4s.JsonDSL._
import org.json4s.JValue
import org.json4s.jackson.JsonMethods._

import org.apache.spark.SparkException
import org.apache.spark.annotation.Since
import org.apache.spark.ml.Model
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.Path
import org.apache.spark.annotation.Since
import org.apache.spark.ml._
import org.apache.spark.ml.attribute.{AttributeGroup, _}
import org.apache.spark.ml.linalg.{Vector, VectorUDT}
import org.apache.spark.ml.linalg._
import org.apache.spark.ml.param._
import org.apache.spark.ml.param.shared._
import org.apache.spark.ml.util._
Expand Down Expand Up @@ -264,14 +264,25 @@ final class ChiSqSelectorModel private[ml] (

@Since("2.0.0")
override def transform(dataset: Dataset[_]): DataFrame = {
val transformedSchema = transformSchema(dataset.schema, logging = true)
val newField = transformedSchema.last

// TODO: Make the transformer natively in ml framework to avoid extra conversion.
val transformer: Vector => Vector = v => chiSqSelector.transform(OldVectors.fromML(v)).asML
val outputSchema = transformSchema(dataset.schema, logging = true)

val newSize = selectedFeatures.length
val func = { vector: Vector =>
vector match {
case SparseVector(_, indices, values) =>
val (newIndices, newValues) = chiSqSelector.compressSparse(indices, values)
Vectors.sparse(newSize, newIndices, newValues)
case DenseVector(values) =>
Vectors.dense(chiSqSelector.compressDense(values))
case other =>
throw new UnsupportedOperationException(
s"Only sparse and dense vectors are supported but got ${other.getClass}.")
}
}

val selector = udf(transformer)
dataset.withColumn($(outputCol), selector(col($(featuresCol))), newField.metadata)
val transformer = udf(func)
dataset.withColumn($(outputCol), transformer(col($(featuresCol))),
outputSchema($(outputCol)).metadata)
}

@Since("1.6.0")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,11 @@ package org.apache.spark.ml.feature

import org.apache.spark.annotation.Since
import org.apache.spark.ml.UnaryTransformer
import org.apache.spark.ml.linalg.{Vector, VectorUDT}
import org.apache.spark.ml.linalg._
import org.apache.spark.ml.param.Param
import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable}
import org.apache.spark.mllib.feature
import org.apache.spark.mllib.linalg.VectorImplicits._
import org.apache.spark.mllib.feature.{ElementwiseProduct => OldElementwiseProduct}
import org.apache.spark.mllib.linalg.{Vectors => OldVectors}
import org.apache.spark.sql.types.DataType

/**
Expand Down Expand Up @@ -55,8 +55,24 @@ class ElementwiseProduct @Since("1.4.0") (@Since("1.4.0") override val uid: Stri

override protected def createTransformFunc: Vector => Vector = {
require(params.contains(scalingVec), s"transformation requires a weight vector")
val elemScaler = new feature.ElementwiseProduct($(scalingVec))
v => elemScaler.transform(v)
val elemScaler = new OldElementwiseProduct(OldVectors.fromML($(scalingVec)))
val vectorSize = $(scalingVec).size

vector: Vector => {
require(vector.size == vectorSize,
s"vector sizes do not match: Expected $vectorSize but found ${vector.size}")
vector match {
case DenseVector(values) =>
val newValues = elemScaler.transformDense(values)
Vectors.dense(newValues)
case SparseVector(size, indices, values) =>
val (newIndices, newValues) = elemScaler.transformSparse(indices, values)
Vectors.sparse(size, newIndices, newValues)
case other =>
throw new UnsupportedOperationException(
s"Only sparse and dense vectors are supported but got ${other.getClass}.")
}
}
}

override protected def outputDataType: DataType = new VectorUDT()
Expand Down
18 changes: 12 additions & 6 deletions mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,11 @@ package org.apache.spark.ml.feature
import org.apache.spark.annotation.Since
import org.apache.spark.ml.Transformer
import org.apache.spark.ml.attribute.AttributeGroup
import org.apache.spark.ml.linalg.Vectors
import org.apache.spark.ml.param._
import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
import org.apache.spark.ml.util._
import org.apache.spark.mllib.feature
import org.apache.spark.mllib.feature.{HashingTF => OldHashingTF}
import org.apache.spark.sql.{DataFrame, Dataset}
import org.apache.spark.sql.functions.{col, udf}
import org.apache.spark.sql.types.{ArrayType, StructType}
Expand Down Expand Up @@ -93,11 +94,16 @@ class HashingTF @Since("1.4.0") (@Since("1.4.0") override val uid: String)
@Since("2.0.0")
override def transform(dataset: Dataset[_]): DataFrame = {
val outputSchema = transformSchema(dataset.schema)
val hashingTF = new feature.HashingTF($(numFeatures)).setBinary($(binary))
// TODO: Make the hashingTF.transform natively in ml framework to avoid extra conversion.
val t = udf { terms: Seq[_] => hashingTF.transform(terms).asML }
val metadata = outputSchema($(outputCol)).metadata
dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata))

val hashingTF = new OldHashingTF($(numFeatures)).setBinary($(binary))
val func = (terms: Seq[_]) => {
val seq = hashingTF.transformImpl(terms)
Vectors.sparse(hashingTF.numFeatures, seq)
}

val transformer = udf(func)
dataset.withColumn($(outputCol), transformer(col($(inputCol))),
outputSchema($(outputCol)).metadata)
}

@Since("1.4.0")
Expand Down
23 changes: 19 additions & 4 deletions mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path

import org.apache.spark.annotation.Since
import org.apache.spark.ml._
import org.apache.spark.ml.linalg.{Vector, VectorUDT}
import org.apache.spark.ml.linalg._
import org.apache.spark.ml.param._
import org.apache.spark.ml.param.shared._
import org.apache.spark.ml.util._
Expand Down Expand Up @@ -132,9 +132,24 @@ class IDFModel private[ml] (
@Since("2.0.0")
override def transform(dataset: Dataset[_]): DataFrame = {
transformSchema(dataset.schema, logging = true)
// TODO: Make the idfModel.transform natively in ml framework to avoid extra conversion.
val idf = udf { vec: Vector => idfModel.transform(OldVectors.fromML(vec)).asML }
dataset.withColumn($(outputCol), idf(col($(inputCol))))

val func = { vector: Vector =>
vector match {
case SparseVector(size, indices, values) =>
val (newIndices, newValues) = feature.IDFModel.transformSparse(idfModel.idf,
indices, values)
Vectors.sparse(size, newIndices, newValues)
case DenseVector(values) =>
val newValues = feature.IDFModel.transformDense(idfModel.idf, values)
Vectors.dense(newValues)
case other =>
throw new UnsupportedOperationException(
s"Only sparse and dense vectors are supported but got ${other.getClass}.")
}
}

val transformer = udf(func)
dataset.withColumn($(outputCol), transformer(col($(inputCol))))
}

@Since("1.4.0")
Expand Down
34 changes: 20 additions & 14 deletions mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,7 @@ import org.apache.spark.ml.param._
import org.apache.spark.ml.param.shared._
import org.apache.spark.ml.util._
import org.apache.spark.mllib.feature
import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, DenseVector => OldDenseVector,
Matrices => OldMatrices, Vector => OldVector, Vectors => OldVectors}
import org.apache.spark.mllib.linalg.MatrixImplicits._
import org.apache.spark.mllib.linalg.VectorImplicits._
import org.apache.spark.rdd.RDD
import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, Vectors => OldVectors}
import org.apache.spark.sql._
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types.{StructField, StructType}
Expand Down Expand Up @@ -92,12 +88,13 @@ class PCA @Since("1.5.0") (
@Since("2.0.0")
override def fit(dataset: Dataset[_]): PCAModel = {
transformSchema(dataset.schema, logging = true)
val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map {
val input = dataset.select($(inputCol)).rdd.map {
case Row(v: Vector) => OldVectors.fromML(v)
}
val pca = new feature.PCA(k = $(k))
val pcaModel = pca.fit(input)
copyValues(new PCAModel(uid, pcaModel.pc, pcaModel.explainedVariance).setParent(this))
copyValues(new PCAModel(uid, pcaModel.pc.asML, pcaModel.explainedVariance.asML)
.setParent(this))
}

@Since("1.5.0")
Expand Down Expand Up @@ -149,15 +146,24 @@ class PCAModel private[ml] (
@Since("2.0.0")
override def transform(dataset: Dataset[_]): DataFrame = {
transformSchema(dataset.schema, logging = true)
val pcaModel = new feature.PCAModel($(k),
OldMatrices.fromML(pc).asInstanceOf[OldDenseMatrix],
OldVectors.fromML(explainedVariance).asInstanceOf[OldDenseVector])

// TODO: Make the transformer natively in ml framework to avoid extra conversion.
val transformer: Vector => Vector = v => pcaModel.transform(OldVectors.fromML(v)).asML
val func = { vector: Vector =>
vector match {
case dv: DenseVector =>
pc.transpose.multiply(dv)
case SparseVector(size, indices, values) =>
/* SparseVector -> single row SparseMatrix */
val sm = Matrices.sparse(size, 1, Array(0, indices.length), indices, values).transpose
val projection = sm.multiply(pc)
Vectors.dense(projection.values)
case _ =>
throw new IllegalArgumentException("Unsupported vector format. Expected " +
s"SparseVector or DenseVector. Instead got: ${vector.getClass}")
}
}

val pcaOp = udf(transformer)
dataset.withColumn($(outputCol), pcaOp(col($(inputCol))))
val transformer = udf(func)
dataset.withColumn($(outputCol), transformer(col($(inputCol))))
}

@Since("1.5.0")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path

import org.apache.spark.annotation.Since
import org.apache.spark.ml._
import org.apache.spark.ml.linalg.{Vector, VectorUDT}
import org.apache.spark.ml.linalg._
import org.apache.spark.ml.param._
import org.apache.spark.ml.param.shared._
import org.apache.spark.ml.util._
Expand Down Expand Up @@ -162,11 +162,34 @@ class StandardScalerModel private[ml] (
transformSchema(dataset.schema, logging = true)
val scaler = new feature.StandardScalerModel(std, mean, $(withStd), $(withMean))

// TODO: Make the transformer natively in ml framework to avoid extra conversion.
val transformer: Vector => Vector = v => scaler.transform(OldVectors.fromML(v)).asML
val func = if ($(withMean)) {
vector: Vector =>
val values = vector match {
// specially handle DenseVector because its toArray does not clone already
case d: DenseVector => d.values.clone()
case v: Vector => v.toArray
}
val newValues = scaler.transfromWithMean(values)
Vectors.dense(newValues)
} else if ($(withStd)) {
vector: Vector =>
vector match {
case DenseVector(values) =>
val newValues = scaler.transformDenseWithStd(values)
Vectors.dense(newValues)
case SparseVector(size, indices, values) =>
val (newIndices, newValues) = scaler.transformSparseWithStd(indices, values)
Vectors.sparse(size, newIndices, newValues)
case other =>
throw new UnsupportedOperationException(
s"Only sparse and dense vectors are supported but got ${other.getClass}.")
}
} else {
vector: Vector => vector
}

val scale = udf(transformer)
dataset.withColumn($(outputCol), scale(col($(inputCol))))
val transformer = udf(func)
dataset.withColumn($(outputCol), transformer(col($(inputCol))))
}

@Since("1.4.0")
Expand Down
Loading