From b693209c7f51ba15b8d68a35755edf0a9a2cb522 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 10 Nov 2014 19:58:52 -0800 Subject: [PATCH 01/21] Ready for Pull request --- .../linalg/distributed/BlockMatrix.scala | 338 ++++++++++++++++++ .../linalg/distributed/BlockMatrixSuite.scala | 120 +++++++ 2 files changed, 458 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala new file mode 100644 index 0000000000000..a98671f431153 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -0,0 +1,338 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.distributed + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark._ +import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices } +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils + +case class BlockPartition( + blockIdRow: Int, + blockIdCol: Int, + mat: DenseMatrix) extends Serializable + +// Information about BlockMatrix maintained on the driver +case class BlockPartitionInfo( + partitionId: Int, + blockIdRow: Int, + blockIdCol: Int, + startRow: Long, + numRows: Int, + startCol: Long, + numCols: Int) extends Serializable + +abstract class BlockMatrixPartitioner( + override val numPartitions: Int, + val rowPerBlock: Int, + val colPerBlock: Int) extends Partitioner { + val name: String + + override def getPartition(key: Any): Int = { + Utils.nonNegativeMod(key.asInstanceOf[Int], numPartitions) + } +} + +class GridPartitioner( + val numRowBlocks: Int, + val numColBlocks: Int, + override val rowPerBlock: Int, + override val colPerBlock: Int) + extends BlockMatrixPartitioner(numRowBlocks * numColBlocks, rowPerBlock, colPerBlock) { + + override val name = "grid" + + override val numPartitions = numRowBlocks * numColBlocks + + override def equals(obj: Any): Boolean = { + obj match { + case r: GridPartitioner => + (this.numPartitions == r.numPartitions) && (this.rowPerBlock == r.rowPerBlock) && + (this.colPerBlock == r.colPerBlock) + case _ => + false + } + } +} + +class RowBasedPartitioner( + override val numPartitions: Int, + override val rowPerBlock: Int, + override val colPerBlock: Int) + extends BlockMatrixPartitioner(numPartitions, rowPerBlock, colPerBlock) { + + override val name = "row" + + override def equals(obj: Any): Boolean = { + obj match { + case r: RowBasedPartitioner => + (this.numPartitions == r.numPartitions) && (this.rowPerBlock == r.rowPerBlock) && + (this.colPerBlock == r.colPerBlock) + case _ => + false + } + } +} + +class ColumnBasedPartitioner( + override val numPartitions: Int, + override val rowPerBlock: Int, + override val colPerBlock: Int) + extends BlockMatrixPartitioner(numPartitions, rowPerBlock, colPerBlock) { + + override val name = "column" + + override def equals(obj: Any): Boolean = { + obj match { + case p: ColumnBasedPartitioner => + (this.numPartitions == p.numPartitions) && (this.rowPerBlock == p.rowPerBlock) && + (this.colPerBlock == p.colPerBlock) + case r: RowBasedPartitioner => + (this.numPartitions == r.numPartitions) && (this.colPerBlock == r.rowPerBlock) + case _ => + false + } + } +} + +class BlockMatrix( + val numRowBlocks: Int, + val numColBlocks: Int, + val rdd: RDD[BlockPartition], + val partitioner: BlockMatrixPartitioner) extends DistributedMatrix with Logging { + + // We need a key-value pair RDD to partition properly + private var matrixRDD = rdd.map { block => + partitioner match { + case r: RowBasedPartitioner => (block.blockIdRow, block) + case c: ColumnBasedPartitioner => (block.blockIdCol, block) + case g: GridPartitioner => (block.blockIdRow + numRowBlocks * block.blockIdCol, block) + case _ => throw new IllegalArgumentException("Unrecognized partitioner") + } + } + + @transient var blockInfo_ : Map[(Int, Int), BlockPartitionInfo] = null + + lazy val dims: (Long, Long) = getDim + + override def numRows(): Long = dims._1 + override def numCols(): Long = dims._2 + + if (partitioner.name.equals("column")) { + require(numColBlocks == partitioner.numPartitions) + } else if (partitioner.name.equals("row")) { + require(numRowBlocks == partitioner.numPartitions) + } else if (partitioner.name.equals("grid")) { + require(numRowBlocks * numColBlocks == partitioner.numPartitions) + } else { + throw new IllegalArgumentException("Unrecognized partitioner.") + } + + def getDim: (Long, Long) = { + val bi = getBlockInfo + val xDim = bi.map { x => + (x._1._1, x._2.numRows.toLong) + }.groupBy(x => x._1).values.map { x => + x.head._2.toLong + }.reduceLeft { + _ + _ + } + + val yDim = bi.map { x => + (x._1._2, x._2.numCols.toLong) + }.groupBy(x => x._1).values.map { x => + x.head._2.toLong + }.reduceLeft { + _ + _ + } + + (xDim, yDim) + } + + private def calculateBlockInfo(): Unit = { + + // collect may cause akka frameSize errors + val blockStartRowColsParts = matrixRDD.mapPartitionsWithIndex { case (partId, iter) => + iter.map { case (id, block) => + ((block.blockIdRow, block.blockIdCol), (partId, block.mat.numRows, block.mat.numCols)) + } + }.collect() + val blockStartRowCols = blockStartRowColsParts.sortBy(_._1) + + // Group blockInfo by rowId, pick the first row and sort on rowId + val rowReps = blockStartRowCols.groupBy(_._1._1).values.map(_.head).toSeq.sortBy(_._1._1) + + // Group blockInfo by columnId, pick the first column and sort on columnId + val colReps = blockStartRowCols.groupBy(_._1._2).values.map(_.head).toSeq.sortBy(_._1._2) + + // Calculate startRows + val cumulativeRowSum = rowReps.scanLeft((0, 0L)) { case (x1, x2) => + (x1._1 + 1, x1._2 + x2._2._2) + }.toMap + + val cumulativeColSum = colReps.scanLeft((0, 0L)) { case (x1, x2) => + (x1._1 + 1, x1._2 + x2._2._3) + }.toMap + + blockInfo_ = blockStartRowCols.map{ case ((rowId, colId), (partId, numRow, numCol)) => + ((rowId, colId), new BlockPartitionInfo(partId, rowId, colId, cumulativeRowSum(rowId), numRow, + cumulativeColSum(colId), numCol)) + }.toMap + } + + def getBlockInfo: Map[(Int, Int), BlockPartitionInfo] = { + if (blockInfo_ == null) { + calculateBlockInfo() + } + blockInfo_ + } + + def normFro(): Double = { + math.sqrt(rdd.map(lm => lm.mat.values.map(x => math.pow(x, 2)).sum).reduce(_ + _)) + } + + /* Cache the underlying RDD. */ + def cache(): DistributedMatrix = { + matrixRDD.cache() + this + } + + /* Set the storage level for the underlying RDD. */ + def persist(storageLevel: StorageLevel): DistributedMatrix = { + matrixRDD.persist(storageLevel) + this + } + + def repartition(part: BlockMatrixPartitioner = partitioner): DistributedMatrix = { + matrixRDD = matrixRDD.partitionBy(part) + this + } + + /* Collect the distributed matrix on the driver. */ + def collect(): DenseMatrix = { + val parts = rdd.map(x => ((x.blockIdRow, x.blockIdCol), x.mat)). + collect().sortBy(x => (x._1._2, x._1._1)) + val nRows = numRows().toInt + val nCols = numCols().toInt + val values = new Array[Double](nRows * nCols) + val blockInfos = getBlockInfo + parts.foreach { part => + val blockInfo = blockInfos((part._1._1, part._1._2)) + // Figure out where this part should be put + var j = 0 + while (j < blockInfo.numCols) { + var i = 0 + val indStart = (j + blockInfo.startCol.toInt) * nRows + blockInfo.startRow.toInt + val indEnd = blockInfo.numRows + val matStart = j * blockInfo.numRows + val mat = part._2.values + while (i < indEnd) { + values(indStart + i) = mat(matStart + i) + i += 1 + } + j += 1 + } + } + new DenseMatrix(nRows, nCols, values) + } + + private[mllib] def toBreeze(): BDM[Double] = { + val localMat = collect() + new BDM[Double](localMat.numRows, localMat.numCols, localMat.values) + } + + def add(other: DistributedMatrix): DistributedMatrix = { + other match { + // We really need a function to check if two matrices are partitioned similarly + case otherBlocked: BlockMatrix => + if (checkPartitioning(otherBlocked, OperationNames.add)){ + val addedBlocks = rdd.zip(otherBlocked.rdd).map{ case (a, b) => + val result = a.mat.toBreeze + b.mat.toBreeze + new BlockPartition(a.blockIdRow, a.blockIdCol, + Matrices.fromBreeze(result).asInstanceOf[DenseMatrix]) + } + new BlockMatrix(numRowBlocks, numColBlocks, addedBlocks, partitioner) + } else { + throw new SparkException( + "Cannot add matrices with non-matching partitioners") + } + case _ => + throw new IllegalArgumentException("Cannot add matrices of different types") + } + } + + def multiply(other: DistributedMatrix): BlockMatrix = { + other match { + case otherBlocked: BlockMatrix => + if (checkPartitioning(otherBlocked, OperationNames.multiply)){ + + val resultPartitioner = new GridPartitioner(numRowBlocks, otherBlocked.numColBlocks, + partitioner.rowPerBlock, otherBlocked.partitioner.colPerBlock) + + val multiplyBlocks = matrixRDD.join(otherBlocked.matrixRDD, partitioner). + map { case (key, (mat1, mat2)) => + val C = mat1.mat multiply mat2.mat + (mat1.blockIdRow + numRowBlocks * mat2.blockIdCol, C.toBreeze) + }.reduceByKey(resultPartitioner, (a, b) => a + b) + + val newBlocks = multiplyBlocks.map{ case (index, mat) => + val colId = index / numRowBlocks + val rowId = index - colId * numRowBlocks + new BlockPartition(rowId, colId, Matrices.fromBreeze(mat).asInstanceOf[DenseMatrix]) + } + new BlockMatrix(numRowBlocks, otherBlocked.numColBlocks, newBlocks, resultPartitioner) + } else { + throw new SparkException( + "Cannot multiply matrices with non-matching partitioners") + } + case _ => + throw new IllegalArgumentException("Cannot add matrices of different types") + } + } + + private def checkPartitioning(other: BlockMatrix, operation: Int): Boolean = { + val otherPartitioner = other.partitioner + operation match { + case OperationNames.add => + partitioner.equals(otherPartitioner) + case OperationNames.multiply => + partitioner.name == "column" && otherPartitioner.name == "row" && + partitioner.numPartitions == otherPartitioner.numPartitions && + partitioner.colPerBlock == otherPartitioner.rowPerBlock && + numColBlocks == other.numRowBlocks + case _ => + throw new IllegalArgumentException("Unsupported operation") + } + } +} + +/** + * Maintains supported and default block matrix operation names. + * + * Currently supported operations: `add`, `multiply`. + */ +private object OperationNames { + + val add: Int = 1 + val multiply: Int = 2 + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala new file mode 100644 index 0000000000000..3cc907f2fbff4 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.distributed + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices} +import org.apache.spark.mllib.util.LocalSparkContext + +class BlockMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 5 + val n = 4 + val rowPerPart = 2 + val colPerPart = 2 + val numRowBlocks = 3 + val numColBlocks = 2 + var rowBasedMat: BlockMatrix = _ + var colBasedMat: BlockMatrix = _ + var gridBasedMat: BlockMatrix = _ + + override def beforeAll() { + super.beforeAll() + val entries: Seq[BlockPartition] = Seq( + new BlockPartition(0, 0, new DenseMatrix(2, 2, Array(1.0, 0.0, 0.0, 2.0))), + new BlockPartition(0, 1, new DenseMatrix(2, 2, Array(0.0, 1.0, 0.0, 0.0))), + new BlockPartition(1, 0, new DenseMatrix(2, 2, Array(3.0, 0.0, 1.5, 0.0))), + new BlockPartition(1, 1, new DenseMatrix(2, 2, Array(1.0, 4.0, 0.0, 1.0))), + new BlockPartition(2, 0, new DenseMatrix(1, 2, Array(1.0, 0.0))), + new BlockPartition(2, 1, new DenseMatrix(1, 2, Array(1.0, 5.0)))) + + val colPart = new ColumnBasedPartitioner(numColBlocks, rowPerPart, colPerPart) + val rowPart = new RowBasedPartitioner(numRowBlocks, rowPerPart, colPerPart) + val gridPart = new GridPartitioner(numRowBlocks, numColBlocks, rowPerPart, colPerPart) + + colBasedMat = + new BlockMatrix(numRowBlocks, numColBlocks, sc.parallelize(entries, numColBlocks), colPart) + rowBasedMat = + new BlockMatrix(numRowBlocks, numColBlocks, sc.parallelize(entries, numRowBlocks), rowPart) + gridBasedMat = + new BlockMatrix(numRowBlocks, numColBlocks, + sc.parallelize(entries, numRowBlocks * numColBlocks), gridPart) + } + + test("size") { + assert(colBasedMat.numRows() === m) + assert(colBasedMat.numCols() === n) + assert(rowBasedMat.numRows() === m) + assert(rowBasedMat.numCols() === n) + assert(gridBasedMat.numRows() === m) + assert(gridBasedMat.numCols() === n) + } + + test("toBreeze and collect") { + val expected = BDM( + (1.0, 0.0, 0.0, 0.0), + (0.0, 2.0, 1.0, 0.0), + (3.0, 1.5, 1.0, 0.0), + (0.0, 0.0, 4.0, 1.0), + (1.0, 0.0, 1.0, 5.0)) + + val dense = Matrices.fromBreeze(expected).asInstanceOf[DenseMatrix] + assert(colBasedMat.toBreeze() === expected) + assert(rowBasedMat.toBreeze() === expected) + assert(gridBasedMat.toBreeze() === expected) + assert(colBasedMat.collect() === dense) + assert(rowBasedMat.collect() === dense) + assert(gridBasedMat.collect() === dense) + } + + test("blockInfo") { + val colMatInfo = colBasedMat.getBlockInfo + val rowMatInfo = rowBasedMat.getBlockInfo + val gridMatInfo = gridBasedMat.getBlockInfo + + assert(colMatInfo((0, 1)).numRows === 2) + assert(colMatInfo((0, 1)).numCols === 2) + assert(colMatInfo((0, 1)).startRow === 0) + assert(colMatInfo((0, 1)).startCol === 2) + assert(colMatInfo((2, 0)).numRows === 1) + assert(colMatInfo((2, 0)).numCols === 2) + assert(colMatInfo((2, 0)).startRow === 4) + assert(colMatInfo((2, 0)).startCol === 0) + + assert(rowMatInfo((0, 1)).numRows === 2) + assert(rowMatInfo((0, 1)).numCols === 2) + assert(rowMatInfo((0, 1)).startRow === 0) + assert(rowMatInfo((0, 1)).startCol === 2) + assert(rowMatInfo((2, 0)).numRows === 1) + assert(rowMatInfo((2, 0)).numCols === 2) + assert(rowMatInfo((2, 0)).startRow === 4) + assert(rowMatInfo((2, 0)).startCol === 0) + + assert(gridMatInfo((0, 1)).numRows === 2) + assert(gridMatInfo((0, 1)).numCols === 2) + assert(gridMatInfo((0, 1)).startRow === 0) + assert(gridMatInfo((0, 1)).startCol === 2) + assert(gridMatInfo((2, 0)).numRows === 1) + assert(gridMatInfo((2, 0)).numCols === 2) + assert(gridMatInfo((2, 0)).startRow === 4) + assert(gridMatInfo((2, 0)).startCol === 0) + } +} From f378e163b04dad88f6e4fe309e45a5a632aa4101 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 10 Nov 2014 21:26:34 -0800 Subject: [PATCH 02/21] [SPARK-3974] Block Matrix Abstractions ready --- .../linalg/distributed/BlockMatrix.scala | 183 ++++++++---------- 1 file changed, 85 insertions(+), 98 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index a98671f431153..0a4837c29de3e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -20,18 +20,32 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark._ -import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices } +import org.apache.spark.mllib.linalg.DenseMatrix import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils -case class BlockPartition( - blockIdRow: Int, - blockIdCol: Int, - mat: DenseMatrix) extends Serializable +/** + * Represents a local matrix that makes up one block of a distributed BlockMatrix + * + * @param blockIdRow The row index of this block + * @param blockIdCol The column index of this block + * @param mat The underlying local matrix + */ +case class BlockPartition(blockIdRow: Int, blockIdCol: Int, mat: DenseMatrix) extends Serializable -// Information about BlockMatrix maintained on the driver +/** + * Information about the BlockMatrix maintained on the driver + * + * @param partitionId The id of the partition the block is found in + * @param blockIdRow The row index of this block + * @param blockIdCol The column index of this block + * @param startRow The starting row index with respect to the distributed BlockMatrix + * @param numRows The number of rows in this block + * @param startCol The starting column index with respect to the distributed BlockMatrix + * @param numCols The number of columns in this block + */ case class BlockPartitionInfo( partitionId: Int, blockIdRow: Int, @@ -41,6 +55,13 @@ case class BlockPartitionInfo( startCol: Long, numCols: Int) extends Serializable +/** + * A partitioner that decides how the matrix is distributed in the cluster + * + * @param numPartitions Number of partitions + * @param rowPerBlock Number of rows that make up each block. + * @param colPerBlock Number of columns that make up each block. + */ abstract class BlockMatrixPartitioner( override val numPartitions: Int, val rowPerBlock: Int, @@ -52,6 +73,14 @@ abstract class BlockMatrixPartitioner( } } +/** + * A grid partitioner, which stores every block in a separate partition. + * + * @param numRowBlocks Number of blocks that form the rows of the matrix. + * @param numColBlocks Number of blocks that form the columns of the matrix. + * @param rowPerBlock Number of rows that make up each block. + * @param colPerBlock Number of columns that make up each block. + */ class GridPartitioner( val numRowBlocks: Int, val numColBlocks: Int, @@ -74,6 +103,14 @@ class GridPartitioner( } } +/** + * A specialized partitioner that stores all blocks in the same row in just one partition. + * + * @param numPartitions Number of partitions. Should be set as the number of blocks that form + * the rows of the matrix. + * @param rowPerBlock Number of rows that make up each block. + * @param colPerBlock Number of columns that make up each block. + */ class RowBasedPartitioner( override val numPartitions: Int, override val rowPerBlock: Int, @@ -93,6 +130,14 @@ class RowBasedPartitioner( } } +/** + * A specialized partitioner that stores all blocks in the same column in just one partition. + * + * @param numPartitions Number of partitions. Should be set as the number of blocks that form + * the columns of the matrix. + * @param rowPerBlock Number of rows that make up each block. + * @param colPerBlock Number of columns that make up each block. + */ class ColumnBasedPartitioner( override val numPartitions: Int, override val rowPerBlock: Int, @@ -114,39 +159,44 @@ class ColumnBasedPartitioner( } } +/** + * Represents a distributed matrix in blocks of local matrices. + * + * @param numRowBlocks Number of blocks that form the rows of this matrix + * @param numColBlocks Number of blocks that form the columns of this matrix + * @param rdd The RDD of BlockPartitions (local matrices) that form this matrix + * @param partitioner A partitioner that specifies how BlockPartitions are stored in the cluster + */ class BlockMatrix( val numRowBlocks: Int, val numColBlocks: Int, val rdd: RDD[BlockPartition], val partitioner: BlockMatrixPartitioner) extends DistributedMatrix with Logging { - // We need a key-value pair RDD to partition properly - private var matrixRDD = rdd.map { block => - partitioner match { - case r: RowBasedPartitioner => (block.blockIdRow, block) - case c: ColumnBasedPartitioner => (block.blockIdCol, block) - case g: GridPartitioner => (block.blockIdRow + numRowBlocks * block.blockIdCol, block) - case _ => throw new IllegalArgumentException("Unrecognized partitioner") - } - } + // A key-value pair RDD is required to partition properly + private var matrixRDD: RDD[(Int, BlockPartition)] = keyBy() @transient var blockInfo_ : Map[(Int, Int), BlockPartitionInfo] = null - lazy val dims: (Long, Long) = getDim + private lazy val dims: (Long, Long) = getDim override def numRows(): Long = dims._1 override def numCols(): Long = dims._2 if (partitioner.name.equals("column")) { - require(numColBlocks == partitioner.numPartitions) + require(numColBlocks == partitioner.numPartitions, "The number of column blocks should match" + + " the number of partitions of the column partitioner.") } else if (partitioner.name.equals("row")) { - require(numRowBlocks == partitioner.numPartitions) + require(numRowBlocks == partitioner.numPartitions, "The number of row blocks should match" + + " the number of partitions of the row partitioner.") } else if (partitioner.name.equals("grid")) { - require(numRowBlocks * numColBlocks == partitioner.numPartitions) + require(numRowBlocks * numColBlocks == partitioner.numPartitions, "The number of blocks " + + "should match the number of partitions of the grid partitioner.") } else { throw new IllegalArgumentException("Unrecognized partitioner.") } + /* Returns the dimensions of the matrix. */ def getDim: (Long, Long) = { val bi = getBlockInfo val xDim = bi.map { x => @@ -194,11 +244,12 @@ class BlockMatrix( }.toMap blockInfo_ = blockStartRowCols.map{ case ((rowId, colId), (partId, numRow, numCol)) => - ((rowId, colId), new BlockPartitionInfo(partId, rowId, colId, cumulativeRowSum(rowId), numRow, - cumulativeColSum(colId), numCol)) + ((rowId, colId), new BlockPartitionInfo(partId, rowId, colId, cumulativeRowSum(rowId), + numRow, cumulativeColSum(colId), numCol)) }.toMap } + /* Returns a map of the information of the blocks that form the distributed matrix. */ def getBlockInfo: Map[(Int, Int), BlockPartitionInfo] = { if (blockInfo_ == null) { calculateBlockInfo() @@ -206,6 +257,7 @@ class BlockMatrix( blockInfo_ } + /* Returns the Frobenius Norm of the matrix */ def normFro(): Double = { math.sqrt(rdd.map(lm => lm.mat.values.map(x => math.pow(x, 2)).sum).reduce(_ + _)) } @@ -222,8 +274,19 @@ class BlockMatrix( this } + private def keyBy(part: BlockMatrixPartitioner = partitioner): RDD[(Int, BlockPartition)] = { + rdd.map { block => + part match { + case r: RowBasedPartitioner => (block.blockIdRow, block) + case c: ColumnBasedPartitioner => (block.blockIdCol, block) + case g: GridPartitioner => (block.blockIdRow + numRowBlocks * block.blockIdCol, block) + case _ => throw new IllegalArgumentException("Unrecognized partitioner") + } + } + } + def repartition(part: BlockMatrixPartitioner = partitioner): DistributedMatrix = { - matrixRDD = matrixRDD.partitionBy(part) + matrixRDD = keyBy(part) this } @@ -259,80 +322,4 @@ class BlockMatrix( val localMat = collect() new BDM[Double](localMat.numRows, localMat.numCols, localMat.values) } - - def add(other: DistributedMatrix): DistributedMatrix = { - other match { - // We really need a function to check if two matrices are partitioned similarly - case otherBlocked: BlockMatrix => - if (checkPartitioning(otherBlocked, OperationNames.add)){ - val addedBlocks = rdd.zip(otherBlocked.rdd).map{ case (a, b) => - val result = a.mat.toBreeze + b.mat.toBreeze - new BlockPartition(a.blockIdRow, a.blockIdCol, - Matrices.fromBreeze(result).asInstanceOf[DenseMatrix]) - } - new BlockMatrix(numRowBlocks, numColBlocks, addedBlocks, partitioner) - } else { - throw new SparkException( - "Cannot add matrices with non-matching partitioners") - } - case _ => - throw new IllegalArgumentException("Cannot add matrices of different types") - } - } - - def multiply(other: DistributedMatrix): BlockMatrix = { - other match { - case otherBlocked: BlockMatrix => - if (checkPartitioning(otherBlocked, OperationNames.multiply)){ - - val resultPartitioner = new GridPartitioner(numRowBlocks, otherBlocked.numColBlocks, - partitioner.rowPerBlock, otherBlocked.partitioner.colPerBlock) - - val multiplyBlocks = matrixRDD.join(otherBlocked.matrixRDD, partitioner). - map { case (key, (mat1, mat2)) => - val C = mat1.mat multiply mat2.mat - (mat1.blockIdRow + numRowBlocks * mat2.blockIdCol, C.toBreeze) - }.reduceByKey(resultPartitioner, (a, b) => a + b) - - val newBlocks = multiplyBlocks.map{ case (index, mat) => - val colId = index / numRowBlocks - val rowId = index - colId * numRowBlocks - new BlockPartition(rowId, colId, Matrices.fromBreeze(mat).asInstanceOf[DenseMatrix]) - } - new BlockMatrix(numRowBlocks, otherBlocked.numColBlocks, newBlocks, resultPartitioner) - } else { - throw new SparkException( - "Cannot multiply matrices with non-matching partitioners") - } - case _ => - throw new IllegalArgumentException("Cannot add matrices of different types") - } - } - - private def checkPartitioning(other: BlockMatrix, operation: Int): Boolean = { - val otherPartitioner = other.partitioner - operation match { - case OperationNames.add => - partitioner.equals(otherPartitioner) - case OperationNames.multiply => - partitioner.name == "column" && otherPartitioner.name == "row" && - partitioner.numPartitions == otherPartitioner.numPartitions && - partitioner.colPerBlock == otherPartitioner.rowPerBlock && - numColBlocks == other.numRowBlocks - case _ => - throw new IllegalArgumentException("Unsupported operation") - } - } -} - -/** - * Maintains supported and default block matrix operation names. - * - * Currently supported operations: `add`, `multiply`. - */ -private object OperationNames { - - val add: Int = 1 - val multiply: Int = 2 - } From aa8f086ad7ac311660a2b0efee18827bb7d834f6 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 11 Nov 2014 11:36:22 -0800 Subject: [PATCH 03/21] [SPARK-3974] Additional comments added --- .../spark/mllib/linalg/distributed/BlockMatrix.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 0a4837c29de3e..3c85af9ae6536 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -22,7 +22,6 @@ import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark._ import org.apache.spark.mllib.linalg.DenseMatrix import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -218,8 +217,8 @@ class BlockMatrix( (xDim, yDim) } + /* Calculates the information for each block and collects it on the driver */ private def calculateBlockInfo(): Unit = { - // collect may cause akka frameSize errors val blockStartRowColsParts = matrixRDD.mapPartitionsWithIndex { case (partId, iter) => iter.map { case (id, block) => @@ -274,6 +273,7 @@ class BlockMatrix( this } + /* Add a key to the underlying rdd for partitioning and joins. */ private def keyBy(part: BlockMatrixPartitioner = partitioner): RDD[(Int, BlockPartition)] = { rdd.map { block => part match { @@ -285,6 +285,12 @@ class BlockMatrix( } } + /** + * Repartition the BlockMatrix using a different partitioner. + * + * @param part The partitioner to partition by + * @return The repartitioned BlockMatrix + */ def repartition(part: BlockMatrixPartitioner = partitioner): DistributedMatrix = { matrixRDD = keyBy(part) this From 589fbb65478851d88ea5a7f5bf54c1fa8d53f055 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 14 Nov 2014 11:16:09 -0800 Subject: [PATCH 04/21] [SPARK-3974] Code review feedback addressed --- .../linalg/distributed/BlockMatrix.scala | 71 ++++++++++--------- 1 file changed, 39 insertions(+), 32 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 3c85af9ae6536..6b8931e9f2b1b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -32,10 +32,10 @@ import org.apache.spark.util.Utils * @param blockIdCol The column index of this block * @param mat The underlying local matrix */ -case class BlockPartition(blockIdRow: Int, blockIdCol: Int, mat: DenseMatrix) extends Serializable +case class SubMatrix(blockIdRow: Int, blockIdCol: Int, mat: DenseMatrix) extends Serializable /** - * Information about the BlockMatrix maintained on the driver + * Information of the submatrices of the BlockMatrix maintained on the driver * * @param partitionId The id of the partition the block is found in * @param blockIdRow The row index of this block @@ -45,7 +45,7 @@ case class BlockPartition(blockIdRow: Int, blockIdCol: Int, mat: DenseMatrix) ex * @param startCol The starting column index with respect to the distributed BlockMatrix * @param numCols The number of columns in this block */ -case class BlockPartitionInfo( +case class SubMatrixInfo( partitionId: Int, blockIdRow: Int, blockIdCol: Int, @@ -67,6 +67,13 @@ abstract class BlockMatrixPartitioner( val colPerBlock: Int) extends Partitioner { val name: String + /** + * Returns the index of the partition the SubMatrix belongs to. + * + * @param key The key for the SubMatrix. Can be its row index, column index or position in the + * grid. + * @return The index of the partition, which the SubMatrix belongs to. + */ override def getPartition(key: Any): Int = { Utils.nonNegativeMod(key.asInstanceOf[Int], numPartitions) } @@ -91,6 +98,7 @@ class GridPartitioner( override val numPartitions = numRowBlocks * numColBlocks + /** Checks whether the partitioners have the same characteristics */ override def equals(obj: Any): Boolean = { obj match { case r: GridPartitioner => @@ -118,6 +126,7 @@ class RowBasedPartitioner( override val name = "row" + /** Checks whether the partitioners have the same characteristics */ override def equals(obj: Any): Boolean = { obj match { case r: RowBasedPartitioner => @@ -145,6 +154,7 @@ class ColumnBasedPartitioner( override val name = "column" + /** Checks whether the partitioners have the same characteristics */ override def equals(obj: Any): Boolean = { obj match { case p: ColumnBasedPartitioner => @@ -163,19 +173,19 @@ class ColumnBasedPartitioner( * * @param numRowBlocks Number of blocks that form the rows of this matrix * @param numColBlocks Number of blocks that form the columns of this matrix - * @param rdd The RDD of BlockPartitions (local matrices) that form this matrix - * @param partitioner A partitioner that specifies how BlockPartitions are stored in the cluster + * @param rdd The RDD of SubMatrixs (local matrices) that form this matrix + * @param partitioner A partitioner that specifies how SubMatrixs are stored in the cluster */ class BlockMatrix( val numRowBlocks: Int, val numColBlocks: Int, - val rdd: RDD[BlockPartition], + val rdd: RDD[SubMatrix], val partitioner: BlockMatrixPartitioner) extends DistributedMatrix with Logging { // A key-value pair RDD is required to partition properly - private var matrixRDD: RDD[(Int, BlockPartition)] = keyBy() + private var matrixRDD: RDD[(Int, SubMatrix)] = keyBy() - @transient var blockInfo_ : Map[(Int, Int), BlockPartitionInfo] = null + @transient var blockInfo_ : Map[(Int, Int), SubMatrixInfo] = null private lazy val dims: (Long, Long) = getDim @@ -184,40 +194,36 @@ class BlockMatrix( if (partitioner.name.equals("column")) { require(numColBlocks == partitioner.numPartitions, "The number of column blocks should match" + - " the number of partitions of the column partitioner.") + s" the number of partitions of the column partitioner. numColBlocks: $numColBlocks, " + + s"partitioner.numPartitions: ${partitioner.numPartitions}") } else if (partitioner.name.equals("row")) { require(numRowBlocks == partitioner.numPartitions, "The number of row blocks should match" + - " the number of partitions of the row partitioner.") + s" the number of partitions of the row partitioner. numRowBlocks: $numRowBlocks, " + + s"partitioner.numPartitions: ${partitioner.numPartitions}") } else if (partitioner.name.equals("grid")) { require(numRowBlocks * numColBlocks == partitioner.numPartitions, "The number of blocks " + - "should match the number of partitions of the grid partitioner.") + s"should match the number of partitions of the grid partitioner. numRowBlocks * " + + s"numColBlocks: ${numRowBlocks * numColBlocks}, " + + s"partitioner.numPartitions: ${partitioner.numPartitions}") } else { throw new IllegalArgumentException("Unrecognized partitioner.") } - /* Returns the dimensions of the matrix. */ + /** Returns the dimensions of the matrix. */ def getDim: (Long, Long) = { val bi = getBlockInfo val xDim = bi.map { x => (x._1._1, x._2.numRows.toLong) - }.groupBy(x => x._1).values.map { x => - x.head._2.toLong - }.reduceLeft { - _ + _ - } + }.groupBy(x => x._1).values.map(_.head._2.toLong).reduceLeft(_ + _) val yDim = bi.map { x => (x._1._2, x._2.numCols.toLong) - }.groupBy(x => x._1).values.map { x => - x.head._2.toLong - }.reduceLeft { - _ + _ - } + }.groupBy(x => x._1).values.map(_.head._2.toLong).reduceLeft(_ + _) (xDim, yDim) } - /* Calculates the information for each block and collects it on the driver */ + /** Calculates the information for each block and collects it on the driver */ private def calculateBlockInfo(): Unit = { // collect may cause akka frameSize errors val blockStartRowColsParts = matrixRDD.mapPartitionsWithIndex { case (partId, iter) => @@ -243,38 +249,38 @@ class BlockMatrix( }.toMap blockInfo_ = blockStartRowCols.map{ case ((rowId, colId), (partId, numRow, numCol)) => - ((rowId, colId), new BlockPartitionInfo(partId, rowId, colId, cumulativeRowSum(rowId), + ((rowId, colId), new SubMatrixInfo(partId, rowId, colId, cumulativeRowSum(rowId), numRow, cumulativeColSum(colId), numCol)) }.toMap } - /* Returns a map of the information of the blocks that form the distributed matrix. */ - def getBlockInfo: Map[(Int, Int), BlockPartitionInfo] = { + /** Returns a map of the information of the blocks that form the distributed matrix. */ + def getBlockInfo: Map[(Int, Int), SubMatrixInfo] = { if (blockInfo_ == null) { calculateBlockInfo() } blockInfo_ } - /* Returns the Frobenius Norm of the matrix */ + /** Returns the Frobenius Norm of the matrix */ def normFro(): Double = { math.sqrt(rdd.map(lm => lm.mat.values.map(x => math.pow(x, 2)).sum).reduce(_ + _)) } - /* Cache the underlying RDD. */ + /** Cache the underlying RDD. */ def cache(): DistributedMatrix = { matrixRDD.cache() this } - /* Set the storage level for the underlying RDD. */ + /** Set the storage level for the underlying RDD. */ def persist(storageLevel: StorageLevel): DistributedMatrix = { matrixRDD.persist(storageLevel) this } - /* Add a key to the underlying rdd for partitioning and joins. */ - private def keyBy(part: BlockMatrixPartitioner = partitioner): RDD[(Int, BlockPartition)] = { + /** Add a key to the underlying rdd for partitioning and joins. */ + private def keyBy(part: BlockMatrixPartitioner = partitioner): RDD[(Int, SubMatrix)] = { rdd.map { block => part match { case r: RowBasedPartitioner => (block.blockIdRow, block) @@ -296,7 +302,7 @@ class BlockMatrix( this } - /* Collect the distributed matrix on the driver. */ + /** Collect the distributed matrix on the driver. */ def collect(): DenseMatrix = { val parts = rdd.map(x => ((x.blockIdRow, x.blockIdCol), x.mat)). collect().sortBy(x => (x._1._2, x._1._1)) @@ -324,6 +330,7 @@ class BlockMatrix( new DenseMatrix(nRows, nCols, values) } + /** Collects data and assembles a local dense breeze matrix (for test only). */ private[mllib] def toBreeze(): BDM[Double] = { val localMat = collect() new BDM[Double](localMat.numRows, localMat.numCols, localMat.values) From 19c17e8d1594a3f7bd5a973a09b341de3a1c857a Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 14 Nov 2014 11:24:23 -0800 Subject: [PATCH 05/21] [SPARK-3974] Changed blockIdRow and blockIdCol --- .../linalg/distributed/BlockMatrix.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 6b8931e9f2b1b..d78880f622f18 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -28,18 +28,18 @@ import org.apache.spark.util.Utils /** * Represents a local matrix that makes up one block of a distributed BlockMatrix * - * @param blockIdRow The row index of this block - * @param blockIdCol The column index of this block + * @param blockRowIndex The row index of this block + * @param blockColIndex The column index of this block * @param mat The underlying local matrix */ -case class SubMatrix(blockIdRow: Int, blockIdCol: Int, mat: DenseMatrix) extends Serializable +case class SubMatrix(blockRowIndex: Int, blockColIndex: Int, mat: DenseMatrix) extends Serializable /** * Information of the submatrices of the BlockMatrix maintained on the driver * * @param partitionId The id of the partition the block is found in - * @param blockIdRow The row index of this block - * @param blockIdCol The column index of this block + * @param blockRowIndex The row index of this block + * @param blockColIndex The column index of this block * @param startRow The starting row index with respect to the distributed BlockMatrix * @param numRows The number of rows in this block * @param startCol The starting column index with respect to the distributed BlockMatrix @@ -47,8 +47,8 @@ case class SubMatrix(blockIdRow: Int, blockIdCol: Int, mat: DenseMatrix) extends */ case class SubMatrixInfo( partitionId: Int, - blockIdRow: Int, - blockIdCol: Int, + blockRowIndex: Int, + blockColIndex: Int, startRow: Long, numRows: Int, startCol: Long, @@ -228,7 +228,7 @@ class BlockMatrix( // collect may cause akka frameSize errors val blockStartRowColsParts = matrixRDD.mapPartitionsWithIndex { case (partId, iter) => iter.map { case (id, block) => - ((block.blockIdRow, block.blockIdCol), (partId, block.mat.numRows, block.mat.numCols)) + ((block.blockRowIndex, block.blockColIndex), (partId, block.mat.numRows, block.mat.numCols)) } }.collect() val blockStartRowCols = blockStartRowColsParts.sortBy(_._1) @@ -283,9 +283,9 @@ class BlockMatrix( private def keyBy(part: BlockMatrixPartitioner = partitioner): RDD[(Int, SubMatrix)] = { rdd.map { block => part match { - case r: RowBasedPartitioner => (block.blockIdRow, block) - case c: ColumnBasedPartitioner => (block.blockIdCol, block) - case g: GridPartitioner => (block.blockIdRow + numRowBlocks * block.blockIdCol, block) + case r: RowBasedPartitioner => (block.blockRowIndex, block) + case c: ColumnBasedPartitioner => (block.blockColIndex, block) + case g: GridPartitioner => (block.blockRowIndex + numRowBlocks * block.blockColIndex, block) case _ => throw new IllegalArgumentException("Unrecognized partitioner") } } @@ -304,7 +304,7 @@ class BlockMatrix( /** Collect the distributed matrix on the driver. */ def collect(): DenseMatrix = { - val parts = rdd.map(x => ((x.blockIdRow, x.blockIdCol), x.mat)). + val parts = rdd.map(x => ((x.blockRowIndex, x.blockColIndex), x.mat)). collect().sortBy(x => (x._1._2, x._1._1)) val nRows = numRows().toInt val nCols = numCols().toInt From b05aabbdd5c3f5db69bbaff3582139b691d696fa Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 14 Nov 2014 12:17:49 -0800 Subject: [PATCH 06/21] [SPARK-3974] Updated tests to reflect changes --- .../linalg/distributed/BlockMatrixSuite.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index 3cc907f2fbff4..c6551e9afdc5d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -38,13 +38,13 @@ class BlockMatrixSuite extends FunSuite with LocalSparkContext { override def beforeAll() { super.beforeAll() - val entries: Seq[BlockPartition] = Seq( - new BlockPartition(0, 0, new DenseMatrix(2, 2, Array(1.0, 0.0, 0.0, 2.0))), - new BlockPartition(0, 1, new DenseMatrix(2, 2, Array(0.0, 1.0, 0.0, 0.0))), - new BlockPartition(1, 0, new DenseMatrix(2, 2, Array(3.0, 0.0, 1.5, 0.0))), - new BlockPartition(1, 1, new DenseMatrix(2, 2, Array(1.0, 4.0, 0.0, 1.0))), - new BlockPartition(2, 0, new DenseMatrix(1, 2, Array(1.0, 0.0))), - new BlockPartition(2, 1, new DenseMatrix(1, 2, Array(1.0, 5.0)))) + val entries: Seq[SubMatrix] = Seq( + new SubMatrix(0, 0, new DenseMatrix(2, 2, Array(1.0, 0.0, 0.0, 2.0))), + new SubMatrix(0, 1, new DenseMatrix(2, 2, Array(0.0, 1.0, 0.0, 0.0))), + new SubMatrix(1, 0, new DenseMatrix(2, 2, Array(3.0, 0.0, 1.5, 0.0))), + new SubMatrix(1, 1, new DenseMatrix(2, 2, Array(1.0, 4.0, 0.0, 1.0))), + new SubMatrix(2, 0, new DenseMatrix(1, 2, Array(1.0, 0.0))), + new SubMatrix(2, 1, new DenseMatrix(1, 2, Array(1.0, 5.0)))) val colPart = new ColumnBasedPartitioner(numColBlocks, rowPerPart, colPerPart) val rowPart = new RowBasedPartitioner(numRowBlocks, rowPerPart, colPerPart) From 49b9586a18cf7338a46c88a51ed23890914d3be4 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 14 Nov 2014 12:44:19 -0800 Subject: [PATCH 07/21] [SPARK-3974] Updated testing utils from master --- .../spark/mllib/linalg/distributed/BlockMatrixSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index c6551e9afdc5d..a2de71a318fad 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -22,9 +22,9 @@ import org.scalatest.FunSuite import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices} -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.MLlibTestSparkContext -class BlockMatrixSuite extends FunSuite with LocalSparkContext { +class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { val m = 5 val n = 4 From d033861d5a2f88b223f601feb4445308399901e8 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 14 Nov 2014 16:18:50 -0800 Subject: [PATCH 08/21] [SPARK-3974] Removed SubMatrixInfo and added constructor without partitioner --- .../linalg/distributed/BlockMatrix.scala | 127 +++++++----------- .../linalg/distributed/BlockMatrixSuite.scala | 39 +----- 2 files changed, 48 insertions(+), 118 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index d78880f622f18..17fcfaf4d0fe1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -21,6 +21,7 @@ import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark._ import org.apache.spark.mllib.linalg.DenseMatrix +import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -28,32 +29,12 @@ import org.apache.spark.util.Utils /** * Represents a local matrix that makes up one block of a distributed BlockMatrix * - * @param blockRowIndex The row index of this block - * @param blockColIndex The column index of this block + * @param blockRowIndex The row index of this block. Must be zero based. + * @param blockColIndex The column index of this block. Must be zero based. * @param mat The underlying local matrix */ case class SubMatrix(blockRowIndex: Int, blockColIndex: Int, mat: DenseMatrix) extends Serializable -/** - * Information of the submatrices of the BlockMatrix maintained on the driver - * - * @param partitionId The id of the partition the block is found in - * @param blockRowIndex The row index of this block - * @param blockColIndex The column index of this block - * @param startRow The starting row index with respect to the distributed BlockMatrix - * @param numRows The number of rows in this block - * @param startCol The starting column index with respect to the distributed BlockMatrix - * @param numCols The number of columns in this block - */ -case class SubMatrixInfo( - partitionId: Int, - blockRowIndex: Int, - blockColIndex: Int, - startRow: Long, - numRows: Int, - startCol: Long, - numCols: Int) extends Serializable - /** * A partitioner that decides how the matrix is distributed in the cluster * @@ -61,7 +42,7 @@ case class SubMatrixInfo( * @param rowPerBlock Number of rows that make up each block. * @param colPerBlock Number of columns that make up each block. */ -abstract class BlockMatrixPartitioner( +private[mllib] abstract class BlockMatrixPartitioner( override val numPartitions: Int, val rowPerBlock: Int, val colPerBlock: Int) extends Partitioner { @@ -173,8 +154,8 @@ class ColumnBasedPartitioner( * * @param numRowBlocks Number of blocks that form the rows of this matrix * @param numColBlocks Number of blocks that form the columns of this matrix - * @param rdd The RDD of SubMatrixs (local matrices) that form this matrix - * @param partitioner A partitioner that specifies how SubMatrixs are stored in the cluster + * @param rdd The RDD of SubMatrices (local matrices) that form this matrix + * @param partitioner A partitioner that specifies how SubMatrices are stored in the cluster */ class BlockMatrix( val numRowBlocks: Int, @@ -182,11 +163,21 @@ class BlockMatrix( val rdd: RDD[SubMatrix], val partitioner: BlockMatrixPartitioner) extends DistributedMatrix with Logging { + /** + * Alternate constructor for BlockMatrix without the input of a partitioner. Will use a Grid + * Partitioner by default. + * + * @param numRowBlocks Number of blocks that form the rows of this matrix + * @param numColBlocks Number of blocks that form the columns of this matrix + * @param rdd The RDD of SubMatrices (local matrices) that form this matrix + */ + def this(numRowBlocks: Int, numColBlocks: Int, rdd: RDD[SubMatrix]) = { + this(numRowBlocks, numColBlocks, rdd, new GridPartitioner(numRowBlocks, numColBlocks, + rdd.first().mat.numRows, rdd.first().mat.numCols)) + } // A key-value pair RDD is required to partition properly private var matrixRDD: RDD[(Int, SubMatrix)] = keyBy() - @transient var blockInfo_ : Map[(Int, Int), SubMatrixInfo] = null - private lazy val dims: (Long, Long) = getDim override def numRows(): Long = dims._1 @@ -211,55 +202,26 @@ class BlockMatrix( /** Returns the dimensions of the matrix. */ def getDim: (Long, Long) = { - val bi = getBlockInfo - val xDim = bi.map { x => - (x._1._1, x._2.numRows.toLong) - }.groupBy(x => x._1).values.map(_.head._2.toLong).reduceLeft(_ + _) - - val yDim = bi.map { x => - (x._1._2, x._2.numCols.toLong) - }.groupBy(x => x._1).values.map(_.head._2.toLong).reduceLeft(_ + _) - (xDim, yDim) - } - - /** Calculates the information for each block and collects it on the driver */ - private def calculateBlockInfo(): Unit = { - // collect may cause akka frameSize errors - val blockStartRowColsParts = matrixRDD.mapPartitionsWithIndex { case (partId, iter) => - iter.map { case (id, block) => - ((block.blockRowIndex, block.blockColIndex), (partId, block.mat.numRows, block.mat.numCols)) + val firstRowColumn = rdd.filter(block => block.blockRowIndex == 0 || block.blockColIndex == 0). + map { block => + ((block.blockRowIndex, block.blockColIndex), (block.mat.numRows, block.mat.numCols)) } - }.collect() - val blockStartRowCols = blockStartRowColsParts.sortBy(_._1) - - // Group blockInfo by rowId, pick the first row and sort on rowId - val rowReps = blockStartRowCols.groupBy(_._1._1).values.map(_.head).toSeq.sortBy(_._1._1) - - // Group blockInfo by columnId, pick the first column and sort on columnId - val colReps = blockStartRowCols.groupBy(_._1._2).values.map(_.head).toSeq.sortBy(_._1._2) - - // Calculate startRows - val cumulativeRowSum = rowReps.scanLeft((0, 0L)) { case (x1, x2) => - (x1._1 + 1, x1._2 + x2._2._2) - }.toMap - val cumulativeColSum = colReps.scanLeft((0, 0L)) { case (x1, x2) => - (x1._1 + 1, x1._2 + x2._2._3) - }.toMap - - blockInfo_ = blockStartRowCols.map{ case ((rowId, colId), (partId, numRow, numCol)) => - ((rowId, colId), new SubMatrixInfo(partId, rowId, colId, cumulativeRowSum(rowId), - numRow, cumulativeColSum(colId), numCol)) - }.toMap - } - - /** Returns a map of the information of the blocks that form the distributed matrix. */ - def getBlockInfo: Map[(Int, Int), SubMatrixInfo] = { - if (blockInfo_ == null) { - calculateBlockInfo() - } - blockInfo_ + firstRowColumn.treeAggregate((0L, 0L))( + seqOp = (c, v) => (c, v) match { case ((x_dim, y_dim), ((indX, indY), (nRow, nCol))) => + if (indX == 0 && indY == 0) { + (x_dim + nRow, y_dim + nCol) + } else if (indX == 0) { + (x_dim, y_dim + nCol) + } else { + (x_dim + nRow, y_dim) + } + }, + combOp = (c1, c2) => (c1, c2) match { + case ((x_dim1, y_dim1), (x_dim2, y_dim2)) => + (x_dim1 + x_dim2, y_dim1 + y_dim2) + }) } /** Returns the Frobenius Norm of the matrix */ @@ -309,23 +271,26 @@ class BlockMatrix( val nRows = numRows().toInt val nCols = numCols().toInt val values = new Array[Double](nRows * nCols) - val blockInfos = getBlockInfo + var rowStart = 0 + var colStart = 0 parts.foreach { part => - val blockInfo = blockInfos((part._1._1, part._1._2)) - // Figure out where this part should be put + if (part._1._1 == 0) rowStart = 0 + val block = part._2 var j = 0 - while (j < blockInfo.numCols) { + while (j < block.numCols) { var i = 0 - val indStart = (j + blockInfo.startCol.toInt) * nRows + blockInfo.startRow.toInt - val indEnd = blockInfo.numRows - val matStart = j * blockInfo.numRows - val mat = part._2.values + val indStart = (j + colStart) * nRows + rowStart + val indEnd = block.numRows + val matStart = j * block.numRows + val mat = block.values while (i < indEnd) { values(indStart + i) = mat(matStart + i) i += 1 } j += 1 } + rowStart += block.numRows + if (part._1._1 == numRowBlocks - 1) colStart += block.numCols } new DenseMatrix(nRows, nCols, values) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index a2de71a318fad..5c74a15f64bf9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -48,15 +48,13 @@ class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { val colPart = new ColumnBasedPartitioner(numColBlocks, rowPerPart, colPerPart) val rowPart = new RowBasedPartitioner(numRowBlocks, rowPerPart, colPerPart) - val gridPart = new GridPartitioner(numRowBlocks, numColBlocks, rowPerPart, colPerPart) colBasedMat = new BlockMatrix(numRowBlocks, numColBlocks, sc.parallelize(entries, numColBlocks), colPart) rowBasedMat = new BlockMatrix(numRowBlocks, numColBlocks, sc.parallelize(entries, numRowBlocks), rowPart) - gridBasedMat = - new BlockMatrix(numRowBlocks, numColBlocks, - sc.parallelize(entries, numRowBlocks * numColBlocks), gridPart) + gridBasedMat = new BlockMatrix(numRowBlocks, numColBlocks, + sc.parallelize(entries, numRowBlocks * numColBlocks)) } test("size") { @@ -84,37 +82,4 @@ class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { assert(rowBasedMat.collect() === dense) assert(gridBasedMat.collect() === dense) } - - test("blockInfo") { - val colMatInfo = colBasedMat.getBlockInfo - val rowMatInfo = rowBasedMat.getBlockInfo - val gridMatInfo = gridBasedMat.getBlockInfo - - assert(colMatInfo((0, 1)).numRows === 2) - assert(colMatInfo((0, 1)).numCols === 2) - assert(colMatInfo((0, 1)).startRow === 0) - assert(colMatInfo((0, 1)).startCol === 2) - assert(colMatInfo((2, 0)).numRows === 1) - assert(colMatInfo((2, 0)).numCols === 2) - assert(colMatInfo((2, 0)).startRow === 4) - assert(colMatInfo((2, 0)).startCol === 0) - - assert(rowMatInfo((0, 1)).numRows === 2) - assert(rowMatInfo((0, 1)).numCols === 2) - assert(rowMatInfo((0, 1)).startRow === 0) - assert(rowMatInfo((0, 1)).startCol === 2) - assert(rowMatInfo((2, 0)).numRows === 1) - assert(rowMatInfo((2, 0)).numCols === 2) - assert(rowMatInfo((2, 0)).startRow === 4) - assert(rowMatInfo((2, 0)).startCol === 0) - - assert(gridMatInfo((0, 1)).numRows === 2) - assert(gridMatInfo((0, 1)).numCols === 2) - assert(gridMatInfo((0, 1)).startRow === 0) - assert(gridMatInfo((0, 1)).startCol === 2) - assert(gridMatInfo((2, 0)).numRows === 1) - assert(gridMatInfo((2, 0)).numCols === 2) - assert(gridMatInfo((2, 0)).startRow === 4) - assert(gridMatInfo((2, 0)).startCol === 0) - } } From 9ae85aa1ebabdc099d7f655bc1d9021d34d2910f Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 20 Nov 2014 11:58:58 -0800 Subject: [PATCH 09/21] [SPARK-3974] Made partitioner a variable inside BlockMatrix instead of a constructor variable --- .../linalg/distributed/BlockMatrix.scala | 33 +++++++++++++++---- .../linalg/distributed/BlockMatrixSuite.scala | 14 ++++++++ 2 files changed, 40 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 17fcfaf4d0fe1..7b4e61b534454 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -155,13 +155,11 @@ class ColumnBasedPartitioner( * @param numRowBlocks Number of blocks that form the rows of this matrix * @param numColBlocks Number of blocks that form the columns of this matrix * @param rdd The RDD of SubMatrices (local matrices) that form this matrix - * @param partitioner A partitioner that specifies how SubMatrices are stored in the cluster */ class BlockMatrix( val numRowBlocks: Int, val numColBlocks: Int, - val rdd: RDD[SubMatrix], - val partitioner: BlockMatrixPartitioner) extends DistributedMatrix with Logging { + val rdd: RDD[SubMatrix]) extends DistributedMatrix with Logging { /** * Alternate constructor for BlockMatrix without the input of a partitioner. Will use a Grid @@ -170,11 +168,31 @@ class BlockMatrix( * @param numRowBlocks Number of blocks that form the rows of this matrix * @param numColBlocks Number of blocks that form the columns of this matrix * @param rdd The RDD of SubMatrices (local matrices) that form this matrix + * @param partitioner A partitioner that specifies how SubMatrices are stored in the cluster */ - def this(numRowBlocks: Int, numColBlocks: Int, rdd: RDD[SubMatrix]) = { - this(numRowBlocks, numColBlocks, rdd, new GridPartitioner(numRowBlocks, numColBlocks, - rdd.first().mat.numRows, rdd.first().mat.numCols)) + def this( + numRowBlocks: Int, + numColBlocks: Int, + rdd: RDD[SubMatrix], + partitioner: BlockMatrixPartitioner) = { + this(numRowBlocks, numColBlocks, rdd) + setPartitioner(partitioner) } + + private[mllib] var partitioner: BlockMatrixPartitioner = { + val firstSubMatrix = rdd.first().mat + new GridPartitioner(numRowBlocks, numColBlocks, + firstSubMatrix.numRows, firstSubMatrix.numCols) + } + + /** + * Set the partitioner for the matrix. For internal use only. Users should use `repartition`. + * @param part A partitioner that specifies how SubMatrices are stored in the cluster + */ + private def setPartitioner(part: BlockMatrixPartitioner): Unit = { + partitioner = part + } + // A key-value pair RDD is required to partition properly private var matrixRDD: RDD[(Int, SubMatrix)] = keyBy() @@ -259,8 +277,9 @@ class BlockMatrix( * @param part The partitioner to partition by * @return The repartitioned BlockMatrix */ - def repartition(part: BlockMatrixPartitioner = partitioner): DistributedMatrix = { + def repartition(part: BlockMatrixPartitioner): DistributedMatrix = { matrixRDD = keyBy(part) + setPartitioner(part) this } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index 5c74a15f64bf9..1d9ff1112ddb4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -66,6 +66,20 @@ class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { assert(gridBasedMat.numCols() === n) } + test("partitioner and repartition") { + assert(colBasedMat.partitioner.name === "column") + assert(rowBasedMat.partitioner.name === "row") + assert(gridBasedMat.partitioner.name === "grid") + + val colPart = new ColumnBasedPartitioner(numColBlocks, rowPerPart, colPerPart) + val rowPart = new RowBasedPartitioner(numRowBlocks, rowPerPart, colPerPart) + gridBasedMat.repartition(rowPart).asInstanceOf[BlockMatrix] + assert(gridBasedMat.partitioner.name === "row") + + gridBasedMat.repartition(colPart).asInstanceOf[BlockMatrix] + assert(gridBasedMat.partitioner.name === "column") + } + test("toBreeze and collect") { val expected = BDM( (1.0, 0.0, 0.0, 0.0), From ab6cde0d90b917e89f97c86ef3c84dcdc64a9b57 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 13 Jan 2015 19:54:55 -0800 Subject: [PATCH 10/21] [SPARK-3974] Modifications cleaning code up, making size calculation more robust --- .../linalg/distributed/BlockMatrix.scala | 239 +++++------------- .../linalg/distributed/BlockMatrixSuite.scala | 54 +--- 2 files changed, 78 insertions(+), 215 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 7b4e61b534454..a4b2bf42390f2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -20,97 +20,51 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark._ -import org.apache.spark.mllib.linalg.DenseMatrix +import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils /** - * Represents a local matrix that makes up one block of a distributed BlockMatrix - * - * @param blockRowIndex The row index of this block. Must be zero based. - * @param blockColIndex The column index of this block. Must be zero based. - * @param mat The underlying local matrix - */ -case class SubMatrix(blockRowIndex: Int, blockColIndex: Int, mat: DenseMatrix) extends Serializable - -/** - * A partitioner that decides how the matrix is distributed in the cluster + * A grid partitioner, which stores every block in a separate partition. * - * @param numPartitions Number of partitions + * @param numRowBlocks Number of blocks that form the rows of the matrix. + * @param numColBlocks Number of blocks that form the columns of the matrix. * @param rowPerBlock Number of rows that make up each block. * @param colPerBlock Number of columns that make up each block. */ -private[mllib] abstract class BlockMatrixPartitioner( - override val numPartitions: Int, +private[mllib] class GridPartitioner( + val numRowBlocks: Int, + val numColBlocks: Int, val rowPerBlock: Int, - val colPerBlock: Int) extends Partitioner { - val name: String + val colPerBlock: Int, + override val numPartitions: Int) extends Partitioner { /** * Returns the index of the partition the SubMatrix belongs to. * - * @param key The key for the SubMatrix. Can be its row index, column index or position in the - * grid. + * @param key The key for the SubMatrix. Can be its position in the grid (its column major index) + * or a tuple of three integers that are the final row index after the multiplication, + * the index of the block to multiply with, and the final column index after the + * multiplication. * @return The index of the partition, which the SubMatrix belongs to. */ override def getPartition(key: Any): Int = { - Utils.nonNegativeMod(key.asInstanceOf[Int], numPartitions) - } -} - -/** - * A grid partitioner, which stores every block in a separate partition. - * - * @param numRowBlocks Number of blocks that form the rows of the matrix. - * @param numColBlocks Number of blocks that form the columns of the matrix. - * @param rowPerBlock Number of rows that make up each block. - * @param colPerBlock Number of columns that make up each block. - */ -class GridPartitioner( - val numRowBlocks: Int, - val numColBlocks: Int, - override val rowPerBlock: Int, - override val colPerBlock: Int) - extends BlockMatrixPartitioner(numRowBlocks * numColBlocks, rowPerBlock, colPerBlock) { - - override val name = "grid" - - override val numPartitions = numRowBlocks * numColBlocks - - /** Checks whether the partitioners have the same characteristics */ - override def equals(obj: Any): Boolean = { - obj match { - case r: GridPartitioner => - (this.numPartitions == r.numPartitions) && (this.rowPerBlock == r.rowPerBlock) && - (this.colPerBlock == r.colPerBlock) + key match { + case ind: (Int, Int) => + Utils.nonNegativeMod(ind._1 + ind._2 * numRowBlocks, numPartitions) + case indices: (Int, Int, Int) => + Utils.nonNegativeMod(indices._1 + indices._3 * numRowBlocks, numPartitions) case _ => - false + throw new IllegalArgumentException("Unrecognized key") } } -} - -/** - * A specialized partitioner that stores all blocks in the same row in just one partition. - * - * @param numPartitions Number of partitions. Should be set as the number of blocks that form - * the rows of the matrix. - * @param rowPerBlock Number of rows that make up each block. - * @param colPerBlock Number of columns that make up each block. - */ -class RowBasedPartitioner( - override val numPartitions: Int, - override val rowPerBlock: Int, - override val colPerBlock: Int) - extends BlockMatrixPartitioner(numPartitions, rowPerBlock, colPerBlock) { - - override val name = "row" /** Checks whether the partitioners have the same characteristics */ override def equals(obj: Any): Boolean = { obj match { - case r: RowBasedPartitioner => + case r: GridPartitioner => (this.numPartitions == r.numPartitions) && (this.rowPerBlock == r.rowPerBlock) && (this.colPerBlock == r.colPerBlock) case _ => @@ -119,36 +73,6 @@ class RowBasedPartitioner( } } -/** - * A specialized partitioner that stores all blocks in the same column in just one partition. - * - * @param numPartitions Number of partitions. Should be set as the number of blocks that form - * the columns of the matrix. - * @param rowPerBlock Number of rows that make up each block. - * @param colPerBlock Number of columns that make up each block. - */ -class ColumnBasedPartitioner( - override val numPartitions: Int, - override val rowPerBlock: Int, - override val colPerBlock: Int) - extends BlockMatrixPartitioner(numPartitions, rowPerBlock, colPerBlock) { - - override val name = "column" - - /** Checks whether the partitioners have the same characteristics */ - override def equals(obj: Any): Boolean = { - obj match { - case p: ColumnBasedPartitioner => - (this.numPartitions == p.numPartitions) && (this.rowPerBlock == p.rowPerBlock) && - (this.colPerBlock == p.colPerBlock) - case r: RowBasedPartitioner => - (this.numPartitions == r.numPartitions) && (this.colPerBlock == r.rowPerBlock) - case _ => - false - } - } -} - /** * Represents a distributed matrix in blocks of local matrices. * @@ -159,7 +83,9 @@ class ColumnBasedPartitioner( class BlockMatrix( val numRowBlocks: Int, val numColBlocks: Int, - val rdd: RDD[SubMatrix]) extends DistributedMatrix with Logging { + val rdd: RDD[((Int, Int), Matrix)]) extends DistributedMatrix with Logging { + + type SubMatrix = ((Int, Int), Matrix) // ((blockRowIndex, blockColIndex), matrix) /** * Alternate constructor for BlockMatrix without the input of a partitioner. Will use a Grid @@ -168,125 +94,92 @@ class BlockMatrix( * @param numRowBlocks Number of blocks that form the rows of this matrix * @param numColBlocks Number of blocks that form the columns of this matrix * @param rdd The RDD of SubMatrices (local matrices) that form this matrix - * @param partitioner A partitioner that specifies how SubMatrices are stored in the cluster + * @param rowPerBlock Number of rows that make up each block. + * @param colPerBlock Number of columns that make up each block. */ def this( numRowBlocks: Int, numColBlocks: Int, - rdd: RDD[SubMatrix], - partitioner: BlockMatrixPartitioner) = { + rdd: RDD[((Int, Int), Matrix)], + rowPerBlock: Int, + colPerBlock: Int) = { this(numRowBlocks, numColBlocks, rdd) - setPartitioner(partitioner) + val part = new GridPartitioner(numRowBlocks, numColBlocks, rowPerBlock, colPerBlock, rdd.partitions.length) + setPartitioner(part) } - private[mllib] var partitioner: BlockMatrixPartitioner = { - val firstSubMatrix = rdd.first().mat + private[mllib] var partitioner: GridPartitioner = { + val firstSubMatrix = rdd.first()._2 new GridPartitioner(numRowBlocks, numColBlocks, - firstSubMatrix.numRows, firstSubMatrix.numCols) + firstSubMatrix.numRows, firstSubMatrix.numCols, rdd.partitions.length) } /** * Set the partitioner for the matrix. For internal use only. Users should use `repartition`. * @param part A partitioner that specifies how SubMatrices are stored in the cluster */ - private def setPartitioner(part: BlockMatrixPartitioner): Unit = { + private def setPartitioner(part: GridPartitioner): Unit = { partitioner = part } - // A key-value pair RDD is required to partition properly - private var matrixRDD: RDD[(Int, SubMatrix)] = keyBy() - private lazy val dims: (Long, Long) = getDim override def numRows(): Long = dims._1 override def numCols(): Long = dims._2 - if (partitioner.name.equals("column")) { - require(numColBlocks == partitioner.numPartitions, "The number of column blocks should match" + - s" the number of partitions of the column partitioner. numColBlocks: $numColBlocks, " + - s"partitioner.numPartitions: ${partitioner.numPartitions}") - } else if (partitioner.name.equals("row")) { - require(numRowBlocks == partitioner.numPartitions, "The number of row blocks should match" + - s" the number of partitions of the row partitioner. numRowBlocks: $numRowBlocks, " + - s"partitioner.numPartitions: ${partitioner.numPartitions}") - } else if (partitioner.name.equals("grid")) { - require(numRowBlocks * numColBlocks == partitioner.numPartitions, "The number of blocks " + - s"should match the number of partitions of the grid partitioner. numRowBlocks * " + - s"numColBlocks: ${numRowBlocks * numColBlocks}, " + - s"partitioner.numPartitions: ${partitioner.numPartitions}") - } else { - throw new IllegalArgumentException("Unrecognized partitioner.") - } - /** Returns the dimensions of the matrix. */ def getDim: (Long, Long) = { - - val firstRowColumn = rdd.filter(block => block.blockRowIndex == 0 || block.blockColIndex == 0). - map { block => - ((block.blockRowIndex, block.blockColIndex), (block.mat.numRows, block.mat.numCols)) + // picks the sizes of the matrix with the maximum indices + def pickSizeByGreaterIndex(example: (Int, Int, Int, Int), base: (Int, Int, Int, Int)): (Int, Int, Int, Int) = { + if (example._1 > base._1 && example._2 > base._2) { + (example._1, example._2, example._3, example._4) + } else if (example._1 > base._1) { + (example._1, base._2, example._3, base._4) + } else if (example._2 > base._2) { + (base._1, example._2, base._3, example._4) + } else { + (base._1, base._2, base._3, base._4) } + } - firstRowColumn.treeAggregate((0L, 0L))( - seqOp = (c, v) => (c, v) match { case ((x_dim, y_dim), ((indX, indY), (nRow, nCol))) => - if (indX == 0 && indY == 0) { - (x_dim + nRow, y_dim + nCol) - } else if (indX == 0) { - (x_dim, y_dim + nCol) - } else { - (x_dim + nRow, y_dim) - } + val lastRowCol = rdd.treeAggregate((0, 0, 0, 0))( + seqOp = (c, v) => (c, v) match { case (base, ((blockXInd, blockYInd), mat)) => + pickSizeByGreaterIndex((blockXInd, blockYInd, mat.numRows, mat.numCols), base) }, combOp = (c1, c2) => (c1, c2) match { - case ((x_dim1, y_dim1), (x_dim2, y_dim2)) => - (x_dim1 + x_dim2, y_dim1 + y_dim2) + case (res1, res2) => + pickSizeByGreaterIndex(res1, res2) }) + + (lastRowCol._1.toLong * partitioner.rowPerBlock + lastRowCol._3, + lastRowCol._2.toLong * partitioner.colPerBlock + lastRowCol._4) } /** Returns the Frobenius Norm of the matrix */ def normFro(): Double = { - math.sqrt(rdd.map(lm => lm.mat.values.map(x => math.pow(x, 2)).sum).reduce(_ + _)) + math.sqrt(rdd.map { + case sparse: ((Int, Int), SparseMatrix) => + sparse._2.values.map(x => math.pow(x, 2)).sum + case dense: ((Int, Int), DenseMatrix) => + dense._2.values.map(x => math.pow(x, 2)).sum + }.reduce(_ + _)) } /** Cache the underlying RDD. */ def cache(): DistributedMatrix = { - matrixRDD.cache() + rdd.cache() this } /** Set the storage level for the underlying RDD. */ def persist(storageLevel: StorageLevel): DistributedMatrix = { - matrixRDD.persist(storageLevel) - this - } - - /** Add a key to the underlying rdd for partitioning and joins. */ - private def keyBy(part: BlockMatrixPartitioner = partitioner): RDD[(Int, SubMatrix)] = { - rdd.map { block => - part match { - case r: RowBasedPartitioner => (block.blockRowIndex, block) - case c: ColumnBasedPartitioner => (block.blockColIndex, block) - case g: GridPartitioner => (block.blockRowIndex + numRowBlocks * block.blockColIndex, block) - case _ => throw new IllegalArgumentException("Unrecognized partitioner") - } - } - } - - /** - * Repartition the BlockMatrix using a different partitioner. - * - * @param part The partitioner to partition by - * @return The repartitioned BlockMatrix - */ - def repartition(part: BlockMatrixPartitioner): DistributedMatrix = { - matrixRDD = keyBy(part) - setPartitioner(part) + rdd.persist(storageLevel) this } /** Collect the distributed matrix on the driver. */ - def collect(): DenseMatrix = { - val parts = rdd.map(x => ((x.blockRowIndex, x.blockColIndex), x.mat)). - collect().sortBy(x => (x._1._2, x._1._1)) + def toLocalMatrix(): Matrix = { + val parts = rdd.collect().sortBy(x => (x._1._2, x._1._1)) val nRows = numRows().toInt val nCols = numCols().toInt val values = new Array[Double](nRows * nCols) @@ -301,7 +194,7 @@ class BlockMatrix( val indStart = (j + colStart) * nRows + rowStart val indEnd = block.numRows val matStart = j * block.numRows - val mat = block.values + val mat = block.toArray while (i < indEnd) { values(indStart + i) = mat(matStart + i) i += 1 @@ -316,7 +209,7 @@ class BlockMatrix( /** Collects data and assembles a local dense breeze matrix (for test only). */ private[mllib] def toBreeze(): BDM[Double] = { - val localMat = collect() - new BDM[Double](localMat.numRows, localMat.numCols, localMat.values) + val localMat = toLocalMatrix() + new BDM[Double](localMat.numRows, localMat.numCols, localMat.toArray) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index 1d9ff1112ddb4..45152f473c16d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -21,7 +21,7 @@ import org.scalatest.FunSuite import breeze.linalg.{DenseMatrix => BDM} -import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices} +import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Matrix} import org.apache.spark.mllib.util.MLlibTestSparkContext class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { @@ -32,55 +32,29 @@ class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { val colPerPart = 2 val numRowBlocks = 3 val numColBlocks = 2 - var rowBasedMat: BlockMatrix = _ - var colBasedMat: BlockMatrix = _ var gridBasedMat: BlockMatrix = _ + type SubMatrix = ((Int, Int), Matrix) override def beforeAll() { super.beforeAll() - val entries: Seq[SubMatrix] = Seq( - new SubMatrix(0, 0, new DenseMatrix(2, 2, Array(1.0, 0.0, 0.0, 2.0))), - new SubMatrix(0, 1, new DenseMatrix(2, 2, Array(0.0, 1.0, 0.0, 0.0))), - new SubMatrix(1, 0, new DenseMatrix(2, 2, Array(3.0, 0.0, 1.5, 0.0))), - new SubMatrix(1, 1, new DenseMatrix(2, 2, Array(1.0, 4.0, 0.0, 1.0))), - new SubMatrix(2, 0, new DenseMatrix(1, 2, Array(1.0, 0.0))), - new SubMatrix(2, 1, new DenseMatrix(1, 2, Array(1.0, 5.0)))) - val colPart = new ColumnBasedPartitioner(numColBlocks, rowPerPart, colPerPart) - val rowPart = new RowBasedPartitioner(numRowBlocks, rowPerPart, colPerPart) + val entries: Seq[SubMatrix] = Seq( + new SubMatrix((0, 0), new DenseMatrix(2, 2, Array(1.0, 0.0, 0.0, 2.0))), + new SubMatrix((0, 1), new DenseMatrix(2, 2, Array(0.0, 1.0, 0.0, 0.0))), + new SubMatrix((1, 0), new DenseMatrix(2, 2, Array(3.0, 0.0, 1.5, 0.0))), + new SubMatrix((1, 1), new DenseMatrix(2, 2, Array(1.0, 4.0, 0.0, 1.0))), + new SubMatrix((2, 0), new DenseMatrix(1, 2, Array(1.0, 0.0))), + new SubMatrix((2, 1), new DenseMatrix(1, 2, Array(1.0, 5.0)))) - colBasedMat = - new BlockMatrix(numRowBlocks, numColBlocks, sc.parallelize(entries, numColBlocks), colPart) - rowBasedMat = - new BlockMatrix(numRowBlocks, numColBlocks, sc.parallelize(entries, numRowBlocks), rowPart) - gridBasedMat = new BlockMatrix(numRowBlocks, numColBlocks, - sc.parallelize(entries, numRowBlocks * numColBlocks)) + gridBasedMat = new BlockMatrix(numRowBlocks, numColBlocks, sc.parallelize(entries, 2)) } test("size") { - assert(colBasedMat.numRows() === m) - assert(colBasedMat.numCols() === n) - assert(rowBasedMat.numRows() === m) - assert(rowBasedMat.numCols() === n) assert(gridBasedMat.numRows() === m) assert(gridBasedMat.numCols() === n) } - test("partitioner and repartition") { - assert(colBasedMat.partitioner.name === "column") - assert(rowBasedMat.partitioner.name === "row") - assert(gridBasedMat.partitioner.name === "grid") - - val colPart = new ColumnBasedPartitioner(numColBlocks, rowPerPart, colPerPart) - val rowPart = new RowBasedPartitioner(numRowBlocks, rowPerPart, colPerPart) - gridBasedMat.repartition(rowPart).asInstanceOf[BlockMatrix] - assert(gridBasedMat.partitioner.name === "row") - - gridBasedMat.repartition(colPart).asInstanceOf[BlockMatrix] - assert(gridBasedMat.partitioner.name === "column") - } - - test("toBreeze and collect") { + test("toBreeze and toLocalMatrix") { val expected = BDM( (1.0, 0.0, 0.0, 0.0), (0.0, 2.0, 1.0, 0.0), @@ -89,11 +63,7 @@ class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { (1.0, 0.0, 1.0, 5.0)) val dense = Matrices.fromBreeze(expected).asInstanceOf[DenseMatrix] - assert(colBasedMat.toBreeze() === expected) - assert(rowBasedMat.toBreeze() === expected) assert(gridBasedMat.toBreeze() === expected) - assert(colBasedMat.collect() === dense) - assert(rowBasedMat.collect() === dense) - assert(gridBasedMat.collect() === dense) + assert(gridBasedMat.toLocalMatrix() === dense) } } From ba414d2c6a16de987c9aa456dafa193d191014d5 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 13 Jan 2015 21:00:43 -0800 Subject: [PATCH 11/21] [SPARK-3974] fixed frobenius norm --- .../linalg/distributed/BlockMatrix.scala | 30 ++++++++++--------- .../linalg/distributed/BlockMatrixSuite.scala | 14 ++++----- 2 files changed, 23 insertions(+), 21 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index a4b2bf42390f2..19e896ec4158c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -104,7 +104,8 @@ class BlockMatrix( rowPerBlock: Int, colPerBlock: Int) = { this(numRowBlocks, numColBlocks, rdd) - val part = new GridPartitioner(numRowBlocks, numColBlocks, rowPerBlock, colPerBlock, rdd.partitions.length) + val part = new GridPartitioner(numRowBlocks, numColBlocks, rowPerBlock, + colPerBlock, rdd.partitions.length) setPartitioner(part) } @@ -130,7 +131,9 @@ class BlockMatrix( /** Returns the dimensions of the matrix. */ def getDim: (Long, Long) = { // picks the sizes of the matrix with the maximum indices - def pickSizeByGreaterIndex(example: (Int, Int, Int, Int), base: (Int, Int, Int, Int)): (Int, Int, Int, Int) = { + def pickSizeByGreaterIndex( + example: (Int, Int, Int, Int), + base: (Int, Int, Int, Int)): (Int, Int, Int, Int) = { if (example._1 > base._1 && example._2 > base._2) { (example._1, example._2, example._3, example._4) } else if (example._1 > base._1) { @@ -157,11 +160,12 @@ class BlockMatrix( /** Returns the Frobenius Norm of the matrix */ def normFro(): Double = { - math.sqrt(rdd.map { - case sparse: ((Int, Int), SparseMatrix) => - sparse._2.values.map(x => math.pow(x, 2)).sum - case dense: ((Int, Int), DenseMatrix) => - dense._2.values.map(x => math.pow(x, 2)).sum + math.sqrt(rdd.map { mat => mat._2 match { + case sparse: SparseMatrix => + sparse.values.map(x => math.pow(x, 2)).sum + case dense: DenseMatrix => + dense.values.map(x => math.pow(x, 2)).sum + } }.reduce(_ + _)) } @@ -177,21 +181,21 @@ class BlockMatrix( this } - /** Collect the distributed matrix on the driver. */ + /** Collect the distributed matrix on the driver as a local matrix. */ def toLocalMatrix(): Matrix = { val parts = rdd.collect().sortBy(x => (x._1._2, x._1._1)) val nRows = numRows().toInt val nCols = numCols().toInt val values = new Array[Double](nRows * nCols) - var rowStart = 0 - var colStart = 0 + parts.foreach { part => - if (part._1._1 == 0) rowStart = 0 + val rowOffset = part._1._1 * partitioner.rowPerBlock + val colOffset = part._1._2 * partitioner.colPerBlock val block = part._2 var j = 0 while (j < block.numCols) { var i = 0 - val indStart = (j + colStart) * nRows + rowStart + val indStart = (j + colOffset) * nRows + rowOffset val indEnd = block.numRows val matStart = j * block.numRows val mat = block.toArray @@ -201,8 +205,6 @@ class BlockMatrix( } j += 1 } - rowStart += block.numRows - if (part._1._1 == numRowBlocks - 1) colStart += block.numCols } new DenseMatrix(nRows, nCols, values) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index 45152f473c16d..f299021268ddb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -41,26 +41,26 @@ class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { val entries: Seq[SubMatrix] = Seq( new SubMatrix((0, 0), new DenseMatrix(2, 2, Array(1.0, 0.0, 0.0, 2.0))), new SubMatrix((0, 1), new DenseMatrix(2, 2, Array(0.0, 1.0, 0.0, 0.0))), - new SubMatrix((1, 0), new DenseMatrix(2, 2, Array(3.0, 0.0, 1.5, 0.0))), - new SubMatrix((1, 1), new DenseMatrix(2, 2, Array(1.0, 4.0, 0.0, 1.0))), - new SubMatrix((2, 0), new DenseMatrix(1, 2, Array(1.0, 0.0))), + new SubMatrix((1, 0), new DenseMatrix(2, 2, Array(3.0, 0.0, 1.0, 1.0))), + new SubMatrix((1, 1), new DenseMatrix(2, 2, Array(1.0, 2.0, 0.0, 1.0))), new SubMatrix((2, 1), new DenseMatrix(1, 2, Array(1.0, 5.0)))) gridBasedMat = new BlockMatrix(numRowBlocks, numColBlocks, sc.parallelize(entries, 2)) } - test("size") { + test("size and frobenius norm") { assert(gridBasedMat.numRows() === m) assert(gridBasedMat.numCols() === n) + assert(gridBasedMat.normFro() === 7.0) } test("toBreeze and toLocalMatrix") { val expected = BDM( (1.0, 0.0, 0.0, 0.0), (0.0, 2.0, 1.0, 0.0), - (3.0, 1.5, 1.0, 0.0), - (0.0, 0.0, 4.0, 1.0), - (1.0, 0.0, 1.0, 5.0)) + (3.0, 1.0, 1.0, 0.0), + (0.0, 1.0, 2.0, 1.0), + (0.0, 0.0, 1.0, 5.0)) val dense = Matrices.fromBreeze(expected).asInstanceOf[DenseMatrix] assert(gridBasedMat.toBreeze() === expected) From 239ab4b47d910428460160311165f620afb605fa Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 19 Jan 2015 10:39:56 -0800 Subject: [PATCH 12/21] [SPARK-3974] Addressed @jkbradley's comments --- .../linalg/distributed/BlockMatrix.scala | 94 ++++++++++--------- .../linalg/distributed/BlockMatrixSuite.scala | 16 +++- 2 files changed, 63 insertions(+), 47 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 19e896ec4158c..0b78630f85c15 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -31,14 +31,14 @@ import org.apache.spark.util.Utils * * @param numRowBlocks Number of blocks that form the rows of the matrix. * @param numColBlocks Number of blocks that form the columns of the matrix. - * @param rowPerBlock Number of rows that make up each block. - * @param colPerBlock Number of columns that make up each block. + * @param rowsPerBlock Number of rows that make up each block. + * @param colsPerBlock Number of columns that make up each block. */ private[mllib] class GridPartitioner( val numRowBlocks: Int, val numColBlocks: Int, - val rowPerBlock: Int, - val colPerBlock: Int, + val rowsPerBlock: Int, + val colsPerBlock: Int, override val numPartitions: Int) extends Partitioner { /** @@ -52,10 +52,10 @@ private[mllib] class GridPartitioner( */ override def getPartition(key: Any): Int = { key match { - case ind: (Int, Int) => - Utils.nonNegativeMod(ind._1 + ind._2 * numRowBlocks, numPartitions) - case indices: (Int, Int, Int) => - Utils.nonNegativeMod(indices._1 + indices._3 * numRowBlocks, numPartitions) + case (rowIndex: Int, colIndex: Int) => + Utils.nonNegativeMod(rowIndex + colIndex * numRowBlocks, numPartitions) + case (rowIndex: Int, innerIndex: Int, colIndex: Int) => + Utils.nonNegativeMod(rowIndex + colIndex * numRowBlocks, numPartitions) case _ => throw new IllegalArgumentException("Unrecognized key") } @@ -65,8 +65,8 @@ private[mllib] class GridPartitioner( override def equals(obj: Any): Boolean = { obj match { case r: GridPartitioner => - (this.numPartitions == r.numPartitions) && (this.rowPerBlock == r.rowPerBlock) && - (this.colPerBlock == r.colPerBlock) + (this.numPartitions == r.numPartitions) && (this.rowsPerBlock == r.rowsPerBlock) && + (this.colsPerBlock == r.colsPerBlock) case _ => false } @@ -94,18 +94,18 @@ class BlockMatrix( * @param numRowBlocks Number of blocks that form the rows of this matrix * @param numColBlocks Number of blocks that form the columns of this matrix * @param rdd The RDD of SubMatrices (local matrices) that form this matrix - * @param rowPerBlock Number of rows that make up each block. - * @param colPerBlock Number of columns that make up each block. + * @param rowsPerBlock Number of rows that make up each block. + * @param colsPerBlock Number of columns that make up each block. */ def this( numRowBlocks: Int, numColBlocks: Int, rdd: RDD[((Int, Int), Matrix)], - rowPerBlock: Int, - colPerBlock: Int) = { + rowsPerBlock: Int, + colsPerBlock: Int) = { this(numRowBlocks, numColBlocks, rdd) - val part = new GridPartitioner(numRowBlocks, numColBlocks, rowPerBlock, - colPerBlock, rdd.partitions.length) + val part = new GridPartitioner(numRowBlocks, numColBlocks, rowsPerBlock, + colsPerBlock, rdd.partitions.length) setPartitioner(part) } @@ -129,33 +129,35 @@ class BlockMatrix( override def numCols(): Long = dims._2 /** Returns the dimensions of the matrix. */ - def getDim: (Long, Long) = { + private def getDim: (Long, Long) = { + case class MatrixMetaData(var rowIndex: Int, var colIndex: Int, + var numRows: Int, var numCols: Int) // picks the sizes of the matrix with the maximum indices - def pickSizeByGreaterIndex( - example: (Int, Int, Int, Int), - base: (Int, Int, Int, Int)): (Int, Int, Int, Int) = { - if (example._1 > base._1 && example._2 > base._2) { - (example._1, example._2, example._3, example._4) - } else if (example._1 > base._1) { - (example._1, base._2, example._3, base._4) - } else if (example._2 > base._2) { - (base._1, example._2, base._3, example._4) - } else { - (base._1, base._2, base._3, base._4) + def pickSizeByGreaterIndex(example: MatrixMetaData, base: MatrixMetaData): MatrixMetaData = { + if (example.rowIndex > base.rowIndex) { + base.rowIndex = example.rowIndex + base.numRows = example.numRows } + if (example.colIndex > base.colIndex) { + base.colIndex = example.colIndex + base.numCols = example.numCols + } + base } - val lastRowCol = rdd.treeAggregate((0, 0, 0, 0))( + val lastRowCol = rdd.treeAggregate(new MatrixMetaData(0, 0, 0, 0))( seqOp = (c, v) => (c, v) match { case (base, ((blockXInd, blockYInd), mat)) => - pickSizeByGreaterIndex((blockXInd, blockYInd, mat.numRows, mat.numCols), base) + pickSizeByGreaterIndex( + new MatrixMetaData(blockXInd, blockYInd, mat.numRows, mat.numCols), base) }, combOp = (c1, c2) => (c1, c2) match { case (res1, res2) => pickSizeByGreaterIndex(res1, res2) }) - - (lastRowCol._1.toLong * partitioner.rowPerBlock + lastRowCol._3, - lastRowCol._2.toLong * partitioner.colPerBlock + lastRowCol._4) + // We add the size of the edge matrices, because they can be less than the specified + // rowsPerBlock or colsPerBlock. + (lastRowCol.rowIndex.toLong * partitioner.rowsPerBlock + lastRowCol.numRows, + lastRowCol.colIndex.toLong * partitioner.colsPerBlock + lastRowCol.numCols) } /** Returns the Frobenius Norm of the matrix */ @@ -165,7 +167,7 @@ class BlockMatrix( sparse.values.map(x => math.pow(x, 2)).sum case dense: DenseMatrix => dense.values.map(x => math.pow(x, 2)).sum - } + } }.reduce(_ + _)) } @@ -181,25 +183,29 @@ class BlockMatrix( this } - /** Collect the distributed matrix on the driver as a local matrix. */ + /** Collect the distributed matrix on the driver as a `DenseMatrix`. */ def toLocalMatrix(): Matrix = { - val parts = rdd.collect().sortBy(x => (x._1._2, x._1._1)) + require(numRows() < Int.MaxValue, "The number of rows of this matrix should be less than " + + s"Int.MaxValue. Currently numRows: ${numRows()}") + require(numCols() < Int.MaxValue, "The number of columns of this matrix should be less than " + + s"Int.MaxValue. Currently numCols: ${numCols()}") val nRows = numRows().toInt val nCols = numCols().toInt - val values = new Array[Double](nRows * nCols) + val mem = nRows * nCols * 8 / 1000000 + if (mem > 500) logWarning(s"Storing this matrix will require $mem MB of memory!") - parts.foreach { part => - val rowOffset = part._1._1 * partitioner.rowPerBlock - val colOffset = part._1._2 * partitioner.colPerBlock - val block = part._2 + val parts = rdd.collect().sortBy(x => (x._1._2, x._1._1)) + val values = new Array[Double](nRows * nCols) + parts.foreach { case ((rowIndex, colIndex), block) => + val rowOffset = rowIndex * partitioner.rowsPerBlock + val colOffset = colIndex * partitioner.colsPerBlock var j = 0 + val mat = block.toArray while (j < block.numCols) { var i = 0 val indStart = (j + colOffset) * nRows + rowOffset - val indEnd = block.numRows val matStart = j * block.numRows - val mat = block.toArray - while (i < indEnd) { + while (i < block.numRows) { values(indStart + i) = mat(matStart + i) i += 1 } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index f299021268ddb..918a48578b959 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -24,14 +24,24 @@ import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Matrix} import org.apache.spark.mllib.util.MLlibTestSparkContext -class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { - +// Input values for the tests +private object BlockMatrixSuite { val m = 5 val n = 4 val rowPerPart = 2 val colPerPart = 2 val numRowBlocks = 3 val numColBlocks = 2 +} + +class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { + + val m = BlockMatrixSuite.m + val n = BlockMatrixSuite.n + val rowPerPart = BlockMatrixSuite.rowPerPart + val colPerPart = BlockMatrixSuite.colPerPart + val numRowBlocks = BlockMatrixSuite.numRowBlocks + val numColBlocks = BlockMatrixSuite.numColBlocks var gridBasedMat: BlockMatrix = _ type SubMatrix = ((Int, Int), Matrix) @@ -63,7 +73,7 @@ class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { (0.0, 0.0, 1.0, 5.0)) val dense = Matrices.fromBreeze(expected).asInstanceOf[DenseMatrix] - assert(gridBasedMat.toBreeze() === expected) assert(gridBasedMat.toLocalMatrix() === dense) + assert(gridBasedMat.toBreeze() === expected) } } From 1e8bb2a592de611f3796c28798356878c6c89541 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 20 Jan 2015 11:02:45 -0800 Subject: [PATCH 13/21] [SPARK-3974] Change return type of cache and persist --- .../apache/spark/mllib/linalg/distributed/BlockMatrix.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 0b78630f85c15..e8f270754f8b4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -172,13 +172,13 @@ class BlockMatrix( } /** Cache the underlying RDD. */ - def cache(): DistributedMatrix = { + def cache(): BlockMatrix = { rdd.cache() this } /** Set the storage level for the underlying RDD. */ - def persist(storageLevel: StorageLevel): DistributedMatrix = { + def persist(storageLevel: StorageLevel): BlockMatrix = { rdd.persist(storageLevel) this } From 1a63b204a59601734de986964d09515a0b5ab082 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 20 Jan 2015 12:45:17 -0800 Subject: [PATCH 14/21] [SPARK-3974] Remove setPartition method. Isn't required --- .../spark/mllib/linalg/distributed/BlockMatrix.scala | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index e8f270754f8b4..4323cc936bf95 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -106,7 +106,7 @@ class BlockMatrix( this(numRowBlocks, numColBlocks, rdd) val part = new GridPartitioner(numRowBlocks, numColBlocks, rowsPerBlock, colsPerBlock, rdd.partitions.length) - setPartitioner(part) + partitioner = part } private[mllib] var partitioner: GridPartitioner = { @@ -115,14 +115,6 @@ class BlockMatrix( firstSubMatrix.numRows, firstSubMatrix.numCols, rdd.partitions.length) } - /** - * Set the partitioner for the matrix. For internal use only. Users should use `repartition`. - * @param part A partitioner that specifies how SubMatrices are stored in the cluster - */ - private def setPartitioner(part: GridPartitioner): Unit = { - partitioner = part - } - private lazy val dims: (Long, Long) = getDim override def numRows(): Long = dims._1 From eebbdf742e5ef94e9c5f278a07fd94b625117716 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 20 Jan 2015 17:43:56 -0800 Subject: [PATCH 15/21] preliminary changes addressing code review --- .../linalg/distributed/BlockMatrix.scala | 22 +++++++++++-------- .../linalg/distributed/BlockMatrixSuite.scala | 1 - 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 4323cc936bf95..3b9e6290e99af 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -42,7 +42,8 @@ private[mllib] class GridPartitioner( override val numPartitions: Int) extends Partitioner { /** - * Returns the index of the partition the SubMatrix belongs to. + * Returns the index of the partition the SubMatrix belongs to. Tries to achieve block wise + * partitioning. * * @param key The key for the SubMatrix. Can be its position in the grid (its column major index) * or a tuple of three integers that are the final row index after the multiplication, @@ -51,13 +52,16 @@ private[mllib] class GridPartitioner( * @return The index of the partition, which the SubMatrix belongs to. */ override def getPartition(key: Any): Int = { + val sqrtPartition = math.round(math.sqrt(numPartitions)).toInt + // numPartitions may not be the square of a number, it can even be a prime number + key match { - case (rowIndex: Int, colIndex: Int) => - Utils.nonNegativeMod(rowIndex + colIndex * numRowBlocks, numPartitions) - case (rowIndex: Int, innerIndex: Int, colIndex: Int) => - Utils.nonNegativeMod(rowIndex + colIndex * numRowBlocks, numPartitions) + case (blockRowIndex: Int, blockColIndex: Int) => + Utils.nonNegativeMod(blockRowIndex + blockColIndex * numRowBlocks, numPartitions) + case (blockRowIndex: Int, innerIndex: Int, blockColIndex: Int) => + Utils.nonNegativeMod(blockRowIndex + blockColIndex * numRowBlocks, numPartitions) case _ => - throw new IllegalArgumentException("Unrecognized key") + throw new IllegalArgumentException(s"Unrecognized key. key: $key") } } @@ -65,8 +69,8 @@ private[mllib] class GridPartitioner( override def equals(obj: Any): Boolean = { obj match { case r: GridPartitioner => - (this.numPartitions == r.numPartitions) && (this.rowsPerBlock == r.rowsPerBlock) && - (this.colsPerBlock == r.colsPerBlock) + (this.numRowBlocks == r.numRowBlocks) && (this.numColBlocks == r.numColBlocks) + (this.rowsPerBlock == r.rowsPerBlock) && (this.colsPerBlock == r.colsPerBlock) case _ => false } @@ -85,7 +89,7 @@ class BlockMatrix( val numColBlocks: Int, val rdd: RDD[((Int, Int), Matrix)]) extends DistributedMatrix with Logging { - type SubMatrix = ((Int, Int), Matrix) // ((blockRowIndex, blockColIndex), matrix) + private type SubMatrix = ((Int, Int), Matrix) // ((blockRowIndex, blockColIndex), matrix) /** * Alternate constructor for BlockMatrix without the input of a partitioner. Will use a Grid diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index 918a48578b959..d61867d3f201a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.mllib.linalg.distributed import org.scalatest.FunSuite - import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Matrix} From f9d664b7ebae288d778c7694dcc380f52e6cde30 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 21 Jan 2015 14:33:17 -0800 Subject: [PATCH 16/21] updated API and modified partitioning scheme --- .../linalg/distributed/BlockMatrix.scala | 97 ++++++++++++------- .../linalg/distributed/BlockMatrixSuite.scala | 3 +- 2 files changed, 62 insertions(+), 38 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 3b9e6290e99af..ef9dccc3111e9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -19,27 +19,24 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} -import org.apache.spark._ +import org.apache.spark.{Logging, Partitioner} import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.Utils /** * A grid partitioner, which stores every block in a separate partition. * * @param numRowBlocks Number of blocks that form the rows of the matrix. * @param numColBlocks Number of blocks that form the columns of the matrix. - * @param rowsPerBlock Number of rows that make up each block. - * @param colsPerBlock Number of columns that make up each block. */ private[mllib] class GridPartitioner( val numRowBlocks: Int, val numColBlocks: Int, - val rowsPerBlock: Int, - val colsPerBlock: Int, - override val numPartitions: Int) extends Partitioner { + val numParts: Int) extends Partitioner { + // Having the number of partitions greater than the number of sub matrices does not help + override val numPartitions = math.min(numParts, numRowBlocks * numColBlocks) /** * Returns the index of the partition the SubMatrix belongs to. Tries to achieve block wise @@ -52,25 +49,38 @@ private[mllib] class GridPartitioner( * @return The index of the partition, which the SubMatrix belongs to. */ override def getPartition(key: Any): Int = { - val sqrtPartition = math.round(math.sqrt(numPartitions)).toInt - // numPartitions may not be the square of a number, it can even be a prime number - key match { case (blockRowIndex: Int, blockColIndex: Int) => - Utils.nonNegativeMod(blockRowIndex + blockColIndex * numRowBlocks, numPartitions) + getBlockId(blockRowIndex, blockColIndex) case (blockRowIndex: Int, innerIndex: Int, blockColIndex: Int) => - Utils.nonNegativeMod(blockRowIndex + blockColIndex * numRowBlocks, numPartitions) + getBlockId(blockRowIndex, blockColIndex) case _ => throw new IllegalArgumentException(s"Unrecognized key. key: $key") } } + /** Partitions sub-matrices as blocks with neighboring sub-matrices. */ + private def getBlockId(blockRowIndex: Int, blockColIndex: Int): Int = { + val totalBlocks = numRowBlocks * numColBlocks + // Gives the number of blocks that need to be in each partition + val partitionRatio = math.ceil(totalBlocks * 1.0 / numPartitions).toInt + // Number of neighboring blocks to take in each row + val subBlocksPerRow = math.ceil(numRowBlocks * 1.0 / partitionRatio).toInt + // Number of neighboring blocks to take in each column + val subBlocksPerCol = math.ceil(numColBlocks * 1.0 / partitionRatio).toInt + // Coordinates of the block + val i = blockRowIndex / subBlocksPerRow + val j = blockColIndex / subBlocksPerCol + val blocksPerRow = math.ceil(numRowBlocks * 1.0 / subBlocksPerRow).toInt + j * blocksPerRow + i + } + /** Checks whether the partitioners have the same characteristics */ override def equals(obj: Any): Boolean = { obj match { case r: GridPartitioner => - (this.numRowBlocks == r.numRowBlocks) && (this.numColBlocks == r.numColBlocks) - (this.rowsPerBlock == r.rowsPerBlock) && (this.colsPerBlock == r.colsPerBlock) + (this.numRowBlocks == r.numRowBlocks) && (this.numColBlocks == r.numColBlocks) && + (this.numPartitions == r.numPartitions) case _ => false } @@ -80,49 +90,61 @@ private[mllib] class GridPartitioner( /** * Represents a distributed matrix in blocks of local matrices. * + * @param rdd The RDD of SubMatrices (local matrices) that form this matrix + * @param nRows Number of rows of this matrix + * @param nCols Number of columns of this matrix * @param numRowBlocks Number of blocks that form the rows of this matrix * @param numColBlocks Number of blocks that form the columns of this matrix - * @param rdd The RDD of SubMatrices (local matrices) that form this matrix + * @param rowsPerBlock Number of rows that make up each block. The blocks forming the final + * rows are not required to have the given number of rows + * @param colsPerBlock Number of columns that make up each block. The blocks forming the final + * columns are not required to have the given number of columns */ class BlockMatrix( + val rdd: RDD[((Int, Int), Matrix)], + private var nRows: Long, + private var nCols: Long, val numRowBlocks: Int, val numColBlocks: Int, - val rdd: RDD[((Int, Int), Matrix)]) extends DistributedMatrix with Logging { + val rowsPerBlock: Int, + val colsPerBlock: Int) extends DistributedMatrix with Logging { private type SubMatrix = ((Int, Int), Matrix) // ((blockRowIndex, blockColIndex), matrix) /** - * Alternate constructor for BlockMatrix without the input of a partitioner. Will use a Grid - * Partitioner by default. + * Alternate constructor for BlockMatrix without the input of the number of rows and columns. * + * @param rdd The RDD of SubMatrices (local matrices) that form this matrix * @param numRowBlocks Number of blocks that form the rows of this matrix * @param numColBlocks Number of blocks that form the columns of this matrix - * @param rdd The RDD of SubMatrices (local matrices) that form this matrix - * @param rowsPerBlock Number of rows that make up each block. - * @param colsPerBlock Number of columns that make up each block. + * @param rowsPerBlock Number of rows that make up each block. The blocks forming the final + * rows are not required to have the given number of rows + * @param colsPerBlock Number of columns that make up each block. The blocks forming the final + * columns are not required to have the given number of columns */ def this( + rdd: RDD[((Int, Int), Matrix)], numRowBlocks: Int, numColBlocks: Int, - rdd: RDD[((Int, Int), Matrix)], rowsPerBlock: Int, colsPerBlock: Int) = { - this(numRowBlocks, numColBlocks, rdd) - val part = new GridPartitioner(numRowBlocks, numColBlocks, rowsPerBlock, - colsPerBlock, rdd.partitions.length) - partitioner = part + this(rdd, 0L, 0L, numRowBlocks, numColBlocks, rowsPerBlock, colsPerBlock) } - private[mllib] var partitioner: GridPartitioner = { - val firstSubMatrix = rdd.first()._2 - new GridPartitioner(numRowBlocks, numColBlocks, - firstSubMatrix.numRows, firstSubMatrix.numCols, rdd.partitions.length) - } + private[mllib] var partitioner: GridPartitioner = + new GridPartitioner(numRowBlocks, numColBlocks, rdd.partitions.length) private lazy val dims: (Long, Long) = getDim - override def numRows(): Long = dims._1 - override def numCols(): Long = dims._2 + override def numRows(): Long = { + if (nRows <= 0L) nRows = dims._1 + nRows + } + + override def numCols(): Long = { + if (nCols <= 0L) nCols = dims._2 + nCols + } /** Returns the dimensions of the matrix. */ private def getDim: (Long, Long) = { @@ -141,6 +163,7 @@ class BlockMatrix( base } + // Aggregate will return an error if the rdd is empty val lastRowCol = rdd.treeAggregate(new MatrixMetaData(0, 0, 0, 0))( seqOp = (c, v) => (c, v) match { case (base, ((blockXInd, blockYInd), mat)) => pickSizeByGreaterIndex( @@ -152,8 +175,8 @@ class BlockMatrix( }) // We add the size of the edge matrices, because they can be less than the specified // rowsPerBlock or colsPerBlock. - (lastRowCol.rowIndex.toLong * partitioner.rowsPerBlock + lastRowCol.numRows, - lastRowCol.colIndex.toLong * partitioner.colsPerBlock + lastRowCol.numCols) + (lastRowCol.rowIndex.toLong * rowsPerBlock + lastRowCol.numRows, + lastRowCol.colIndex.toLong * colsPerBlock + lastRowCol.numCols) } /** Returns the Frobenius Norm of the matrix */ @@ -193,8 +216,8 @@ class BlockMatrix( val parts = rdd.collect().sortBy(x => (x._1._2, x._1._1)) val values = new Array[Double](nRows * nCols) parts.foreach { case ((rowIndex, colIndex), block) => - val rowOffset = rowIndex * partitioner.rowsPerBlock - val colOffset = colIndex * partitioner.colsPerBlock + val rowOffset = rowIndex * rowsPerBlock + val colOffset = colIndex * colsPerBlock var j = 0 val mat = block.toArray while (j < block.numCols) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index d61867d3f201a..678f24bdbee45 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -54,7 +54,8 @@ class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { new SubMatrix((1, 1), new DenseMatrix(2, 2, Array(1.0, 2.0, 0.0, 1.0))), new SubMatrix((2, 1), new DenseMatrix(1, 2, Array(1.0, 5.0)))) - gridBasedMat = new BlockMatrix(numRowBlocks, numColBlocks, sc.parallelize(entries, 2)) + gridBasedMat = new BlockMatrix(sc.parallelize(entries, 2), numRowBlocks, numColBlocks, + rowPerPart, colPerPart) } test("size and frobenius norm") { From 1694c9e3be57b83e6342a75a454614a404a97348 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 26 Jan 2015 22:48:28 -0800 Subject: [PATCH 17/21] almost finished addressing comments --- .../linalg/distributed/BlockMatrix.scala | 115 +++++++----------- 1 file changed, 42 insertions(+), 73 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index ef9dccc3111e9..05f588f1ba3e9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -30,13 +30,25 @@ import org.apache.spark.storage.StorageLevel * * @param numRowBlocks Number of blocks that form the rows of the matrix. * @param numColBlocks Number of blocks that form the columns of the matrix. + * @param suggestedNumPartitions Number of partitions to partition the rdd into. The final number + * of partitions will be set to `min(suggestedNumPartitions, + * numRowBlocks * numColBlocks)`, because setting the number of + * partitions greater than the number of sub matrices is not useful. */ private[mllib] class GridPartitioner( val numRowBlocks: Int, val numColBlocks: Int, - val numParts: Int) extends Partitioner { + suggestedNumPartitions: Int) extends Partitioner { // Having the number of partitions greater than the number of sub matrices does not help - override val numPartitions = math.min(numParts, numRowBlocks * numColBlocks) + override val numPartitions = math.min(suggestedNumPartitions, numRowBlocks * numColBlocks) + + val totalBlocks = numRowBlocks.toLong * numColBlocks + // Gives the number of blocks that need to be in each partition + val targetNumBlocksPerPartition = math.ceil(totalBlocks * 1.0 / numPartitions).toInt + // Number of neighboring blocks to take in each row + val numRowBlocksPerPartition = math.ceil(numRowBlocks * 1.0 / targetNumBlocksPerPartition).toInt + // Number of neighboring blocks to take in each column + val numColBlocksPerPartition = math.ceil(numColBlocks * 1.0 / targetNumBlocksPerPartition).toInt /** * Returns the index of the partition the SubMatrix belongs to. Tries to achieve block wise @@ -51,27 +63,20 @@ private[mllib] class GridPartitioner( override def getPartition(key: Any): Int = { key match { case (blockRowIndex: Int, blockColIndex: Int) => - getBlockId(blockRowIndex, blockColIndex) + getPartitionId(blockRowIndex, blockColIndex) case (blockRowIndex: Int, innerIndex: Int, blockColIndex: Int) => - getBlockId(blockRowIndex, blockColIndex) + getPartitionId(blockRowIndex, blockColIndex) case _ => throw new IllegalArgumentException(s"Unrecognized key. key: $key") } } /** Partitions sub-matrices as blocks with neighboring sub-matrices. */ - private def getBlockId(blockRowIndex: Int, blockColIndex: Int): Int = { - val totalBlocks = numRowBlocks * numColBlocks - // Gives the number of blocks that need to be in each partition - val partitionRatio = math.ceil(totalBlocks * 1.0 / numPartitions).toInt - // Number of neighboring blocks to take in each row - val subBlocksPerRow = math.ceil(numRowBlocks * 1.0 / partitionRatio).toInt - // Number of neighboring blocks to take in each column - val subBlocksPerCol = math.ceil(numColBlocks * 1.0 / partitionRatio).toInt + private def getPartitionId(blockRowIndex: Int, blockColIndex: Int): Int = { // Coordinates of the block - val i = blockRowIndex / subBlocksPerRow - val j = blockColIndex / subBlocksPerCol - val blocksPerRow = math.ceil(numRowBlocks * 1.0 / subBlocksPerRow).toInt + val i = blockRowIndex / numRowBlocksPerPartition + val j = blockColIndex / numColBlocksPerPartition + val blocksPerRow = math.ceil(numRowBlocks * 1.0 / numRowBlocksPerPartition).toInt j * blocksPerRow + i } @@ -91,10 +96,10 @@ private[mllib] class GridPartitioner( * Represents a distributed matrix in blocks of local matrices. * * @param rdd The RDD of SubMatrices (local matrices) that form this matrix - * @param nRows Number of rows of this matrix - * @param nCols Number of columns of this matrix - * @param numRowBlocks Number of blocks that form the rows of this matrix - * @param numColBlocks Number of blocks that form the columns of this matrix + * @param nRows Number of rows of this matrix. If the supplied value is less than or equal to zero, + * the number of rows will be calculated when `numRows` is invoked. + * @param nCols Number of columns of this matrix. If the supplied value is less than or equal to + * zero, the number of columns will be calculated when `numCols` is invoked. * @param rowsPerBlock Number of rows that make up each block. The blocks forming the final * rows are not required to have the given number of rows * @param colsPerBlock Number of columns that make up each block. The blocks forming the final @@ -104,8 +109,6 @@ class BlockMatrix( val rdd: RDD[((Int, Int), Matrix)], private var nRows: Long, private var nCols: Long, - val numRowBlocks: Int, - val numColBlocks: Int, val rowsPerBlock: Int, val colsPerBlock: Int) extends DistributedMatrix with Logging { @@ -115,8 +118,6 @@ class BlockMatrix( * Alternate constructor for BlockMatrix without the input of the number of rows and columns. * * @param rdd The RDD of SubMatrices (local matrices) that form this matrix - * @param numRowBlocks Number of blocks that form the rows of this matrix - * @param numColBlocks Number of blocks that form the columns of this matrix * @param rowsPerBlock Number of rows that make up each block. The blocks forming the final * rows are not required to have the given number of rows * @param colsPerBlock Number of columns that make up each block. The blocks forming the final @@ -124,16 +125,11 @@ class BlockMatrix( */ def this( rdd: RDD[((Int, Int), Matrix)], - numRowBlocks: Int, - numColBlocks: Int, rowsPerBlock: Int, colsPerBlock: Int) = { - this(rdd, 0L, 0L, numRowBlocks, numColBlocks, rowsPerBlock, colsPerBlock) + this(rdd, 0L, 0L, rowsPerBlock, colsPerBlock) } - private[mllib] var partitioner: GridPartitioner = - new GridPartitioner(numRowBlocks, numColBlocks, rdd.partitions.length) - private lazy val dims: (Long, Long) = getDim override def numRows(): Long = { @@ -146,48 +142,21 @@ class BlockMatrix( nCols } + val numRowBlocks = math.ceil(numRows() * 1.0 / rowsPerBlock).toInt + val numColBlocks = math.ceil(numCols() * 1.0 / colsPerBlock).toInt + + private[mllib] var partitioner: GridPartitioner = + new GridPartitioner(numRowBlocks, numColBlocks, rdd.partitions.length) + + + /** Returns the dimensions of the matrix. */ private def getDim: (Long, Long) = { - case class MatrixMetaData(var rowIndex: Int, var colIndex: Int, - var numRows: Int, var numCols: Int) - // picks the sizes of the matrix with the maximum indices - def pickSizeByGreaterIndex(example: MatrixMetaData, base: MatrixMetaData): MatrixMetaData = { - if (example.rowIndex > base.rowIndex) { - base.rowIndex = example.rowIndex - base.numRows = example.numRows - } - if (example.colIndex > base.colIndex) { - base.colIndex = example.colIndex - base.numCols = example.numCols - } - base - } - - // Aggregate will return an error if the rdd is empty - val lastRowCol = rdd.treeAggregate(new MatrixMetaData(0, 0, 0, 0))( - seqOp = (c, v) => (c, v) match { case (base, ((blockXInd, blockYInd), mat)) => - pickSizeByGreaterIndex( - new MatrixMetaData(blockXInd, blockYInd, mat.numRows, mat.numCols), base) - }, - combOp = (c1, c2) => (c1, c2) match { - case (res1, res2) => - pickSizeByGreaterIndex(res1, res2) - }) - // We add the size of the edge matrices, because they can be less than the specified - // rowsPerBlock or colsPerBlock. - (lastRowCol.rowIndex.toLong * rowsPerBlock + lastRowCol.numRows, - lastRowCol.colIndex.toLong * colsPerBlock + lastRowCol.numCols) - } + val (rows, cols) = rdd.map { case ((blockRowIndex, blockColIndex), mat) => + (blockRowIndex * rowsPerBlock + mat.numRows, blockColIndex * colsPerBlock + mat.numCols) + }.reduce((x0, x1) => (math.max(x0._1, x1._1), math.max(x0._2, x1._2))) - /** Returns the Frobenius Norm of the matrix */ - def normFro(): Double = { - math.sqrt(rdd.map { mat => mat._2 match { - case sparse: SparseMatrix => - sparse.values.map(x => math.pow(x, 2)).sum - case dense: DenseMatrix => - dense.values.map(x => math.pow(x, 2)).sum - } - }.reduce(_ + _)) + (math.max(rows, nRows), math.max(cols, nCols)) } /** Cache the underlying RDD. */ @@ -210,14 +179,14 @@ class BlockMatrix( s"Int.MaxValue. Currently numCols: ${numCols()}") val nRows = numRows().toInt val nCols = numCols().toInt - val mem = nRows * nCols * 8 / 1000000 + val mem = nRows.toLong * nCols / 125000 if (mem > 500) logWarning(s"Storing this matrix will require $mem MB of memory!") - val parts = rdd.collect().sortBy(x => (x._1._2, x._1._1)) + val parts = rdd.collect() val values = new Array[Double](nRows * nCols) - parts.foreach { case ((rowIndex, colIndex), block) => - val rowOffset = rowIndex * rowsPerBlock - val colOffset = colIndex * colsPerBlock + parts.foreach { case ((blockRowIndex, blockColIndex), block) => + val rowOffset = blockRowIndex * rowsPerBlock + val colOffset = blockColIndex * colsPerBlock var j = 0 val mat = block.toArray while (j < block.numCols) { From 5eecd487c54f65a9eacdf8b29eea218e0a27eb20 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 27 Jan 2015 11:37:54 -0800 Subject: [PATCH 18/21] fixed gridPartitioner and added tests --- .../linalg/distributed/BlockMatrix.scala | 81 ++++++++++++++----- .../linalg/distributed/BlockMatrixSuite.scala | 69 +++++++++++----- 2 files changed, 110 insertions(+), 40 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 05f588f1ba3e9..6d7ead9f9b71a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} +import org.apache.spark.util.Utils import org.apache.spark.{Logging, Partitioner} import org.apache.spark.mllib.linalg._ @@ -39,16 +40,17 @@ private[mllib] class GridPartitioner( val numRowBlocks: Int, val numColBlocks: Int, suggestedNumPartitions: Int) extends Partitioner { + private val totalBlocks = numRowBlocks.toLong * numColBlocks // Having the number of partitions greater than the number of sub matrices does not help - override val numPartitions = math.min(suggestedNumPartitions, numRowBlocks * numColBlocks) + override val numPartitions = math.min(suggestedNumPartitions, totalBlocks).toInt - val totalBlocks = numRowBlocks.toLong * numColBlocks - // Gives the number of blocks that need to be in each partition - val targetNumBlocksPerPartition = math.ceil(totalBlocks * 1.0 / numPartitions).toInt + private val blockLengthsPerPartition = findOptimalBlockLengths // Number of neighboring blocks to take in each row - val numRowBlocksPerPartition = math.ceil(numRowBlocks * 1.0 / targetNumBlocksPerPartition).toInt + private val numRowBlocksPerPartition = blockLengthsPerPartition._1 // Number of neighboring blocks to take in each column - val numColBlocksPerPartition = math.ceil(numColBlocks * 1.0 / targetNumBlocksPerPartition).toInt + private val numColBlocksPerPartition = blockLengthsPerPartition._2 + // Number of rows of partitions + private val blocksPerRow = math.ceil(numRowBlocks * 1.0 / numRowBlocksPerPartition).toInt /** * Returns the index of the partition the SubMatrix belongs to. Tries to achieve block wise @@ -73,11 +75,54 @@ private[mllib] class GridPartitioner( /** Partitions sub-matrices as blocks with neighboring sub-matrices. */ private def getPartitionId(blockRowIndex: Int, blockColIndex: Int): Int = { + require(0 <= blockRowIndex && blockRowIndex < numRowBlocks, "The blockRowIndex in the key " + + s"must be in the range 0 <= blockRowIndex < numRowBlocks. blockRowIndex: $blockRowIndex," + + s"numRowBlocks: $numRowBlocks") + require(0 <= blockRowIndex && blockColIndex < numColBlocks, "The blockColIndex in the key " + + s"must be in the range 0 <= blockRowIndex < numColBlocks. blockColIndex: $blockColIndex, " + + s"numColBlocks: $numColBlocks") // Coordinates of the block val i = blockRowIndex / numRowBlocksPerPartition val j = blockColIndex / numColBlocksPerPartition - val blocksPerRow = math.ceil(numRowBlocks * 1.0 / numRowBlocksPerPartition).toInt - j * blocksPerRow + i + // The mod shouldn't be required but is added as a guarantee for possible corner cases + Utils.nonNegativeMod(j * blocksPerRow + i, numPartitions) + } + + /** Tries to calculate the optimal number of blocks that should be in each partition. */ + private def findOptimalBlockLengths: (Int, Int) = { + // Gives the optimal number of blocks that need to be in each partition + val targetNumBlocksPerPartition = math.ceil(totalBlocks * 1.0 / numPartitions).toInt + // Number of neighboring blocks to take in each row + var m = math.ceil(math.sqrt(targetNumBlocksPerPartition)).toInt + // Number of neighboring blocks to take in each column + var n = math.ceil(targetNumBlocksPerPartition * 1.0 / m).toInt + // Try to make m and n close to each other while making sure that we don't exceed the number + // of partitions + var numBlocksForRows = math.ceil(numRowBlocks * 1.0 / m) + var numBlocksForCols = math.ceil(numColBlocks * 1.0 / n) + while ((numBlocksForRows * numBlocksForCols > numPartitions) && (m * n != 0)) { + if (numRowBlocks <= numColBlocks) { + m += 1 + n = math.ceil(targetNumBlocksPerPartition * 1.0 / m).toInt + } else { + n += 1 + m = math.ceil(targetNumBlocksPerPartition * 1.0 / n).toInt + } + numBlocksForRows = math.ceil(numRowBlocks * 1.0 / m) + numBlocksForCols = math.ceil(numColBlocks * 1.0 / n) + } + // If a good partitioning scheme couldn't be found, set the side with the smaller dimension to + // 1 and the other to the number of targetNumBlocksPerPartition + if (m * n == 0) { + if (numRowBlocks <= numColBlocks) { + m = 1 + n = targetNumBlocksPerPartition + } else { + n = 1 + m = targetNumBlocksPerPartition + } + } + (m, n) } /** Checks whether the partitioners have the same characteristics */ @@ -148,8 +193,6 @@ class BlockMatrix( private[mllib] var partitioner: GridPartitioner = new GridPartitioner(numRowBlocks, numColBlocks, rdd.partitions.length) - - /** Returns the dimensions of the matrix. */ private def getDim: (Long, Long) = { val (rows, cols) = rdd.map { case ((blockRowIndex, blockColIndex), mat) => @@ -177,9 +220,11 @@ class BlockMatrix( s"Int.MaxValue. Currently numRows: ${numRows()}") require(numCols() < Int.MaxValue, "The number of columns of this matrix should be less than " + s"Int.MaxValue. Currently numCols: ${numCols()}") + require(numRows() * numCols() < Int.MaxValue, "The length of the values array must be " + + s"less than Int.MaxValue. Currently numRows * numCols: ${numRows() * numCols()}") val nRows = numRows().toInt val nCols = numCols().toInt - val mem = nRows.toLong * nCols / 125000 + val mem = nRows * nCols / 125000 if (mem > 500) logWarning(s"Storing this matrix will require $mem MB of memory!") val parts = rdd.collect() @@ -187,17 +232,9 @@ class BlockMatrix( parts.foreach { case ((blockRowIndex, blockColIndex), block) => val rowOffset = blockRowIndex * rowsPerBlock val colOffset = blockColIndex * colsPerBlock - var j = 0 - val mat = block.toArray - while (j < block.numCols) { - var i = 0 - val indStart = (j + colOffset) * nRows + rowOffset - val matStart = j * block.numRows - while (i < block.numRows) { - values(indStart + i) = mat(matStart + i) - i += 1 - } - j += 1 + block.foreachActive { (i, j, v) => + val indexOffset = (j + colOffset) * nRows + rowOffset + i + values(indexOffset) = v } } new DenseMatrix(nRows, nCols, values) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index 678f24bdbee45..a5016731d1f20 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -23,24 +23,13 @@ import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Matrix} import org.apache.spark.mllib.util.MLlibTestSparkContext -// Input values for the tests -private object BlockMatrixSuite { +class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { + val m = 5 val n = 4 val rowPerPart = 2 val colPerPart = 2 - val numRowBlocks = 3 - val numColBlocks = 2 -} - -class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { - - val m = BlockMatrixSuite.m - val n = BlockMatrixSuite.n - val rowPerPart = BlockMatrixSuite.rowPerPart - val colPerPart = BlockMatrixSuite.colPerPart - val numRowBlocks = BlockMatrixSuite.numRowBlocks - val numColBlocks = BlockMatrixSuite.numColBlocks + val numPartitions = 3 var gridBasedMat: BlockMatrix = _ type SubMatrix = ((Int, Int), Matrix) @@ -54,14 +43,58 @@ class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { new SubMatrix((1, 1), new DenseMatrix(2, 2, Array(1.0, 2.0, 0.0, 1.0))), new SubMatrix((2, 1), new DenseMatrix(1, 2, Array(1.0, 5.0)))) - gridBasedMat = new BlockMatrix(sc.parallelize(entries, 2), numRowBlocks, numColBlocks, - rowPerPart, colPerPart) + gridBasedMat = new BlockMatrix(sc.parallelize(entries, numPartitions), rowPerPart, colPerPart) } - test("size and frobenius norm") { + test("size") { assert(gridBasedMat.numRows() === m) assert(gridBasedMat.numCols() === n) - assert(gridBasedMat.normFro() === 7.0) + } + + test("grid partitioner partitioning") { + val partitioner = gridBasedMat.partitioner + assert(partitioner.getPartition((0, 0)) === 0) + assert(partitioner.getPartition((0, 1)) === 0) + assert(partitioner.getPartition((1, 0)) === 1) + assert(partitioner.getPartition((1, 1)) === 1) + assert(partitioner.getPartition((2, 0)) === 2) + assert(partitioner.getPartition((2, 1)) === 2) + assert(partitioner.getPartition((1, 0, 1)) === 1) + assert(partitioner.getPartition((2, 0, 0)) === 2) + + val part2 = new GridPartitioner(10, 20, 10) + assert(part2.getPartition((0, 0)) === 0) + assert(part2.getPartition((0, 1)) === 0) + assert(part2.getPartition((0, 6)) === 2) + assert(part2.getPartition((3, 7)) === 2) + assert(part2.getPartition((3, 8)) === 4) + assert(part2.getPartition((3, 13)) === 6) + assert(part2.getPartition((9, 14)) === 7) + assert(part2.getPartition((9, 15)) === 7) + assert(part2.getPartition((9, 19)) === 9) + + intercept[IllegalArgumentException] { + part2.getPartition((-1, 0)) + } + + intercept[IllegalArgumentException] { + part2.getPartition((10, 0)) + } + + intercept[IllegalArgumentException] { + part2.getPartition((9, 20)) + } + + val part3 = new GridPartitioner(20, 10, 10) + assert(part3.getPartition((0, 0)) === 0) + assert(part3.getPartition((1, 0)) === 0) + assert(part3.getPartition((6, 0)) === 1) + assert(part3.getPartition((7, 3)) === 1) + assert(part3.getPartition((8, 3)) === 2) + assert(part3.getPartition((13, 3)) === 3) + assert(part3.getPartition((14, 9)) === 8) + assert(part3.getPartition((15, 9)) === 8) + assert(part3.getPartition((19, 9)) === 9) } test("toBreeze and toLocalMatrix") { From 24ec7b8981de973a3eaf7c1b570315d4e02aaf57 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 23:15:53 -0800 Subject: [PATCH 19/21] update grid partitioner --- .../linalg/distributed/BlockMatrix.scala | 137 +++++++----------- .../linalg/distributed/BlockMatrixSuite.scala | 94 +++++++----- 2 files changed, 109 insertions(+), 122 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 6d7ead9f9b71a..de80367ba9211 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -18,125 +18,88 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} -import org.apache.spark.util.Utils import org.apache.spark.{Logging, Partitioner} -import org.apache.spark.mllib.linalg._ -import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.mllib.linalg.{DenseMatrix, Matrix} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel /** - * A grid partitioner, which stores every block in a separate partition. + * A grid partitioner, which uses a regular grid to partition coordinates. * - * @param numRowBlocks Number of blocks that form the rows of the matrix. - * @param numColBlocks Number of blocks that form the columns of the matrix. - * @param suggestedNumPartitions Number of partitions to partition the rdd into. The final number - * of partitions will be set to `min(suggestedNumPartitions, - * numRowBlocks * numColBlocks)`, because setting the number of - * partitions greater than the number of sub matrices is not useful. + * @param rows Number of rows. + * @param cols Number of columns. + * @param rowsPerPart Number of rows per partition, which may be less at the bottom edge. + * @param colsPerPart Number of columns per partition, which may be less at the right edge. */ private[mllib] class GridPartitioner( - val numRowBlocks: Int, - val numColBlocks: Int, - suggestedNumPartitions: Int) extends Partitioner { - private val totalBlocks = numRowBlocks.toLong * numColBlocks - // Having the number of partitions greater than the number of sub matrices does not help - override val numPartitions = math.min(suggestedNumPartitions, totalBlocks).toInt - - private val blockLengthsPerPartition = findOptimalBlockLengths - // Number of neighboring blocks to take in each row - private val numRowBlocksPerPartition = blockLengthsPerPartition._1 - // Number of neighboring blocks to take in each column - private val numColBlocksPerPartition = blockLengthsPerPartition._2 - // Number of rows of partitions - private val blocksPerRow = math.ceil(numRowBlocks * 1.0 / numRowBlocksPerPartition).toInt + val rows: Int, + val cols: Int, + val rowsPerPart: Int, + val colsPerPart: Int) extends Partitioner { + + require(rows > 0) + require(cols > 0) + require(rowsPerPart > 0) + require(colsPerPart > 0) + + private val rowPartitions = math.ceil(rows / rowsPerPart).toInt + private val colPartitions = math.ceil(cols / colsPerPart).toInt + + override val numPartitions = rowPartitions * colPartitions /** - * Returns the index of the partition the SubMatrix belongs to. Tries to achieve block wise - * partitioning. + * Returns the index of the partition the input coordinate belongs to. * - * @param key The key for the SubMatrix. Can be its position in the grid (its column major index) - * or a tuple of three integers that are the final row index after the multiplication, - * the index of the block to multiply with, and the final column index after the + * @param key The coordinate (i, j) or a tuple (i, j, k), where k is the inner index used in * multiplication. - * @return The index of the partition, which the SubMatrix belongs to. + * @return The index of the partition, which the coordinate belongs to. */ override def getPartition(key: Any): Int = { key match { - case (blockRowIndex: Int, blockColIndex: Int) => - getPartitionId(blockRowIndex, blockColIndex) - case (blockRowIndex: Int, innerIndex: Int, blockColIndex: Int) => - getPartitionId(blockRowIndex, blockColIndex) + case (i: Int, j: Int) => + getPartitionId(i, j) + case (i: Int, j: Int, _) => + getPartitionId(i, j) case _ => - throw new IllegalArgumentException(s"Unrecognized key. key: $key") + throw new IllegalArgumentException(s"Unrecognized key: $key") } } /** Partitions sub-matrices as blocks with neighboring sub-matrices. */ - private def getPartitionId(blockRowIndex: Int, blockColIndex: Int): Int = { - require(0 <= blockRowIndex && blockRowIndex < numRowBlocks, "The blockRowIndex in the key " + - s"must be in the range 0 <= blockRowIndex < numRowBlocks. blockRowIndex: $blockRowIndex," + - s"numRowBlocks: $numRowBlocks") - require(0 <= blockRowIndex && blockColIndex < numColBlocks, "The blockColIndex in the key " + - s"must be in the range 0 <= blockRowIndex < numColBlocks. blockColIndex: $blockColIndex, " + - s"numColBlocks: $numColBlocks") - // Coordinates of the block - val i = blockRowIndex / numRowBlocksPerPartition - val j = blockColIndex / numColBlocksPerPartition - // The mod shouldn't be required but is added as a guarantee for possible corner cases - Utils.nonNegativeMod(j * blocksPerRow + i, numPartitions) - } - - /** Tries to calculate the optimal number of blocks that should be in each partition. */ - private def findOptimalBlockLengths: (Int, Int) = { - // Gives the optimal number of blocks that need to be in each partition - val targetNumBlocksPerPartition = math.ceil(totalBlocks * 1.0 / numPartitions).toInt - // Number of neighboring blocks to take in each row - var m = math.ceil(math.sqrt(targetNumBlocksPerPartition)).toInt - // Number of neighboring blocks to take in each column - var n = math.ceil(targetNumBlocksPerPartition * 1.0 / m).toInt - // Try to make m and n close to each other while making sure that we don't exceed the number - // of partitions - var numBlocksForRows = math.ceil(numRowBlocks * 1.0 / m) - var numBlocksForCols = math.ceil(numColBlocks * 1.0 / n) - while ((numBlocksForRows * numBlocksForCols > numPartitions) && (m * n != 0)) { - if (numRowBlocks <= numColBlocks) { - m += 1 - n = math.ceil(targetNumBlocksPerPartition * 1.0 / m).toInt - } else { - n += 1 - m = math.ceil(targetNumBlocksPerPartition * 1.0 / n).toInt - } - numBlocksForRows = math.ceil(numRowBlocks * 1.0 / m) - numBlocksForCols = math.ceil(numColBlocks * 1.0 / n) - } - // If a good partitioning scheme couldn't be found, set the side with the smaller dimension to - // 1 and the other to the number of targetNumBlocksPerPartition - if (m * n == 0) { - if (numRowBlocks <= numColBlocks) { - m = 1 - n = targetNumBlocksPerPartition - } else { - n = 1 - m = targetNumBlocksPerPartition - } - } - (m, n) + private def getPartitionId(i: Int, j: Int): Int = { + require(0 <= i && i < rows, s"Row index $i out of range [0, $rows).") + require(0 <= j && j < cols, s"Column index $j out of range [0, $cols).") + i / rowsPerPart + j / colsPerPart * rowPartitions } /** Checks whether the partitioners have the same characteristics */ override def equals(obj: Any): Boolean = { obj match { case r: GridPartitioner => - (this.numRowBlocks == r.numRowBlocks) && (this.numColBlocks == r.numColBlocks) && - (this.numPartitions == r.numPartitions) + (this.rows == r.rows) && (this.cols == r.cols) && + (this.rowsPerPart == r.rowsPerPart) && (this.colsPerPart == r.colsPerPart) case _ => false } } } +private[mllib] object GridPartitioner { + + def apply(rows: Int, cols: Int, rowsPerPart: Int, colsPerPart: Int): GridPartitioner = { + new GridPartitioner(rows, cols, rowsPerPart, colsPerPart) + } + + def apply(rows: Int, cols: Int, suggestedNumPartitions: Int): GridPartitioner = { + require(suggestedNumPartitions > 0) + val scale = 1.0 / math.sqrt(suggestedNumPartitions) + val rowsPerPart = math.round(math.max(scale * rows, 1.0)).toInt + val colsPerPart = math.round(math.max(scale * cols, 1.0)).toInt + new GridPartitioner(rows, cols, rowsPerPart, colsPerPart) + } +} + /** * Represents a distributed matrix in blocks of local matrices. * @@ -191,7 +154,7 @@ class BlockMatrix( val numColBlocks = math.ceil(numCols() * 1.0 / colsPerBlock).toInt private[mllib] var partitioner: GridPartitioner = - new GridPartitioner(numRowBlocks, numColBlocks, rdd.partitions.length) + GridPartitioner(numRowBlocks, numColBlocks, suggestedNumPartitions = rdd.partitions.size) /** Returns the dimensions of the matrix. */ private def getDim: (Long, Long) = { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index a5016731d1f20..b52057beb7703 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -17,8 +17,10 @@ package org.apache.spark.mllib.linalg.distributed -import org.scalatest.FunSuite +import scala.util.Random + import breeze.linalg.{DenseMatrix => BDM} +import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Matrix} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -51,50 +53,72 @@ class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { assert(gridBasedMat.numCols() === n) } - test("grid partitioner partitioning") { - val partitioner = gridBasedMat.partitioner - assert(partitioner.getPartition((0, 0)) === 0) - assert(partitioner.getPartition((0, 1)) === 0) - assert(partitioner.getPartition((1, 0)) === 1) - assert(partitioner.getPartition((1, 1)) === 1) - assert(partitioner.getPartition((2, 0)) === 2) - assert(partitioner.getPartition((2, 1)) === 2) - assert(partitioner.getPartition((1, 0, 1)) === 1) - assert(partitioner.getPartition((2, 0, 0)) === 2) - - val part2 = new GridPartitioner(10, 20, 10) - assert(part2.getPartition((0, 0)) === 0) - assert(part2.getPartition((0, 1)) === 0) - assert(part2.getPartition((0, 6)) === 2) - assert(part2.getPartition((3, 7)) === 2) - assert(part2.getPartition((3, 8)) === 4) - assert(part2.getPartition((3, 13)) === 6) - assert(part2.getPartition((9, 14)) === 7) - assert(part2.getPartition((9, 15)) === 7) - assert(part2.getPartition((9, 19)) === 9) + test("grid partitioner") { + val random = new Random() + // This should generate a 4x4 grid of 1x2 blocks. + val part0 = GridPartitioner(4, 7, suggestedNumPartitions = 12) + val expected0 = Array( + Array(0, 0, 4, 4, 8, 8, 12), + Array(1, 1, 5, 5, 9, 9, 13), + Array(2, 2, 6, 6, 10, 10, 14), + Array(3, 3, 7, 7, 11, 11, 15)) + for (i <- 0 until 4; j <- 0 until 7) { + assert(part0.getPartition((i, j)) === expected0(i)(j)) + assert(part0.getPartition((i, j, random.nextInt())) === expected0(i)(j)) + } + + intercept[IllegalArgumentException] { + part0.getPartition((-1, 0)) + } + + intercept[IllegalArgumentException] { + part0.getPartition((4, 0)) + } + + intercept[IllegalArgumentException] { + part0.getPartition((0, -1)) + } intercept[IllegalArgumentException] { - part2.getPartition((-1, 0)) + part0.getPartition((0, 7)) + } + + val part1 = GridPartitioner(2, 2, suggestedNumPartitions = 5) + val expected1 = Array( + Array(0, 2), + Array(1, 3)) + for (i <- 0 until 2; j <- 0 until 2) { + assert(part1.getPartition((i, j)) === expected1(i)(j)) + assert(part1.getPartition((i, j, random.nextInt())) === expected1(i)(j)) } + val part2 = GridPartitioner(2, 2, suggestedNumPartitions = 5) + assert(part0 !== part2) + assert(part1 === part2) + + val part3 = new GridPartitioner(2, 3, rowsPerPart = 1, colsPerPart = 2) + val expected3 = Array( + Array(0, 0, 2), + Array(1, 1, 3)) + for (i <- 0 until 2; j <- 0 until 3) { + assert(part3.getPartition((i, j)) === expected3(i)(j)) + assert(part3.getPartition((i, j, random.nextInt())) === expected3(i)(j)) + } + + val part4 = GridPartitioner(2, 3, rowsPerPart = 1, colsPerPart = 2) + assert(part3 === part4) + intercept[IllegalArgumentException] { - part2.getPartition((10, 0)) + new GridPartitioner(2, 2, rowsPerPart = 0, colsPerPart = 1) } intercept[IllegalArgumentException] { - part2.getPartition((9, 20)) + GridPartitioner(2, 2, rowsPerPart = 1, colsPerPart = 0) } - val part3 = new GridPartitioner(20, 10, 10) - assert(part3.getPartition((0, 0)) === 0) - assert(part3.getPartition((1, 0)) === 0) - assert(part3.getPartition((6, 0)) === 1) - assert(part3.getPartition((7, 3)) === 1) - assert(part3.getPartition((8, 3)) === 2) - assert(part3.getPartition((13, 3)) === 3) - assert(part3.getPartition((14, 9)) === 8) - assert(part3.getPartition((15, 9)) === 8) - assert(part3.getPartition((19, 9)) === 9) + intercept[IllegalArgumentException] { + GridPartitioner(2, 2, suggestedNumPartitions = 0) + } } test("toBreeze and toLocalMatrix") { From e1d3ee871c8d73eb111336c6d22d160728582be2 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 23:46:24 -0800 Subject: [PATCH 20/21] minor updates --- .../linalg/distributed/BlockMatrix.scala | 91 ++++++++++--------- 1 file changed, 48 insertions(+), 43 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index de80367ba9211..0ab74ba294535 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -52,17 +52,17 @@ private[mllib] class GridPartitioner( * Returns the index of the partition the input coordinate belongs to. * * @param key The coordinate (i, j) or a tuple (i, j, k), where k is the inner index used in - * multiplication. + * multiplication. k is ignored in computing partitions. * @return The index of the partition, which the coordinate belongs to. */ override def getPartition(key: Any): Int = { key match { case (i: Int, j: Int) => getPartitionId(i, j) - case (i: Int, j: Int, _) => + case (i: Int, j: Int, _: Int) => getPartitionId(i, j) case _ => - throw new IllegalArgumentException(s"Unrecognized key: $key") + throw new IllegalArgumentException(s"Unrecognized key: $key.") } } @@ -73,7 +73,6 @@ private[mllib] class GridPartitioner( i / rowsPerPart + j / colsPerPart * rowPartitions } - /** Checks whether the partitioners have the same characteristics */ override def equals(obj: Any): Boolean = { obj match { case r: GridPartitioner => @@ -87,10 +86,12 @@ private[mllib] class GridPartitioner( private[mllib] object GridPartitioner { + /** Creates a new [[GridPartitioner]] instance. */ def apply(rows: Int, cols: Int, rowsPerPart: Int, colsPerPart: Int): GridPartitioner = { new GridPartitioner(rows, cols, rowsPerPart, colsPerPart) } + /** Creates a new [[GridPartitioner]] instance with the input suggested number of partitions. */ def apply(rows: Int, cols: Int, suggestedNumPartitions: Int): GridPartitioner = { require(suggestedNumPartitions > 0) val scale = 1.0 / math.sqrt(suggestedNumPartitions) @@ -103,24 +104,25 @@ private[mllib] object GridPartitioner { /** * Represents a distributed matrix in blocks of local matrices. * - * @param rdd The RDD of SubMatrices (local matrices) that form this matrix - * @param nRows Number of rows of this matrix. If the supplied value is less than or equal to zero, - * the number of rows will be calculated when `numRows` is invoked. - * @param nCols Number of columns of this matrix. If the supplied value is less than or equal to - * zero, the number of columns will be calculated when `numCols` is invoked. + * @param blocks The RDD of sub-matrix blocks (blockRowIndex, blockColIndex, sub-matrix) that form + * this distributed matrix. * @param rowsPerBlock Number of rows that make up each block. The blocks forming the final * rows are not required to have the given number of rows * @param colsPerBlock Number of columns that make up each block. The blocks forming the final * columns are not required to have the given number of columns + * @param nRows Number of rows of this matrix. If the supplied value is less than or equal to zero, + * the number of rows will be calculated when `numRows` is invoked. + * @param nCols Number of columns of this matrix. If the supplied value is less than or equal to + * zero, the number of columns will be calculated when `numCols` is invoked. */ class BlockMatrix( - val rdd: RDD[((Int, Int), Matrix)], - private var nRows: Long, - private var nCols: Long, + val blocks: RDD[((Int, Int), Matrix)], val rowsPerBlock: Int, - val colsPerBlock: Int) extends DistributedMatrix with Logging { + val colsPerBlock: Int, + private var nRows: Long, + private var nCols: Long) extends DistributedMatrix with Logging { - private type SubMatrix = ((Int, Int), Matrix) // ((blockRowIndex, blockColIndex), matrix) + private type MatrixBlock = ((Int, Int), Matrix) // ((blockRowIndex, blockColIndex), sub-matrix) /** * Alternate constructor for BlockMatrix without the input of the number of rows and columns. @@ -135,18 +137,16 @@ class BlockMatrix( rdd: RDD[((Int, Int), Matrix)], rowsPerBlock: Int, colsPerBlock: Int) = { - this(rdd, 0L, 0L, rowsPerBlock, colsPerBlock) + this(rdd, rowsPerBlock, colsPerBlock, 0L, 0L) } - private lazy val dims: (Long, Long) = getDim - override def numRows(): Long = { - if (nRows <= 0L) nRows = dims._1 + if (nRows <= 0L) estimateDim() nRows } override def numCols(): Long = { - if (nCols <= 0L) nCols = dims._2 + if (nCols <= 0L) estimateDim() nCols } @@ -154,26 +154,31 @@ class BlockMatrix( val numColBlocks = math.ceil(numCols() * 1.0 / colsPerBlock).toInt private[mllib] var partitioner: GridPartitioner = - GridPartitioner(numRowBlocks, numColBlocks, suggestedNumPartitions = rdd.partitions.size) - - /** Returns the dimensions of the matrix. */ - private def getDim: (Long, Long) = { - val (rows, cols) = rdd.map { case ((blockRowIndex, blockColIndex), mat) => - (blockRowIndex * rowsPerBlock + mat.numRows, blockColIndex * colsPerBlock + mat.numCols) - }.reduce((x0, x1) => (math.max(x0._1, x1._1), math.max(x0._2, x1._2))) - - (math.max(rows, nRows), math.max(cols, nCols)) + GridPartitioner(numRowBlocks, numColBlocks, suggestedNumPartitions = blocks.partitions.size) + + /** Estimates the dimensions of the matrix. */ + private def estimateDim(): Unit = { + val (rows, cols) = blocks.map { case ((blockRowIndex, blockColIndex), mat) => + (blockRowIndex.toLong * rowsPerBlock + mat.numRows, + blockColIndex.toLong * colsPerBlock + mat.numCols) + }.reduce { (x0, x1) => + (math.max(x0._1, x1._1), math.max(x0._2, x1._2)) + } + if (nRows <= 0L) nRows = rows + assert(rows <= nRows, s"The number of rows $rows is more than claimed $nRows.") + if (nCols <= 0L) nCols = cols + assert(cols <= nCols, s"The number of columns $cols is more than claimed $nCols.") } - /** Cache the underlying RDD. */ - def cache(): BlockMatrix = { - rdd.cache() + /** Caches the underlying RDD. */ + def cache(): this.type = { + blocks.cache() this } - /** Set the storage level for the underlying RDD. */ - def persist(storageLevel: StorageLevel): BlockMatrix = { - rdd.persist(storageLevel) + /** Persists the underlying RDD with the specified storage level. */ + def persist(storageLevel: StorageLevel): this.type = { + blocks.persist(storageLevel) this } @@ -185,22 +190,22 @@ class BlockMatrix( s"Int.MaxValue. Currently numCols: ${numCols()}") require(numRows() * numCols() < Int.MaxValue, "The length of the values array must be " + s"less than Int.MaxValue. Currently numRows * numCols: ${numRows() * numCols()}") - val nRows = numRows().toInt - val nCols = numCols().toInt - val mem = nRows * nCols / 125000 + val m = numRows().toInt + val n = numCols().toInt + val mem = m * n / 125000 if (mem > 500) logWarning(s"Storing this matrix will require $mem MB of memory!") - val parts = rdd.collect() - val values = new Array[Double](nRows * nCols) - parts.foreach { case ((blockRowIndex, blockColIndex), block) => + val localBlocks = blocks.collect() + val values = new Array[Double](m * n) + localBlocks.foreach { case ((blockRowIndex, blockColIndex), submat) => val rowOffset = blockRowIndex * rowsPerBlock val colOffset = blockColIndex * colsPerBlock - block.foreachActive { (i, j, v) => - val indexOffset = (j + colOffset) * nRows + rowOffset + i + submat.foreachActive { (i, j, v) => + val indexOffset = (j + colOffset) * m + rowOffset + i values(indexOffset) = v } } - new DenseMatrix(nRows, nCols, values) + new DenseMatrix(m, n, values) } /** Collects data and assembles a local dense breeze matrix (for test only). */ From feb32a78a0412975133e761b1d1e460eb9a7f42a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 23:49:21 -0800 Subject: [PATCH 21/21] update tests --- .../linalg/distributed/BlockMatrixSuite.scala | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index b52057beb7703..05efbc8e8d0b8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -33,19 +33,18 @@ class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext { val colPerPart = 2 val numPartitions = 3 var gridBasedMat: BlockMatrix = _ - type SubMatrix = ((Int, Int), Matrix) override def beforeAll() { super.beforeAll() - val entries: Seq[SubMatrix] = Seq( - new SubMatrix((0, 0), new DenseMatrix(2, 2, Array(1.0, 0.0, 0.0, 2.0))), - new SubMatrix((0, 1), new DenseMatrix(2, 2, Array(0.0, 1.0, 0.0, 0.0))), - new SubMatrix((1, 0), new DenseMatrix(2, 2, Array(3.0, 0.0, 1.0, 1.0))), - new SubMatrix((1, 1), new DenseMatrix(2, 2, Array(1.0, 2.0, 0.0, 1.0))), - new SubMatrix((2, 1), new DenseMatrix(1, 2, Array(1.0, 5.0)))) + val blocks: Seq[((Int, Int), Matrix)] = Seq( + ((0, 0), new DenseMatrix(2, 2, Array(1.0, 0.0, 0.0, 2.0))), + ((0, 1), new DenseMatrix(2, 2, Array(0.0, 1.0, 0.0, 0.0))), + ((1, 0), new DenseMatrix(2, 2, Array(3.0, 0.0, 1.0, 1.0))), + ((1, 1), new DenseMatrix(2, 2, Array(1.0, 2.0, 0.0, 1.0))), + ((2, 1), new DenseMatrix(1, 2, Array(1.0, 5.0)))) - gridBasedMat = new BlockMatrix(sc.parallelize(entries, numPartitions), rowPerPart, colPerPart) + gridBasedMat = new BlockMatrix(sc.parallelize(blocks, numPartitions), rowPerPart, colPerPart) } test("size") {