You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by MLnick <gi...@git.apache.org> on 2017/05/02 18:48:53 UTC

[GitHub] spark pull request #17742: [Spark-11968][ML][MLLIB]Optimize MLLIB ALS recomm...

Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/17742#discussion_r114395288
  
    --- Diff: mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala ---
    @@ -276,44 +277,53 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] {
           num: Int): RDD[(Int, Array[(Int, Double)])] = {
         val srcBlocks = blockify(rank, srcFeatures)
         val dstBlocks = blockify(rank, dstFeatures)
    -    val ratings = srcBlocks.cartesian(dstBlocks).flatMap {
    -      case ((srcIds, srcFactors), (dstIds, dstFactors)) =>
    -        val m = srcIds.length
    -        val n = dstIds.length
    -        val ratings = srcFactors.transpose.multiply(dstFactors)
    -        val output = new Array[(Int, (Int, Double))](m * n)
    -        var k = 0
    -        ratings.foreachActive { (i, j, r) =>
    -          output(k) = (srcIds(i), (dstIds(j), r))
    -          k += 1
    +    /**
    +     * Use dot to replace blas 3 gemm is the key approach to improve efficiency.
    +     * By this change, we can get the topK elements of each block to reduce the GC time.
    +     * Comparing with BLAS.dot, hand-written dot is high efficiency.
    +     */
    +    val ratings = srcBlocks.cartesian(dstBlocks).flatMap { case (srcIter, dstIter) =>
    +      val m = srcIter.size
    +      val n = math.min(dstIter.size, num)
    +      val output = new Array[(Int, (Int, Double))](m * n)
    +      var j = 0
    +      srcIter.foreach { case (srcId, srcFactor) =>
    +        val pq = new BoundedPriorityQueue[(Int, Double)](n)(Ordering.by(_._2))
    +        dstIter.foreach { case (dstId, dstFactor) =>
    +          /**
    +           * The below code is equivalent to
    +           * val score = blas.ddot(rank, srcFactor, 1, dstFactor, 1)
    +           */
    +          var score: Double = 0
    +          var k = 0
    +          while (k < rank) {
    +            score += srcFactor(k) * dstFactor(k)
    +            k += 1
    +          }
    +          pq += ((dstId, score))
    +        }
    +        val pqIter = pq.iterator
    +        var i = 0
    +        while (i < n) {
    +          output(j + i) = (srcId, pqIter.next())
    +          i += 1
             }
    -        output.toSeq
    +        j += n
    +      }
    +      output.toSeq
         }
         ratings.topByKey(num)(Ordering.by(_._2))
       }
     
       /**
    -   * Blockifies features to use Level-3 BLAS.
    +   * Blockifies features to improve the efficiency of cartesian product
        */
       private def blockify(
           rank: Int,
    -      features: RDD[(Int, Array[Double])]): RDD[(Array[Int], DenseMatrix)] = {
    +      features: RDD[(Int, Array[Double])]): RDD[Seq[(Int, Array[Double])]] = {
    --- End diff --
    
    We can remove `rank` argument here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org