Skip to content

Commit

Permalink
nit
Browse files Browse the repository at this point in the history
nit

opt_blas

opt_blas
  • Loading branch information
zhengruifeng committed May 6, 2020
1 parent 473e5a2 commit 35e007e
Show file tree
Hide file tree
Showing 2 changed files with 40 additions and 43 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -81,41 +81,34 @@ class MultivariateGaussian @Since("2.0.0") (
u - 0.5 * BLAS.dot(v, v)
}

private[ml] def pdf(X: Matrix): Vector = {
val m = X.numRows
val n = X.numCols
val mat = new DenseMatrix(m, n, Array.ofDim[Double](m * n))
private[ml] def pdf(X: Matrix): DenseVector = {
val mat = DenseMatrix.zeros(X.numRows, X.numCols)
pdf(X, mat)
}

private[ml] def pdf(X: Matrix, mat: DenseMatrix): Vector = {
private[ml] def pdf(X: Matrix, mat: DenseMatrix): DenseVector = {
require(!mat.isTransposed)
val localU = u
val localRootSigmaInvMat = rootSigmaInvMat
val localRootSigmaInvMulMu = rootSigmaInvMulMu.toArray

BLAS.gemm(1.0, X, localRootSigmaInvMat.transpose, 0.0, mat)
val arr = mat.values
BLAS.gemm(1.0, X, rootSigmaInvMat.transpose, 0.0, mat)
val m = mat.numRows
val n = mat.numCols

val pdfArr = Array.ofDim[Double](m)
val pdfVec = mat.multiply(rootSigmaInvMulMu)

val blas = BLAS.getBLAS(n)
val squared1 = BLAS.dot(rootSigmaInvMulMu, rootSigmaInvMulMu)

val localU = u
var i = 0
while (i < m) {
var squaredSum = 0.0
var index = i
var j = 0
while (j < n) {
val d = arr(index) - localRootSigmaInvMulMu(j)
squaredSum += d * d
index += m
j += 1
}
pdfArr(i) = math.exp(localU - 0.5 * squaredSum)
val squared2 = blas.ddot(n, mat.values, i, m, mat.values, i, m)
val dot = pdfVec(i)
val squaredSum = squared1 + squared2 - dot - dot
pdfVec.values(i) = math.exp(localU - 0.5 * squaredSum)
i += 1
}

Vectors.dense(pdfArr)
pdfVec
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -376,6 +376,15 @@ class GaussianMixture @Since("2.0.0") (

/**
* Set block size for stacking input data in matrices.
* If blockSize == 1, then stacking will be skipped, and each vector is treated individually;
* If blockSize &gt; 1, then vectors will be stacked to blocks, and high-level BLAS routines
* will be used if possible (for example, GEMV instead of DOT, GEMM instead of GEMV).
* Recommended size is between 10 and 1000. An appropriate choice of the block size depends
* on the sparsity and dim of input datasets, the underlying BLAS implementation (for example,
* f2jBLAS, OpenBLAS, intel MKL) and its configuration (for example, number of threads).
* Note that existing BLAS implementations are mainly optimized for dense matrices, if the
* input dataset is sparse, stacking may bring no performance gain, the worse is possible
* performance regression.
* Default is 1.
*
* @group expertSetParam
Expand Down Expand Up @@ -809,6 +818,7 @@ private class BlockExpectationAggregator(
private lazy val newWeights = Array.ofDim[Double](k)
@transient private lazy val newMeansMat = DenseMatrix.zeros(numFeatures, k)
@transient private lazy val newCovsMat = DenseMatrix.zeros(covSize, k)
@transient private lazy val auxiliaryProbMat = DenseMatrix.zeros(blockSize, k)
@transient private lazy val auxiliaryMat = DenseMatrix.zeros(blockSize, numFeatures)
@transient private lazy val auxiliaryCovVec = Vectors.zeros(covSize).toDense

Expand Down Expand Up @@ -840,15 +850,19 @@ private class BlockExpectationAggregator(
val (matrix: Matrix, weights: Array[Double]) = weightedBlock
require(matrix.isTransposed)
val size = matrix.numRows
val weightArr = bcWeights.value
require(weights.length == size)

val probMat = if (blockSize == size) auxiliaryProbMat else DenseMatrix.zeros(size, k)
require(!probMat.isTransposed)
java.util.Arrays.fill(probMat.values, EPSILON)

val probMat = DenseMatrix.zeros(size, k)
val mat = if (blockSize == size) auxiliaryMat else DenseMatrix.zeros(size, numFeatures)
var j = 0
val blas1 = BLAS.getBLAS(size)
while (j < k) {
val pdfVec = oldGaussians(j).pdf(matrix, mat)
var i = 0
while (i < size) { probMat.update(i, j, EPSILON + weightArr(j) * pdfVec(i)); i += 1 }
blas1.daxpy(size, bcWeights.value(j), pdfVec.values, 0, 1,
probMat.values, j * size, 1)
j += 1
}

Expand All @@ -858,7 +872,7 @@ private class BlockExpectationAggregator(
case dm: DenseMatrix =>
Iterator.tabulate(size) { i =>
java.util.Arrays.fill(covVec.values, 0.0)
// when input block is dense, directly using nativeBLAS to avoid array copy
// when input block is dense, directly use nativeBLAS to avoid array copy
BLAS.nativeBLAS.dspr("U", numFeatures, 1.0, dm.values, i * numFeatures, 1,
covVec.values, 0)
covVec
Expand All @@ -872,25 +886,15 @@ private class BlockExpectationAggregator(
}
}

val probArr = Array.ofDim[Double](k)
val blas2 = BLAS.getBLAS(k)
covVecIter.zip(weights.iterator).zipWithIndex.foreach {
case ((covVec, weight), i) =>
var j = 0
var probSum = 0.0
while (j < k) { probSum += probMat(i, j); j += 1 }
val probSum = blas2.dasum(k, probMat.values, i, size)
blas2.dscal(k, weight / probSum, probMat.values, i, size)
blas2.daxpy(k, 1.0, probMat.values, i, size, newWeights, 0, 1)
BLAS.nativeBLAS.dger(covSize, k, 1.0, covVec.values, 0, 1,
probMat.values, i, size, newCovsMat.values, 0, covSize)
newLogLikelihood += math.log(probSum) * weight

j = 0
while (j < k) {
val w = probMat(i, j) / probSum * weight
newWeights(j) += w
probArr(j) = w
probMat.update(i, j, w)
j += 1
}

BLAS.nativeBLAS.dger(covSize, k, 1.0, covVec.values, 1,
probArr, 1, newCovsMat.values, covSize)
}

BLAS.gemm(1.0, matrix.transpose, probMat, 1.0, newMeansMat)
Expand Down

0 comments on commit 35e007e

Please sign in to comment.