|
17 | 17 |
|
18 | 18 | package org.apache.spark.mllib.linalg.distributed
|
19 | 19 |
|
| 20 | +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, Matrix => BM} |
20 | 21 | import scala.collection.mutable.ArrayBuffer
|
21 | 22 |
|
22 |
| -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, Matrix => BM, SparseVector => BSV, Vector => BV} |
23 |
| - |
24 | 23 | import org.apache.spark.{Partitioner, SparkException}
|
25 | 24 | import org.apache.spark.annotation.Since
|
26 | 25 | import org.apache.spark.internal.Logging
|
27 | 26 | import org.apache.spark.mllib.linalg._
|
28 | 27 | import org.apache.spark.rdd.RDD
|
29 | 28 | import org.apache.spark.storage.StorageLevel
|
30 | 29 |
|
| 30 | + |
31 | 31 | /**
|
32 | 32 | * A grid partitioner, which uses a regular grid to partition coordinates.
|
33 | 33 | *
|
@@ -273,24 +273,37 @@ class BlockMatrix @Since("1.3.0") (
|
273 | 273 | require(cols < Int.MaxValue, s"The number of columns should be less than Int.MaxValue ($cols).")
|
274 | 274 |
|
275 | 275 | val rows = blocks.flatMap { case ((blockRowIdx, blockColIdx), mat) =>
|
276 |
| - mat.rowIter.zipWithIndex.map { |
| 276 | + mat.rowIter.zipWithIndex.filter(_._1.size > 0).map { |
277 | 277 | case (vector, rowIdx) =>
|
278 |
| - blockRowIdx * rowsPerBlock + rowIdx -> ((blockColIdx, vector.asBreeze)) |
| 278 | + blockRowIdx * rowsPerBlock + rowIdx -> ((blockColIdx, vector)) |
279 | 279 | }
|
280 | 280 | }.groupByKey().map { case (rowIdx, vectors) =>
|
281 |
| - val numberNonZeroPerRow = vectors.map(_._2.activeSize).sum.toDouble / cols.toDouble |
282 |
| - |
283 |
| - val wholeVector = if (numberNonZeroPerRow <= 0.1) { // Sparse at 1/10th nnz |
284 |
| - BSV.zeros[Double](cols) |
285 |
| - } else { |
286 |
| - BDV.zeros[Double](cols) |
287 |
| - } |
| 281 | + val numberNonZero = vectors.map(_._2.numActives).sum |
| 282 | + val numberNonZeroPerRow = numberNonZero.toDouble / cols.toDouble |
| 283 | + |
| 284 | + val wholeVector = |
| 285 | + if (numberNonZeroPerRow <= 0.1) { // Sparse at 1/10th nnz |
| 286 | + val arrBufferIndices = new ArrayBuffer[Int](numberNonZero) |
| 287 | + val arrBufferValues = new ArrayBuffer[Double](numberNonZero) |
| 288 | + |
| 289 | + vectors.foreach { case (blockColIdx: Int, vec: Vector) => |
| 290 | + val offset = colsPerBlock * blockColIdx |
| 291 | + vec.foreachActive { case (colIdx: Int, value: Double) => |
| 292 | + arrBufferIndices += offset + colIdx |
| 293 | + arrBufferValues += value |
| 294 | + } |
| 295 | + } |
| 296 | + Vectors.sparse(cols, arrBufferIndices.toArray, arrBufferValues.toArray) |
| 297 | + } else { |
| 298 | + val wholeVectorBuf = BDV.zeros[Double](cols) |
| 299 | + vectors.foreach { case (blockColIdx: Int, vec: Vector) => |
| 300 | + val offset = colsPerBlock * blockColIdx |
| 301 | + wholeVectorBuf(offset until Math.min(cols, offset + colsPerBlock)) := vec.asBreeze |
| 302 | + } |
| 303 | + Vectors.fromBreeze(wholeVectorBuf) |
| 304 | + } |
288 | 305 |
|
289 |
| - vectors.foreach { case (blockColIdx: Int, vec: BV[_]) => |
290 |
| - val offset = colsPerBlock * blockColIdx |
291 |
| - wholeVector(offset until Math.min(cols, offset + colsPerBlock)) := vec |
292 |
| - } |
293 |
| - new IndexedRow(rowIdx, Vectors.fromBreeze(wholeVector)) |
| 306 | + IndexedRow(rowIdx, wholeVector) |
294 | 307 | }
|
295 | 308 | new IndexedRowMatrix(rows)
|
296 | 309 | }
|
|
0 commit comments