Skip to content

[SPARK-3974][MLlib] Distributed Block Matrix Abstractions #3200

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 24 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
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
Original file line number Diff line number Diff line change
@@ -0,0 +1,216 @@
/*
* 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.{Logging, Partitioner}
import org.apache.spark.mllib.linalg.{DenseMatrix, Matrix}
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel

/**
* A grid partitioner, which uses a regular grid to partition coordinates.
*
* @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 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 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. 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, _: Int) =>
getPartitionId(i, j)
case _ =>
throw new IllegalArgumentException(s"Unrecognized key: $key.")
}
}

/** Partitions sub-matrices as blocks with neighboring sub-matrices. */
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
}

override def equals(obj: Any): Boolean = {
obj match {
case r: GridPartitioner =>
(this.rows == r.rows) && (this.cols == r.cols) &&
(this.rowsPerPart == r.rowsPerPart) && (this.colsPerPart == r.colsPerPart)
case _ =>
false
}
}
}

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)
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.
*
* @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 blocks: RDD[((Int, Int), Matrix)],
val rowsPerBlock: Int,
val colsPerBlock: Int,
private var nRows: Long,
private var nCols: Long) extends DistributedMatrix with Logging {

private type MatrixBlock = ((Int, Int), Matrix) // ((blockRowIndex, blockColIndex), sub-matrix)

/**
* 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 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)],
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, rowsPerBlock, colsPerBlock, 0L, 0L)
}

override def numRows(): Long = {
if (nRows <= 0L) estimateDim()
nRows
}

override def numCols(): Long = {
if (nCols <= 0L) estimateDim()
nCols
}

val numRowBlocks = math.ceil(numRows() * 1.0 / rowsPerBlock).toInt
val numColBlocks = math.ceil(numCols() * 1.0 / colsPerBlock).toInt

private[mllib] var partitioner: GridPartitioner =
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))
}
Copy link
Member

Choose a reason for hiding this comment

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

style: indentation

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.")
}

/** Caches the underlying RDD. */
def cache(): this.type = {
blocks.cache()
this
}

/** Persists the underlying RDD with the specified storage level. */
def persist(storageLevel: StorageLevel): this.type = {
blocks.persist(storageLevel)
this
}

/** Collect the distributed matrix on the driver as a `DenseMatrix`. */
def toLocalMatrix(): Matrix = {
Copy link
Member

Choose a reason for hiding this comment

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

Rename toLocal?

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()}")
require(numRows() * numCols() < Int.MaxValue, "The length of the values array must be " +
s"less than Int.MaxValue. Currently numRows * numCols: ${numRows() * numCols()}")
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 localBlocks = blocks.collect()
val values = new Array[Double](m * n)
localBlocks.foreach { case ((blockRowIndex, blockColIndex), submat) =>
val rowOffset = blockRowIndex * rowsPerBlock
val colOffset = blockColIndex * colsPerBlock
submat.foreachActive { (i, j, v) =>
val indexOffset = (j + colOffset) * m + rowOffset + i
values(indexOffset) = v
}
}
new DenseMatrix(m, n, values)
}

/** Collects data and assembles a local dense breeze matrix (for test only). */
private[mllib] def toBreeze(): BDM[Double] = {
Copy link
Member

Choose a reason for hiding this comment

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

If this is just for testing, then I'd make it private[distributed]. If it should fit with other APIs, then it should return a Matrix, not a DenseMatrix.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is how it is currently for all distributed matrices, and each return a BDM. Maybe we can change all of them later.

Copy link
Member

Choose a reason for hiding this comment

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

Oh I see. Yeah, I think we should change that later, but later is fine since it's internal.

val localMat = toLocalMatrix()
new BDM[Double](localMat.numRows, localMat.numCols, localMat.toArray)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,135 @@
/*
* 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 scala.util.Random

Copy link
Contributor

Choose a reason for hiding this comment

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

remove this line. scalatest counts as a 3rd-party import but not scala import.

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

class BlockMatrixSuite extends FunSuite with MLlibTestSparkContext {

val m = 5
Copy link
Member

Choose a reason for hiding this comment

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

I would put these fixed values in a private BlockMatrixSuite object and then import them inside the class.

val n = 4
val rowPerPart = 2
val colPerPart = 2
val numPartitions = 3
var gridBasedMat: BlockMatrix = _

override def beforeAll() {
super.beforeAll()

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(blocks, numPartitions), rowPerPart, colPerPart)
}

test("size") {
assert(gridBasedMat.numRows() === m)
assert(gridBasedMat.numCols() === n)
}

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] {
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] {
new GridPartitioner(2, 2, rowsPerPart = 0, colsPerPart = 1)
}

intercept[IllegalArgumentException] {
GridPartitioner(2, 2, rowsPerPart = 1, colsPerPart = 0)
}

intercept[IllegalArgumentException] {
GridPartitioner(2, 2, suggestedNumPartitions = 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.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.toLocalMatrix() === dense)
assert(gridBasedMat.toBreeze() === expected)
}
}