You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2016/06/23 02:08:13 UTC
flink git commit: [FLINK-3919] [ml] Implement DistributedRowMatrix
Repository: flink
Updated Branches:
refs/heads/master 08b075aa5 -> 63504a3cb
[FLINK-3919] [ml] Implement DistributedRowMatrix
This closes #1996.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/63504a3c
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/63504a3c
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/63504a3c
Branch: refs/heads/master
Commit: 63504a3cb331b9fb29cf833eeaa66afe5ba5d5f1
Parents: 08b075a
Author: chobeat <si...@gmail.com>
Authored: Tue May 17 14:19:15 2016 +0200
Committer: Chiwan Park <ch...@apache.org>
Committed: Thu Jun 23 11:01:33 2016 +0900
----------------------------------------------------------------------
.../ml/math/distributed/DistributedMatrix.scala | 39 +++++
.../math/distributed/DistributedRowMatrix.scala | 161 +++++++++++++++++++
.../distributed/DistributedRowMatrixSuite.scala | 104 ++++++++++++
3 files changed, 304 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/63504a3c/flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedMatrix.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedMatrix.scala b/flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedMatrix.scala
new file mode 100644
index 0000000..f34641e
--- /dev/null
+++ b/flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedMatrix.scala
@@ -0,0 +1,39 @@
+/*
+ * 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.flink.ml.math.distributed
+
+/** Common trait used by distributed data structures representing a matrix. */
+trait DistributedMatrix {
+ /** Returns number of rows in matrix.
+ *
+ * @return Number of rows
+ */
+ def numRows: Int
+
+ /** Returns number of columns in matrix.
+ *
+ * @return Number of columns
+ */
+ def numCols: Int
+}
+
+object DistributedMatrix {
+ type MatrixColIndex = Int
+ type MatrixRowIndex = Int
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/63504a3c/flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala b/flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala
new file mode 100644
index 0000000..9092e5c
--- /dev/null
+++ b/flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala
@@ -0,0 +1,161 @@
+/*
+ * 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.flink.ml.math.distributed
+
+import org.apache.flink.api.scala._
+import org.apache.flink.ml.math.Breeze._
+import org.apache.flink.ml.math.distributed.DistributedMatrix._
+import org.apache.flink.ml.math._
+
+/** Represents distributed row-major matrix.
+ *
+ * @param data [[DataSet]] which contains [[IndexedRow]]s
+ * @param numRows Number of rows
+ * @param numCols Number of columns
+ */
+class DistributedRowMatrix(
+ val data: DataSet[IndexedRow],
+ val numRows: Int,
+ val numCols: Int
+) extends DistributedMatrix {
+
+ /** Collects the data in the form of a sequence of coordinates associated with their values.
+ * This operation immediately triggers program execution.
+ */
+ def toCOO: Seq[(MatrixRowIndex, MatrixColIndex, Double)] = {
+ val localRows = data.collect()
+
+ for {
+ IndexedRow(rowIndex, vector) <- localRows
+ (columnIndex, value) <- vector
+ } yield (rowIndex, columnIndex, value)
+ }
+
+ /** Collects the data in the form of a SparseMatrix. This operation immediately triggers program
+ * execution.
+ */
+ def toLocalSparseMatrix: SparseMatrix = {
+ val localMatrix = SparseMatrix.fromCOO(this.numRows, this.numCols, this.toCOO)
+ require(localMatrix.numRows == this.numRows)
+ require(localMatrix.numCols == this.numCols)
+
+ localMatrix
+ }
+
+ // TODO: convert to dense representation on the distributed matrix and collect it afterward
+ /** Collects the data in the form of a DenseMatrix. This operation immediately triggers program
+ * execution.
+ */
+ def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
+
+ /** Applies a high-order function to couple of rows.
+ *
+ * @param func a function to be applied
+ * @param other a [[DistributedRowMatrix]] to apply the function together
+ */
+ def byRowOperation(
+ func: (Vector, Vector) => Vector,
+ other: DistributedRowMatrix
+ ): DistributedRowMatrix = {
+ val otherData = other.data
+ require(this.numCols == other.numCols)
+ require(this.numRows == other.numRows)
+
+ val result = this.data
+ .fullOuterJoin(otherData)
+ .where("rowIndex")
+ .equalTo("rowIndex")(
+ (left: IndexedRow, right: IndexedRow) => {
+ val row1 = Option(left) match {
+ case Some(row: IndexedRow) => row
+ case None =>
+ IndexedRow(right.rowIndex, SparseVector.fromCOO(right.values.size, List((0, 0.0))))
+ }
+ val row2 = Option(right) match {
+ case Some(row: IndexedRow) => row
+ case None =>
+ IndexedRow(left.rowIndex, SparseVector.fromCOO(left.values.size, List((0, 0.0))))
+ }
+ IndexedRow(row1.rowIndex, func(row1.values, row2.values))
+ }
+ )
+ new DistributedRowMatrix(result, numRows, numCols)
+ }
+
+ /** Adds this matrix to another matrix.
+ *
+ * @param other a [[DistributedRowMatrix]] to be added
+ */
+ def add(other: DistributedRowMatrix): DistributedRowMatrix = {
+ val addFunction = (x: Vector, y: Vector) => (x.asBreeze + y.asBreeze).fromBreeze
+ this.byRowOperation(addFunction, other)
+ }
+
+ /** Subtracts another matrix from this matrix.
+ *
+ * @param other a [[DistributedRowMatrix]] to be subtracted from this matrix
+ */
+ def subtract(other: DistributedRowMatrix): DistributedRowMatrix = {
+ val subFunction = (x: Vector, y: Vector) => (x.asBreeze - y.asBreeze).fromBreeze
+ this.byRowOperation(subFunction, other)
+ }
+}
+
+object DistributedRowMatrix {
+
+ /** Builds a [[DistributedRowMatrix]] from a [[DataSet]] in COO.
+ *
+ * @param data [[DataSet]] which contains matrix elements in the form of
+ * (row index, column index, value)
+ * @param numRows Number of rows
+ * @param numCols Number of columns
+ * @param isSorted If false, sorts the row to properly build the matrix representation.
+ * If already sorted, set this parameter to true to skip sorting.
+ */
+ def fromCOO(data: DataSet[(MatrixRowIndex, MatrixColIndex, Double)],
+ numRows: Int,
+ numCols: Int,
+ isSorted: Boolean = false
+ ): DistributedRowMatrix = {
+ val vectorData: DataSet[(MatrixRowIndex, SparseVector)] = data
+ .groupBy(0)
+ .reduceGroup(sparseRow => {
+ require(sparseRow.nonEmpty)
+ val sortedRow =
+ if (isSorted) {
+ sparseRow.toList
+ } else {
+ sparseRow.toList.sortBy(row => row._2)
+ }
+ val (indices, values) = sortedRow.map(x => (x._2, x._3)).unzip
+ (sortedRow.head._1, SparseVector(numCols, indices.toArray, values.toArray))
+ })
+
+ val zippedData = vectorData.map(x => IndexedRow(x._1.toInt, x._2))
+
+ new DistributedRowMatrix(zippedData, numRows, numCols)
+ }
+}
+
+/** Represents a row in row-major matrix. */
+case class IndexedRow(rowIndex: MatrixRowIndex, values: Vector) extends Ordered[IndexedRow] {
+ def compare(other: IndexedRow) = this.rowIndex.compare(other.rowIndex)
+
+ override def toString: String = s"($rowIndex, ${values.toString})"
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/63504a3c/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrixSuite.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrixSuite.scala b/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrixSuite.scala
new file mode 100644
index 0000000..598b0f3
--- /dev/null
+++ b/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrixSuite.scala
@@ -0,0 +1,104 @@
+/*
+ * 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.flink.ml.math.distributed
+
+import org.apache.flink.api.scala._
+import org.apache.flink.test.util.FlinkTestBase
+import org.scalatest.{FlatSpec, Matchers}
+
+class DistributedRowMatrixSuite extends FlatSpec with Matchers with FlinkTestBase {
+ behavior of "DistributedRowMatrix"
+
+ val rawSampleData = List(
+ (0, 0, 3.0),
+ (0, 1, 3.0),
+ (0, 3, 4.0),
+ (2, 3, 4.0),
+ (1, 4, 3.0),
+ (1, 1, 3.0),
+ (2, 1, 3.0),
+ (2, 2, 3.0)
+ )
+
+ it should "contain the initialization data" in {
+ val env = ExecutionEnvironment.getExecutionEnvironment
+ val rowDataset = env.fromCollection(rawSampleData)
+ val dmatrix = DistributedRowMatrix.fromCOO(rowDataset, 3, 5)
+
+ dmatrix.toCOO.toSet.filter(_._3 != 0) shouldBe rawSampleData.toSet
+ }
+
+ it should "return the correct dimensions when provided by the user" in {
+ val env = ExecutionEnvironment.getExecutionEnvironment
+ val rowDataset = env.fromCollection(rawSampleData)
+ val dmatrix = DistributedRowMatrix.fromCOO(rowDataset, 3, 5)
+
+ dmatrix.numCols shouldBe 5
+ dmatrix.numRows shouldBe 3
+ }
+
+
+ it should "return a sparse local matrix containing the initialization data" in {
+ val env = ExecutionEnvironment.getExecutionEnvironment
+ val rowDataset = env.fromCollection(rawSampleData)
+ val dmatrix = DistributedRowMatrix.fromCOO(rowDataset, 3, 5)
+
+ dmatrix.toLocalSparseMatrix.iterator.filter(_._3 != 0).toSet shouldBe rawSampleData.toSet
+ }
+
+ it should "return a dense local matrix containing the initialization data" in {
+ val env = ExecutionEnvironment.getExecutionEnvironment
+ val rowDataset = env.fromCollection(rawSampleData)
+ val dmatrix = DistributedRowMatrix.fromCOO(rowDataset, 3, 5)
+
+ dmatrix.toLocalDenseMatrix.iterator.filter(_._3 != 0).toSet shouldBe rawSampleData.toSet
+ }
+
+ "add" should "correctly add two distributed row matrices" in {
+ val env = ExecutionEnvironment.getExecutionEnvironment
+ val rawSampleSum1 = List(
+ (0, 0, 1.0),
+ (7, 4, 3.0),
+ (0, 1, 8.0),
+ (2, 8, 12.0)
+ )
+
+ val rawSampleSum2 = List(
+ (0, 0, 2.0),
+ (3, 4, 4.0),
+ (2, 8, 8.0)
+ )
+
+ val addBlockMatrix1 = DistributedRowMatrix.fromCOO(env.fromCollection(rawSampleSum1), 10, 10)
+ val addBlockMatrix2 = DistributedRowMatrix.fromCOO(env.fromCollection(rawSampleSum2), 10, 10)
+
+ val expected = List(
+ (0, 0, 3.0),
+ (0, 1, 8.0),
+ (3, 4, 4.0),
+ (2, 8, 20.0),
+ (7, 4, 3.0)
+ )
+ val result = addBlockMatrix1
+ .add(addBlockMatrix2)
+ .toLocalSparseMatrix
+ .filter(_._3 != 0.0)
+ result.toSet shouldEqual expected.toSet
+ }
+}