You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by chobeat <gi...@git.apache.org> on 2016/05/17 13:05:04 UTC

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

GitHub user chobeat opened a pull request:

    https://github.com/apache/flink/pull/1996

    [FLINK-3919][flink-ml] Distributed Linear Algebra: row-based matrix

    First PR of the Distributed Linear Algebra contribution.
    
    It contains a minimal implementation of a row-based distributed matrix and the following operations:
    
    Build from COO
    
    Conversion from distributed to: 
    * local dense format
    * local sparse format 
    * local COO format.
    
    by-row UDF on two matrices
    sum
    subtraction
    
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/radicalbit/flink FLINK-1873-c

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1996.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1996
    
----
commit 09b75c32d7d433cc66619075b9d53c3a9ae7e6da
Author: chobeat <si...@gmail.com>
Date:   2016-05-17T12:19:15Z

    DistributedRowMatrix

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #1996: [FLINK-3919][flink-ml] Distributed Linear Algebra:...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r68012799
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,162 @@
    +/*
    + * 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._
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @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.
    +    * @return
    +    */
    +  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
    +    * @return
    +    */
    +  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
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  /**
    +    * Apply a high-order function to couple of rows
    +    * @param fun
    +    * @param other
    +    * @return
    +    */
    +  def byRowOperation(fun: (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, fun(row1.values, row2.values))
    +          }
    +      )
    +    new DistributedRowMatrix(result, numRows, numCols)
    +  }
    +
    +  /**
    +    * Add the matrix to another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def sum(other: DistributedRowMatrix): DistributedRowMatrix = {
    --- End diff --
    
    My reply probably got lost because I posted it on Jira instead of Github, sorry.
    
    "Umh, probably you're right. I checked breeze and they use addition for matrix addition and sum for element-wise sum."


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #1996: [FLINK-3919][flink-ml] Distributed Linear Algebra:...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r68041009
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,162 @@
    +/*
    + * 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._
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @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.
    +    * @return
    +    */
    +  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
    +    * @return
    +    */
    +  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
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  /**
    +    * Apply a high-order function to couple of rows
    +    * @param fun
    +    * @param other
    +    * @return
    +    */
    +  def byRowOperation(fun: (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, fun(row1.values, row2.values))
    +          }
    +      )
    +    new DistributedRowMatrix(result, numRows, numCols)
    +  }
    +
    +  /**
    +    * Add the matrix to another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def sum(other: DistributedRowMatrix): DistributedRowMatrix = {
    --- End diff --
    
    It is not your fault. :-) It seems my fault. Sorry. Let's merge this to master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65067355
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    It is and I don't remember why we went for `Int`. Should I move everything to Long?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #1996: [FLINK-3919][flink-ml] Distributed Linear Algebra:...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65688309
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,162 @@
    +/*
    + * 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._
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @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.
    +    * @return
    +    */
    +  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
    +    * @return
    +    */
    +  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
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  /**
    +    * Apply a high-order function to couple of rows
    +    * @param fun
    +    * @param other
    +    * @return
    +    */
    +  def byRowOperation(fun: (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, fun(row1.values, row2.values))
    +          }
    +      )
    +    new DistributedRowMatrix(result, numRows, numCols)
    +  }
    +
    +  /**
    +    * Add the matrix to another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def sum(other: DistributedRowMatrix): DistributedRowMatrix = {
    --- End diff --
    
    Is `add` more proper name for this method? Please do not update this PR if you agree with me but just notify me because I'm rebasing this PR on current master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65066995
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    I simplify my comments. Sorry for confusing. Type of number of columns could be `Int` only but we should provide the number of rows in both methods (DataSet and raw number).
    
    By the way, isn't `Long` reasonable for type of number of rows?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65076170
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    I think it is okay.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-b...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on the pull request:

    https://github.com/apache/flink/pull/1996
  
    @chiwanpark I think it should be better to leave to the user the computation of the dimensionality. I tried different options and all of them are sub-optimal. I would leave this feature for later as a next step if it's ok.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-b...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65309988
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,179 @@
    +/*
    + * 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 breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.Breeze._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => numRows.collect().head
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => numCols.collect().head
    +  }
    +
    +  lazy val numRows: DataSet[Int] = numRowsOpt match {
    +    case Some(rows) => data.getExecutionEnvironment.fromElements(rows)
    +    case None => data.max("rowIndex").map(_.rowIndex + 1)
    +  }
    +
    +  lazy val numCols: DataSet[Int] = numColsOpt match {
    +    case Some(cols) => data.getExecutionEnvironment.fromElements(cols)
    +    case None => data.first(1).map(_.values.size)
    +  }
    +
    +  val getRowData = data
    +
    +  /**
    +    * Collects the data in the form of a sequence of coordinates associated with their values.
    +    * @return
    +    */
    +  def toCOO: Seq[(Int, Int, 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
    +    * @return
    +    */
    +  def toLocalSparseMatrix: SparseMatrix = {
    +    val localMatrix =
    +      SparseMatrix.fromCOO(this.getNumRows, this.getNumCols, this.toCOO)
    +    require(localMatrix.numRows == this.getNumRows)
    +    require(localMatrix.numCols == this.getNumCols)
    +    localMatrix
    +  }
    +
    +  //TODO: convert to dense representation on the distributed matrix and collect it afterward
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  /**
    +    * Apply a high-order function to couple of rows
    +    * @param fun
    +    * @param other
    +    * @return
    +    */
    +  def byRowOperation(fun: (Vector, Vector) => Vector,
    +                     other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val otherData = other.getRowData
    +    require(this.getNumCols == other.getNumCols)
    +    require(this.getNumRows == other.getNumRows)
    +
    +    val result = this.data
    +      .fullOuterJoin(otherData)
    +      .where("rowIndex")
    +      .equalTo("rowIndex")(
    +          (left: IndexedRow, right: IndexedRow) => {
    +            val row1 = Option(left).getOrElse(IndexedRow(
    +                    right.rowIndex,
    +                    SparseVector.fromCOO(right.values.size, List((0, 0.0)))))
    +            val row2 = Option(right).getOrElse(IndexedRow(
    +                    left.rowIndex,
    +                    SparseVector.fromCOO(left.values.size, List((0, 0.0)))))
    --- End diff --
    
    I would like to rewrite this block like following to avoid create unnecessary `IndexedRow` object:
    
    ```scala
    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))))
    }
    ```
     


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65060682
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    As I said above, I would like to change return type of these methods to `DataSet[Long]`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65102583
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,182 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    +
    +  val getRowData = data
    +
    +  private def calcCols: Int =
    +    data.first(1).collect().headOption match {
    +      case Some(vector) => vector.values.size
    +      case None => 0
    +    }
    +
    +  /**
    +    * Collects the data in the form of a sequence of coordinates associated with their values.
    +    * @return
    +    */
    +  def toCOO: Seq[(Int, Int, 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
    +    * @return
    +    */
    +  def toLocalSparseMatrix: SparseMatrix = {
    +    val localMatrix =
    +      SparseMatrix.fromCOO(this.getNumRows, this.getNumCols, this.toCOO)
    +    require(localMatrix.numRows == this.getNumRows)
    +    require(localMatrix.numCols == this.getNumCols)
    +    localMatrix
    +  }
    +
    +  //TODO: convert to dense representation on the distributed matrix and collect it afterward
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  /**
    +    * Apply a high-order function to couple of rows
    +    * @param fun
    +    * @param other
    +    * @return
    +    */
    +  def byRowOperation(fun: (Vector, Vector) => Vector,
    +                     other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val otherData = other.getRowData
    +    require(this.getNumCols == other.getNumCols)
    +    require(this.getNumRows == other.getNumRows)
    +
    +
    +    val result = this.data
    +      .fullOuterJoin(otherData)
    +      .where("rowIndex")
    +      .equalTo("rowIndex")(
    +          (left: IndexedRow, right: IndexedRow) => {
    +            val row1 = Option(left).getOrElse(IndexedRow(
    +                    right.rowIndex,
    +                    SparseVector.fromCOO(right.values.size, List((0, 0.0)))))
    +            val row2 = Option(right).getOrElse(IndexedRow(
    +                    left.rowIndex,
    +                    SparseVector.fromCOO(left.values.size, List((0, 0.0)))))
    +
    +            IndexedRow(row1.rowIndex, fun(row1.values, row2.values))
    +          }
    +      )
    +    new DistributedRowMatrix(result, numRowsOpt, numColsOpt)
    +  }
    +
    +  /**
    +    * Add the matrix to another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def sum(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val sumFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze + y.asBreeze).fromBreeze
    +    this.byRowOperation(sumFunction, other)
    +  }
    +
    +  /**
    +    * Subtracts another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def subtract(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val subFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze - y.asBreeze).fromBreeze
    +    this.byRowOperation(subFunction, other)
    +  }
    +}
    +
    +object DistributedRowMatrix {
    +
    +  type MatrixRowIndex = Int
    +
    +  /**
    +    * Builds a DistributedRowMatrix from a dataset in COO
    +    * @param isSorted If false, sorts the row to properly build the matrix representation.
    +    *                 If already sorted, set this parameter to true to skip sorting.
    +    * @return
    +    */
    +  def fromCOO(data: DataSet[(Int, Int, Double)],
    +              numRows: Int,
    +              numCols: Int,
    +              isSorted: Boolean = false): DistributedRowMatrix = {
    +    val vectorData: DataSet[(Int, 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, Some(numRows), Some(numCols))
    +  }
    +}
    +
    +case class IndexedRow(rowIndex: Int, values: Vector)
    +    extends Ordered[IndexedRow] {
    --- End diff --
    
    Does we need to extend `Ordered` trait? It seems unnecessary but if it is necessary, please explain reason.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65102444
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    Ah. okay. Let's use `Int` for indices now.
    
    About counting, we can re-use `DataSetUtils.countElementsPerPartition` method. (https://github.com/apache/flink/blob/master/flink-scala/src/main/scala/org/apache/flink/api/scala/utils/package.scala#L56)
    
    ```scala
    import org.apache.flink.scala.utils._
    
    ...
    
    lazy val numRowsInDataSet = numRowsOpt match {
      case Some(value) => data.getExecutionEnvironment.fromElements(value)
      case None => data.countElementsPerPartition.map(_._2).reduce(_ + _).map(_.toInt)
    }
    
    ...
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1996#issuecomment-222488228
  
    There are some public methods (`sum`, `subtract`, `byRowOperation`, etc.) without scaladoc. Please add scaladoc for the methods.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65084114
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    Also I remember why we are using `Int` and not `Long`: local vectors and matrices use `Int` for indices and so we would need to modify them or reimplement them because these distributed matrices rely on local implementations for many tasks. So they are not an option right now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65134910
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,182 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    +
    +  val getRowData = data
    +
    +  private def calcCols: Int =
    +    data.first(1).collect().headOption match {
    +      case Some(vector) => vector.values.size
    +      case None => 0
    +    }
    +
    +  /**
    +    * Collects the data in the form of a sequence of coordinates associated with their values.
    +    * @return
    +    */
    +  def toCOO: Seq[(Int, Int, 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
    +    * @return
    +    */
    +  def toLocalSparseMatrix: SparseMatrix = {
    +    val localMatrix =
    +      SparseMatrix.fromCOO(this.getNumRows, this.getNumCols, this.toCOO)
    +    require(localMatrix.numRows == this.getNumRows)
    +    require(localMatrix.numCols == this.getNumCols)
    +    localMatrix
    +  }
    +
    +  //TODO: convert to dense representation on the distributed matrix and collect it afterward
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  /**
    +    * Apply a high-order function to couple of rows
    +    * @param fun
    +    * @param other
    +    * @return
    +    */
    +  def byRowOperation(fun: (Vector, Vector) => Vector,
    +                     other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val otherData = other.getRowData
    +    require(this.getNumCols == other.getNumCols)
    +    require(this.getNumRows == other.getNumRows)
    +
    +
    +    val result = this.data
    +      .fullOuterJoin(otherData)
    +      .where("rowIndex")
    +      .equalTo("rowIndex")(
    +          (left: IndexedRow, right: IndexedRow) => {
    +            val row1 = Option(left).getOrElse(IndexedRow(
    +                    right.rowIndex,
    +                    SparseVector.fromCOO(right.values.size, List((0, 0.0)))))
    +            val row2 = Option(right).getOrElse(IndexedRow(
    +                    left.rowIndex,
    +                    SparseVector.fromCOO(left.values.size, List((0, 0.0)))))
    +
    +            IndexedRow(row1.rowIndex, fun(row1.values, row2.values))
    +          }
    +      )
    +    new DistributedRowMatrix(result, numRowsOpt, numColsOpt)
    +  }
    +
    +  /**
    +    * Add the matrix to another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def sum(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val sumFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze + y.asBreeze).fromBreeze
    +    this.byRowOperation(sumFunction, other)
    +  }
    +
    +  /**
    +    * Subtracts another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def subtract(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val subFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze - y.asBreeze).fromBreeze
    +    this.byRowOperation(subFunction, other)
    +  }
    +}
    +
    +object DistributedRowMatrix {
    +
    +  type MatrixRowIndex = Int
    +
    +  /**
    +    * Builds a DistributedRowMatrix from a dataset in COO
    +    * @param isSorted If false, sorts the row to properly build the matrix representation.
    +    *                 If already sorted, set this parameter to true to skip sorting.
    +    * @return
    +    */
    +  def fromCOO(data: DataSet[(Int, Int, Double)],
    +              numRows: Int,
    +              numCols: Int,
    +              isSorted: Boolean = false): DistributedRowMatrix = {
    +    val vectorData: DataSet[(Int, 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, Some(numRows), Some(numCols))
    +  }
    +}
    +
    +case class IndexedRow(rowIndex: Int, values: Vector)
    +    extends Ordered[IndexedRow] {
    --- End diff --
    
    I thought it was the most elegant and generic way to sort a list of IndexedRow. If it's not the suggested way in Flink's codebase, I can remove it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65061860
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    Actually it was decided to support only Int indices, so moving to Long doesn't make much sense. 
    
    You're right in saying that those informations should be in a DataSet but at the same time they are used to create the Job and verify some requirements. An option would be to leave them as Int and turn the Option[Int] parameter to an Int. This way the user will be forced to input the matrix's dimensions and make the behaviour of the constructor more clear.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-b...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on the pull request:

    https://github.com/apache/flink/pull/1996
  
    @chiwanpark Yeah I thought I could write the documentation as a third PR but I would like to review the block matrix first because it may change in structure. Anyway I will soon begin working on a doc page with the general structure and some examples for the row-based matrix. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-b...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65356053
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,166 @@
    +/*
    + * 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 breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.Breeze._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @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.
    +    * @return
    +    */
    +  def toCOO: Seq[(Int, Int, 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
    +    * @return
    +    */
    +  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
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  /**
    +    * Apply a high-order function to couple of rows
    +    * @param fun
    +    * @param other
    +    * @return
    +    */
    +  def byRowOperation(fun: (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, fun(row1.values, row2.values))
    +          }
    +      )
    +    new DistributedRowMatrix(result, numRows, numCols)
    +  }
    +
    +  /**
    +    * Add the matrix to another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def sum(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val sumFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze + y.asBreeze).fromBreeze
    +    this.byRowOperation(sumFunction, other)
    +  }
    +
    +  /**
    +    * Subtracts another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def subtract(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val subFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze - y.asBreeze).fromBreeze
    +    this.byRowOperation(subFunction, other)
    +  }
    +}
    +
    +object DistributedRowMatrix {
    +
    +  type MatrixRowIndex = Int
    --- End diff --
    
    yup, I tried to stay general but I'm not sure I've been disciplined in using MatrixRowIndex instead of Int so maybe I should remove it and leave to an eventual refactoring the generalization.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-b...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1996
  
    @chobeat Okay. Then we should force user to calculate dimensionality of matrix by changing type of number parameters in constructor.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-b...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65309134
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,182 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    +
    +  val getRowData = data
    +
    +  private def calcCols: Int =
    +    data.first(1).collect().headOption match {
    +      case Some(vector) => vector.values.size
    +      case None => 0
    +    }
    +
    +  /**
    +    * Collects the data in the form of a sequence of coordinates associated with their values.
    +    * @return
    +    */
    +  def toCOO: Seq[(Int, Int, 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
    +    * @return
    +    */
    +  def toLocalSparseMatrix: SparseMatrix = {
    +    val localMatrix =
    +      SparseMatrix.fromCOO(this.getNumRows, this.getNumCols, this.toCOO)
    +    require(localMatrix.numRows == this.getNumRows)
    +    require(localMatrix.numCols == this.getNumCols)
    +    localMatrix
    +  }
    +
    +  //TODO: convert to dense representation on the distributed matrix and collect it afterward
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  /**
    +    * Apply a high-order function to couple of rows
    +    * @param fun
    +    * @param other
    +    * @return
    +    */
    +  def byRowOperation(fun: (Vector, Vector) => Vector,
    +                     other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val otherData = other.getRowData
    +    require(this.getNumCols == other.getNumCols)
    +    require(this.getNumRows == other.getNumRows)
    +
    +
    +    val result = this.data
    +      .fullOuterJoin(otherData)
    +      .where("rowIndex")
    +      .equalTo("rowIndex")(
    +          (left: IndexedRow, right: IndexedRow) => {
    +            val row1 = Option(left).getOrElse(IndexedRow(
    +                    right.rowIndex,
    +                    SparseVector.fromCOO(right.values.size, List((0, 0.0)))))
    +            val row2 = Option(right).getOrElse(IndexedRow(
    +                    left.rowIndex,
    +                    SparseVector.fromCOO(left.values.size, List((0, 0.0)))))
    +
    +            IndexedRow(row1.rowIndex, fun(row1.values, row2.values))
    +          }
    +      )
    +    new DistributedRowMatrix(result, numRowsOpt, numColsOpt)
    +  }
    +
    +  /**
    +    * Add the matrix to another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def sum(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val sumFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze + y.asBreeze).fromBreeze
    +    this.byRowOperation(sumFunction, other)
    +  }
    +
    +  /**
    +    * Subtracts another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def subtract(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val subFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze - y.asBreeze).fromBreeze
    +    this.byRowOperation(subFunction, other)
    +  }
    +}
    +
    +object DistributedRowMatrix {
    +
    +  type MatrixRowIndex = Int
    +
    +  /**
    +    * Builds a DistributedRowMatrix from a dataset in COO
    +    * @param isSorted If false, sorts the row to properly build the matrix representation.
    +    *                 If already sorted, set this parameter to true to skip sorting.
    +    * @return
    +    */
    +  def fromCOO(data: DataSet[(Int, Int, Double)],
    +              numRows: Int,
    +              numCols: Int,
    +              isSorted: Boolean = false): DistributedRowMatrix = {
    +    val vectorData: DataSet[(Int, 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, Some(numRows), Some(numCols))
    +  }
    +}
    +
    +case class IndexedRow(rowIndex: Int, values: Vector)
    +    extends Ordered[IndexedRow] {
    --- End diff --
    
    Ah, okay. Let's leave it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-b...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1996
  
    Hi @chobeat, thanks for update PR. After addressing comments on source code, I think the last thing to merge this is adding documentation for this. But you can add the documentation after block-based matrix is merged.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-b...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65318407
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,179 @@
    +/*
    + * 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 breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.Breeze._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    --- End diff --
    
    Yup, I forgot to remove that. It was supposed to be part of the common trait but in the end I chose not to do it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #1996: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-ba...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the issue:

    https://github.com/apache/flink/pull/1996
  
    Looks good to me, +1. I'll merge this in few hours.
    
    But I think we should change the type of row index to `Long` in near future. I think we can deal with the incompatibility problem with local matrices by assuming that the number of rows is less than `Int.MaxValue` and converting row index type from `Long` to `Int` in collecting time. I'll submit an issue related to this to JIRA after merge this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65145402
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    Yes, you are right. Thanks for correction.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65065451
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    +
    +  val getRowData = data
    +
    +  private def calcCols: Int =
    +    data.first(1).collect().headOption match {
    +      case Some(vector) => vector.values.size
    +      case None => 0
    +    }
    +
    +  /**
    +    * Collects the data in the form of a sequence of coordinates associated with their values.
    +    * @return
    +    */
    +  def toCOO: Seq[(Int, Int, 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
    +    * @return
    +    */
    +  def toLocalSparseMatrix: SparseMatrix = {
    +    val localMatrix = SparseMatrix.fromCOO(this.getNumRows, this.getNumCols, this.toCOO)
    +    require(localMatrix.numRows == this.getNumRows)
    +    require(localMatrix.numCols == this.getNumCols)
    +    localMatrix
    +  }
    +
    +  //TODO: convert to dense representation on the distributed matrix and collect it afterward
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  def byRowOperation(
    +      fun: (Vector, Vector) => Vector, other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val otherData = other.getRowData
    +    require(this.getNumCols == other.getNumCols)
    +    require(this.getNumRows == other.getNumRows)
    +
    +    val ev1: TypeInformation[Int] = TypeInformation.of(classOf[Int])
    +
    +    val result = this.data
    +      .fullOuterJoin(otherData)
    +      .where(_.rowIndex)
    +      .equalTo(_.rowIndex)(ev1)(
    --- End diff --
    
    Good to know. I will have to fix this in the other PR too because I used it a lot. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-b...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65355487
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,166 @@
    +/*
    + * 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 breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.Breeze._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @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.
    +    * @return
    +    */
    +  def toCOO: Seq[(Int, Int, 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
    +    * @return
    +    */
    +  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
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  /**
    +    * Apply a high-order function to couple of rows
    +    * @param fun
    +    * @param other
    +    * @return
    +    */
    +  def byRowOperation(fun: (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, fun(row1.values, row2.values))
    +          }
    +      )
    +    new DistributedRowMatrix(result, numRows, numCols)
    +  }
    +
    +  /**
    +    * Add the matrix to another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def sum(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val sumFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze + y.asBreeze).fromBreeze
    +    this.byRowOperation(sumFunction, other)
    +  }
    +
    +  /**
    +    * Subtracts another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def subtract(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val subFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze - y.asBreeze).fromBreeze
    +    this.byRowOperation(subFunction, other)
    +  }
    +}
    +
    +object DistributedRowMatrix {
    +
    +  type MatrixRowIndex = Int
    --- End diff --
    
    Is this for changing type of matrix row index in future? But there is no usage in other parts.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65067628
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    I meant the type of **row** number. Does we use row number as indices?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65068513
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    Yup, they are the same thing in this format. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65060611
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    --- End diff --
    
    please scaladoc here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #1996: [FLINK-3919][flink-ml] Distributed Linear Algebra:...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65671113
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,166 @@
    +/*
    + * 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 breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.Breeze._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @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.
    +    * @return
    +    */
    +  def toCOO: Seq[(Int, Int, 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
    +    * @return
    +    */
    +  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
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  /**
    +    * Apply a high-order function to couple of rows
    +    * @param fun
    +    * @param other
    +    * @return
    +    */
    +  def byRowOperation(fun: (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, fun(row1.values, row2.values))
    +          }
    +      )
    +    new DistributedRowMatrix(result, numRows, numCols)
    +  }
    +
    +  /**
    +    * Add the matrix to another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def sum(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val sumFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze + y.asBreeze).fromBreeze
    +    this.byRowOperation(sumFunction, other)
    +  }
    +
    +  /**
    +    * Subtracts another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def subtract(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val subFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze - y.asBreeze).fromBreeze
    +    this.byRowOperation(subFunction, other)
    +  }
    +}
    +
    +object DistributedRowMatrix {
    +
    +  type MatrixRowIndex = Int
    --- End diff --
    
    I did the same for columns and I moved the definition to the `DistributedMatrix` trait.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65060527
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedMatrix.scala ---
    @@ -0,0 +1,25 @@
    +/*
    + * 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
    +
    +trait DistributedMatrix {
    +
    +  def getNumRows: Int
    +  def getNumCols: Int
    --- End diff --
    
    How about changing the return type of `getNumRows` and `getNumCols` to `DataSet[Long]`? Returning number of elements directly in distributed data in Flink is expensive operation. We can use broadcast variable to transfer the numbers to future operation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-b...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65311449
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,179 @@
    +/*
    + * 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 breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.Breeze._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    --- End diff --
    
    It seems that `getRowData` is redundant. Changing `data` to public (by adding `val` keyword in previous `data`) would be better.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65143303
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    Ok, fixed and added some tests for dimensions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-b...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on the pull request:

    https://github.com/apache/flink/pull/1996
  
    @chiwanpark before merging I need to go over the numRows/numCols issue again because I noticed they create problems in another project of mine. I think that the `collect()` there is too risky and obscure so I would like to refactor it. I'm working on it right now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65072614
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    I'm working on it but I have a doubt: for example if the user provides a value for the number of rows, `getRowNumber:DataSet[Long]` (or `Int`) how is supposed to provide the number? It feels unnatural to parallelize a value that is already available locally. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65142014
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    I've found a flaw in this way of thought. It was already in my code but we are keeping doing the same error: the matrix representation is sparse so counting the actual rows is wrong. We should count the max index provided. If no info is given about the matrix dimensionality, we assume that the row with the highest indices is the last at the bottom of the matrix. Counting them would give a totally wrong result.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65064313
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    Ok, I'm not sure how it would look in terms of usability but I will try. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65063499
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    I still prefer numbers in `DataSet` but providing the numbers in both methods (in `DataSet[Int]` and `Int`) would be okay. Sometimes users don't know the size of matrix (such as generating the matrix from other data).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on the pull request:

    https://github.com/apache/flink/pull/1996#issuecomment-222473905
  
    Hi @chobeat, thanks for opening pull request. I would like to shepherd this PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65064961
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    +
    +  val getRowData = data
    +
    +  private def calcCols: Int =
    +    data.first(1).collect().headOption match {
    +      case Some(vector) => vector.values.size
    +      case None => 0
    +    }
    +
    +  /**
    +    * Collects the data in the form of a sequence of coordinates associated with their values.
    +    * @return
    +    */
    +  def toCOO: Seq[(Int, Int, 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
    +    * @return
    +    */
    +  def toLocalSparseMatrix: SparseMatrix = {
    +    val localMatrix = SparseMatrix.fromCOO(this.getNumRows, this.getNumCols, this.toCOO)
    +    require(localMatrix.numRows == this.getNumRows)
    +    require(localMatrix.numCols == this.getNumCols)
    +    localMatrix
    +  }
    +
    +  //TODO: convert to dense representation on the distributed matrix and collect it afterward
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  def byRowOperation(
    +      fun: (Vector, Vector) => Vector, other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val otherData = other.getRowData
    +    require(this.getNumCols == other.getNumCols)
    +    require(this.getNumRows == other.getNumRows)
    +
    +    val ev1: TypeInformation[Int] = TypeInformation.of(classOf[Int])
    +
    +    val result = this.data
    +      .fullOuterJoin(otherData)
    +      .where(_.rowIndex)
    +      .equalTo(_.rowIndex)(ev1)(
    --- End diff --
    
    Key selector function creates copying keys overhead. please change this to like following:
    
    ```scala
    val result = this.data
      .fullOuterJoin(otherData)
      .where("rowIndex")
      .equalTo("rowIndex") {
        (left: IndexedRow, right: IndexedRow) => ...
      }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #1996: [FLINK-3919][flink-ml] Distributed Linear Algebra:...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/1996


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Algebra: row-b...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65309186
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,179 @@
    +/*
    + * 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 breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.Breeze._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => numRows.collect().head
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => numCols.collect().head
    +  }
    +
    +  lazy val numRows: DataSet[Int] = numRowsOpt match {
    +    case Some(rows) => data.getExecutionEnvironment.fromElements(rows)
    +    case None => data.max("rowIndex").map(_.rowIndex + 1)
    +  }
    +
    +  lazy val numCols: DataSet[Int] = numColsOpt match {
    +    case Some(cols) => data.getExecutionEnvironment.fromElements(cols)
    +    case None => data.first(1).map(_.values.size)
    +  }
    +
    +  val getRowData = data
    +
    +  /**
    +    * Collects the data in the form of a sequence of coordinates associated with their values.
    +    * @return
    +    */
    +  def toCOO: Seq[(Int, Int, 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
    +    * @return
    +    */
    +  def toLocalSparseMatrix: SparseMatrix = {
    +    val localMatrix =
    +      SparseMatrix.fromCOO(this.getNumRows, this.getNumCols, this.toCOO)
    +    require(localMatrix.numRows == this.getNumRows)
    +    require(localMatrix.numCols == this.getNumCols)
    +    localMatrix
    +  }
    +
    +  //TODO: convert to dense representation on the distributed matrix and collect it afterward
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  /**
    +    * Apply a high-order function to couple of rows
    +    * @param fun
    +    * @param other
    +    * @return
    +    */
    +  def byRowOperation(fun: (Vector, Vector) => Vector,
    +                     other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val otherData = other.getRowData
    +    require(this.getNumCols == other.getNumCols)
    +    require(this.getNumRows == other.getNumRows)
    +
    +    val result = this.data
    +      .fullOuterJoin(otherData)
    +      .where("rowIndex")
    +      .equalTo("rowIndex")(
    +          (left: IndexedRow, right: IndexedRow) => {
    +            val row1 = Option(left).getOrElse(IndexedRow(
    +                    right.rowIndex,
    +                    SparseVector.fromCOO(right.values.size, List((0, 0.0)))))
    +            val row2 = Option(right).getOrElse(IndexedRow(
    +                    left.rowIndex,
    +                    SparseVector.fromCOO(left.values.size, List((0, 0.0)))))
    +
    +            IndexedRow(row1.rowIndex, fun(row1.values, row2.values))
    +          }
    +      )
    +    new DistributedRowMatrix(result, numRowsOpt, numColsOpt)
    +  }
    +
    +  /**
    +    * Add the matrix to another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def sum(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val sumFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze + y.asBreeze).fromBreeze
    +    this.byRowOperation(sumFunction, other)
    +  }
    +
    +  /**
    +    * Subtracts another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def subtract(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val subFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze - y.asBreeze).fromBreeze
    +    this.byRowOperation(subFunction, other)
    +  }
    +}
    +
    +object DistributedRowMatrix {
    +
    +  type MatrixRowIndex = Int
    +
    +  /**
    +    * Builds a DistributedRowMatrix from a dataset in COO
    +    * @param isSorted If false, sorts the row to properly build the matrix representation.
    +    *                 If already sorted, set this parameter to true to skip sorting.
    +    * @return
    +    */
    +  def fromCOO(data: DataSet[(Int, Int, Double)],
    +              numRows: Int,
    +              numCols: Int,
    +              isSorted: Boolean = false): DistributedRowMatrix = {
    +    val vectorData: DataSet[(Int, 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, Some(numRows), Some(numCols))
    +  }
    +}
    +
    +case class IndexedRow(rowIndex: Int, values: Vector)
    +    extends Ordered[IndexedRow] {
    +
    +  def compare(other: IndexedRow) = this.rowIndex.compare(other.rowIndex)
    +
    +  override def toString: String = s"($rowIndex,${values.toString}"
    --- End diff --
    
    Missing a closing parentheses


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65060576
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedMatrix.scala ---
    @@ -0,0 +1,25 @@
    +/*
    + * 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
    +
    +trait DistributedMatrix {
    --- End diff --
    
    We need scaladoc here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #1996: [FLINK-3919][flink-ml] Distributed Linear Algebra:...

Posted by chiwanpark <gi...@git.apache.org>.
Github user chiwanpark commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65657170
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,166 @@
    +/*
    + * 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 breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.Breeze._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +
    +/**
    +  * Distributed row-major matrix representation.
    +  * @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.
    +    * @return
    +    */
    +  def toCOO: Seq[(Int, Int, 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
    +    * @return
    +    */
    +  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
    +  def toLocalDenseMatrix: DenseMatrix = this.toLocalSparseMatrix.toDenseMatrix
    +
    +  /**
    +    * Apply a high-order function to couple of rows
    +    * @param fun
    +    * @param other
    +    * @return
    +    */
    +  def byRowOperation(fun: (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, fun(row1.values, row2.values))
    +          }
    +      )
    +    new DistributedRowMatrix(result, numRows, numCols)
    +  }
    +
    +  /**
    +    * Add the matrix to another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def sum(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val sumFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze + y.asBreeze).fromBreeze
    +    this.byRowOperation(sumFunction, other)
    +  }
    +
    +  /**
    +    * Subtracts another matrix.
    +    * @param other
    +    * @return
    +    */
    +  def subtract(other: DistributedRowMatrix): DistributedRowMatrix = {
    +    val subFunction: (Vector, Vector) => Vector = (x: Vector, y: Vector) =>
    +      (x.asBreeze - y.asBreeze).fromBreeze
    +    this.byRowOperation(subFunction, other)
    +  }
    +}
    +
    +object DistributedRowMatrix {
    +
    +  type MatrixRowIndex = Int
    --- End diff --
    
    It would be better to replace `Int` used for row index to `MatrixRowIndex`. It makes changing type of row index in future easier.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-3919][flink-ml] Distributed Linear Alge...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1996#discussion_r65081970
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/math/distributed/DistributedRowMatrix.scala ---
    @@ -0,0 +1,167 @@
    +/*
    + * 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 java.lang
    +
    +import breeze.linalg.{CSCMatrix => BreezeSparseMatrix, Matrix => BreezeMatrix, Vector => BreezeVector}
    +import org.apache.flink.api.common.functions.RichGroupReduceFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala._
    +import org.apache.flink.ml.math.{Matrix => FlinkMatrix, _}
    +import org.apache.flink.util.Collector
    +import org.apache.flink.ml.math.Breeze._
    +import scala.collection.JavaConversions._
    +
    +/**
    +  *
    +  * @param numRowsOpt If None, will be calculated from the DataSet.
    +  * @param numColsOpt If None, will be calculated from the DataSet.
    +  */
    +class DistributedRowMatrix(data: DataSet[IndexedRow],
    +                           numRowsOpt: Option[Int] = None,
    +                           numColsOpt: Option[Int] = None)
    +    extends DistributedMatrix {
    +
    +  lazy val getNumRows: Int = numRowsOpt match {
    +    case Some(rows) => rows
    +    case None => data.count().toInt
    +  }
    +
    +  lazy val getNumCols: Int = numColsOpt match {
    +    case Some(cols) => cols
    +    case None => calcCols
    +  }
    --- End diff --
    
    Is there a way to return the result of an accumulator lazily? I'm trying to use one to implement the row count but I don't know accumulators well enough. If it's not good, should I just reduce the rows summing 1 for each element?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---