Skip to content
Closed
Changes from 1 commit
Commits
Show all changes
24 commits
Select commit Hold shift + click to select a range
b693209
Ready for Pull request
Nov 11, 2014
f378e16
[SPARK-3974] Block Matrix Abstractions ready
Nov 11, 2014
aa8f086
[SPARK-3974] Additional comments added
Nov 11, 2014
589fbb6
[SPARK-3974] Code review feedback addressed
Nov 14, 2014
19c17e8
[SPARK-3974] Changed blockIdRow and blockIdCol
Nov 14, 2014
b05aabb
[SPARK-3974] Updated tests to reflect changes
brkyvz Nov 14, 2014
645afbe
[SPARK-3974] Pull latest master
brkyvz Nov 14, 2014
49b9586
[SPARK-3974] Updated testing utils from master
brkyvz Nov 14, 2014
d033861
[SPARK-3974] Removed SubMatrixInfo and added constructor without part…
brkyvz Nov 15, 2014
9ae85aa
[SPARK-3974] Made partitioner a variable inside BlockMatrix instead o…
brkyvz Nov 20, 2014
ab6cde0
[SPARK-3974] Modifications cleaning code up, making size calculation …
brkyvz Jan 14, 2015
ba414d2
[SPARK-3974] fixed frobenius norm
brkyvz Jan 14, 2015
239ab4b
[SPARK-3974] Addressed @jkbradley's comments
brkyvz Jan 19, 2015
1e8bb2a
[SPARK-3974] Change return type of cache and persist
brkyvz Jan 20, 2015
1a63b20
[SPARK-3974] Remove setPartition method. Isn't required
brkyvz Jan 20, 2015
eebbdf7
preliminary changes addressing code review
brkyvz Jan 21, 2015
f9d664b
updated API and modified partitioning scheme
brkyvz Jan 21, 2015
1694c9e
almost finished addressing comments
brkyvz Jan 27, 2015
140f20e
Merge branch 'master' of github.com:apache/spark into SPARK-3974
brkyvz Jan 27, 2015
5eecd48
fixed gridPartitioner and added tests
brkyvz Jan 27, 2015
24ec7b8
update grid partitioner
mengxr Jan 28, 2015
e1d3ee8
minor updates
mengxr Jan 28, 2015
feb32a7
update tests
mengxr Jan 28, 2015
a8eace2
Merge pull request #2 from mengxr/brkyvz-SPARK-3974
brkyvz Jan 28, 2015
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
almost finished addressing comments
  • Loading branch information
brkyvz committed Jan 27, 2015
commit 1694c9e3be57b83e6342a75a454614a404a97348
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
}

Expand All @@ -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
Expand All @@ -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 {

Expand All @@ -115,25 +118,18 @@ 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
* columns are not required to have the given number of columns
*/
def this(
rdd: RDD[((Int, Int), Matrix)],
numRowBlocks: Int,
numColBlocks: Int,
rowsPerBlock: Int,
colsPerBlock: Int) = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The list of arguments cannot provide the complete info about the matrix. For example, if the last block row and the last block column are all missing. Then you cannot figure out the exact matrix size from this list of arguments.

It would be necessary to have numRows, numCols, rowsPerBlock, colsPerBlock, and the RDD as input. We can provide factory methods (in follow-up PRs) to create block matrices from other formats, which could figure out the exact numRows and numCols and use them in the constructor.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will it really be the case that the whole row of blocks will be missing for the last row? That means that those rows (or columns) contain no information. Then why store (use) them?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We cannot make such assumption about the data. It is not rare that we have an empty column/row, which is the last column/row and the only column/row in the last column/row block. For example, in the popular mnist-digit dataset, the last column of the training data is empty.

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 = {
Expand All @@ -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. */
Expand All @@ -210,14 +179,14 @@ class BlockMatrix(
s"Int.MaxValue. Currently numCols: ${numCols()}")
val nRows = numRows().toInt
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd check numRows and numCols here before converting to Int and throw an error if the matrix is too large.

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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It would be good if we use foreachActive is that PR is merged first.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I can modify it in the next 3 upcoming PRs (matrix multiplication-addition, conversions, repartition) if it doesn't.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

foreachActive is merged:)

while (j < block.numCols) {
Expand Down