Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Prev Previous commit
Next Next commit
update pr
  • Loading branch information
zhengruifeng committed Jan 16, 2017
commit 6973301f0e90a1aa3793da1535d3afff128bcda5
Original file line number Diff line number Diff line change
Expand Up @@ -135,11 +135,20 @@ class GaussianMixtureModel private[ml] (
@Since("2.1.0")
def computeLogLikelihood(dataset: Dataset[_]): Double = {
SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new VectorUDT)
val sc = dataset.sparkSession.sparkContext
val bcweightVec = sc.broadcast(Vectors.dense(weights))
transform(dataset).select($(probabilityCol)).map {
case Row(probs: Vector) =>
val likelihood = BLAS.dot(probs, bcweightVec.value)

val spark = dataset.sparkSession

import spark.implicits._

val bcWeightAndDists =
spark.sparkContext.broadcast(weights.zip(gaussians))

dataset.select(col($(featuresCol))).map {
case Row(point: Vector) =>
val likelihood = bcWeightAndDists.value.map {
case (weight, dist) =>
EPSILON + weight * dist.pdf(point)
}.sum
math.log(likelihood)
}.reduce(_ + _)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.random.BernoulliCellSampler

/**
* Helper methods to load, save and pre-process data used in ML Lib.
* Helper methods to load, save and pre-process data used in MLLib.
*/
@Since("0.8.0")
object MLUtils extends Logging {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -149,6 +149,32 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext
assert(model.computeLogLikelihood(dataset) === llk)
}

test("check LogLikelihood") {
val rdd = sc.parallelize(1 to 5).map(i => TestRow(Vectors.dense(i, i + 1)))
val dataset = spark.createDataFrame(rdd)

val gaussian1 = new MultivariateGaussian(Vectors.dense(Array(-1.0, -1.0)),
Matrices.dense(2, 2, Array(1.0, 0.0, 0.0, 1.0)))
val gaussian2 = new MultivariateGaussian(Vectors.dense(Array(1.0, 1.0)),
Matrices.dense(2, 2, Array(1.0, 0.0, 0.0, 1.0)))
val model = new GaussianMixtureModel("gmm", Array(0.2, 0.8), Array(gaussian1, gaussian2))

val llk = model.computeLogLikelihood(dataset)
assert(llk ~== -52.804472030823533 relTol 1E-4)
/*
Using the following Python code to compute the log-likelihood:

import numpy as np
from scipy.stats import multivariate_normal
data = np.array([[1, 2], [2, 3], [3, 4], [4, 5], [5, 6]])
pdf1 = multivariate_normal.pdf(data, mean=[-1, -1], cov=np.array([[1, 0], [0, 1]]))
pdf2 = multivariate_normal.pdf(data, mean=[1, 1], cov=np.array([[1, 0], [0, 1]]))
pdf = pdf1 * 0.2 + pdf2 * 0.8
> sum(np.log(pdf))
-52.804472030823533
*/
}

test("read/write") {
def checkModelData(model: GaussianMixtureModel, model2: GaussianMixtureModel): Unit = {
assert(model.weights === model2.weights)
Expand Down