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

[GitHub] flink pull request: [FLINK-1979] Add logistic loss, hinge loss and...

GitHub user skavulya opened a pull request:

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

    [FLINK-1979] Add logistic loss, hinge loss and regularization penalties 

    Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
    If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
    In addition to going through the list, please provide a meaningful description of your changes.
    
    - [ ] General
      - The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
      - The pull request addresses only one issue
      - Each commit in the PR has a meaningful commit message (including the JIRA id)
    
    - [ ] Documentation
      - Documentation has been added for new functionality
      - Old documentation affected by the pull request has been updated
      - JavaDoc for public methods has been added
    
    - [ ] Tests & Build
      - Functionality added by the pull request is covered by tests
      - `mvn clean verify` has been executed successfully locally or a Travis build has passed
    
    \u2026zation

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

    $ git pull https://github.com/skavulya/flink loss-functions2

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

    https://github.com/apache/flink/pull/1985.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 #1985
    
----
commit 014108ac9ea2e46ac021e9bf3824624d54357f74
Author: spkavuly <so...@intel.com>
Date:   2016-05-11T22:55:13Z

    Add logistic loss, hinge loss and regularization penalties for optimization

----


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63509413
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/GradientDescent.scala ---
    @@ -287,11 +287,8 @@ class GradientDescentL2 extends GradientDescent {
           regularizationConstant: Double,
           learningRate: Double)
         : Vector = {
    -    // add the gradient of the L2 regularization
    -    BLAS.axpy(regularizationConstant, weightVector, gradient)
     
    -    // update the weights according to the learning rate
    -    BLAS.axpy(-learningRate, gradient, weightVector)
    +    L2Regularization.takeStep(weightVector, gradient, regularizationConstant,learningRate)
    --- End diff --
    
    whitespace missing between comma and `learningRate`


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63562423
  
    --- Diff: flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/optimization/RegularizationPenaltyITSuite.scala ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.optimization
    +
    +import org.apache.flink.ml.math.DenseVector
    +import org.apache.flink.test.util.FlinkTestBase
    +import org.scalatest.{FlatSpec, Matchers}
    +
    +
    +class RegularizationPenaltyITSuite extends FlatSpec with Matchers with FlinkTestBase {
    --- End diff --
    
    Thanks @tillrohrmann  @thvasilo for all your comments. I'll make the changes for passing the regression penalty as a parameter, fix the formatting issue and update the unit tests


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63657433
  
    --- Diff: flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/optimization/RegularizationPenaltyITSuite.scala ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.optimization
    +
    +import org.apache.flink.ml.math.DenseVector
    +import org.apache.flink.test.util.FlinkTestBase
    +import org.scalatest.{FlatSpec, Matchers}
    +
    +
    +class RegularizationPenaltyITSuite extends FlatSpec with Matchers with FlinkTestBase {
    --- End diff --
    
    You're right @thvasilo. This would be not an integration test because it tests the regularization component without firing up a Flink cluster and executing an actual job. So we could remove the `ITSuite` from the name.


---
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-1979] Add logistic loss, hinge loss and regulariz...

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

    https://github.com/apache/flink/pull/1985#discussion_r65291552
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/RegularizationPenalty.scala ---
    @@ -0,0 +1,215 @@
    +/*
    + * 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.optimization
    +
    +import org.apache.flink.ml.math.{Vector, BLAS}
    +import org.apache.flink.ml.math.Breeze._
    +import breeze.linalg.{norm => BreezeNorm}
    +
    +/** Represents a type of regularization penalty
    +  *
    +  * Regularization penalties are used to restrict the optimization problem to solutions with
    +  * certain desirable characteristics, such as sparsity for the L1 penalty, or penalizing large
    +  * weights for the L2 penalty.
    +  *
    +  * The regularization term, `R(w)` is added to the objective function, `f(w) = L(w) + lambda*R(w)`
    +  * where lambda is the regularization parameter used to tune the amount of regularization applied.
    +  */
    +trait RegularizationPenalty extends Serializable {
    +
    +  /** Calculates the new weights based on the gradient and regularization penalty
    +    *
    +    * @param weightVector The weights to be updated
    +    * @param gradient The gradient used to update the weights
    +    * @param regularizationConstant The regularization parameter to be applied 
    +    * @param learningRate The effective step size for this iteration
    +    * @return Updated weights
    +    */
    +  def takeStep(
    +      weightVector: Vector,
    +      gradient: Vector,
    +      regularizationConstant: Double,
    +      learningRate: Double)
    +    : Vector
    +
    +  /** Adds regularization to the loss value
    +    *
    +    * @param oldLoss The loss to be updated
    +    * @param weightVector The gradient used to update the loss
    +    * @param regularizationConstant The regularization parameter to be applied
    +    * @return Updated loss
    +    */
    +  def regLoss(oldLoss: Double, weightVector: Vector, regularizationConstant: Double): Double
    +
    +}
    +
    +
    +/** `L_2` regularization penalty.
    +  *
    +  * The regularization function is the square of the L2 norm `1/2*||w||_2^2`
    +  * with `w` being the weight vector. The function penalizes large weights,
    +  * favoring solutions with more small weights rather than few large ones.
    +  */
    +object L2Regularization extends RegularizationPenalty {
    +
    +  /** Calculates the new weights based on the gradient and L2 regularization penalty
    +    *
    +    * The updated weight is `w - learningRate *(gradient + lambda * w)` where
    +    * `w` is the weight vector, and `lambda` is the regularization parameter.
    +    *
    +    * @param weightVector The weights to be updated
    +    * @param gradient The gradient according to which we will update the weights
    +    * @param regularizationConstant The regularization parameter to be applied
    +    * @param learningRate The effective step size for this iteration
    +    * @return Updated weights
    +    */
    +  override def takeStep(
    +      weightVector: Vector,
    +      gradient: Vector,
    +      regularizationConstant: Double,
    +      learningRate: Double)
    +    : Vector = {
    +    // add the gradient of the L2 regularization
    +    BLAS.axpy(regularizationConstant, weightVector, gradient)
    +
    +    // update the weights according to the learning rate
    +    BLAS.axpy(-learningRate, gradient, weightVector)
    +
    +    weightVector
    +  }
    +
    +  /** Adds regularization to the loss value
    +    *
    +    * The updated loss is `oldLoss + lambda * 1/2*||w||_2^2` where
    +    * `w` is the weight vector, and `lambda` is the regularization parameter
    +    *
    +    * @param oldLoss The loss to be updated
    +    * @param weightVector The gradient used to update the loss
    +    * @param regularizationConstant The regularization parameter to be applied
    +    * @return Updated loss
    +    */
    +  override def regLoss(oldLoss: Double, weightVector: Vector, regularizationConstant: Double)
    +    : Double = {
    +    val squareNorm = BLAS.dot(weightVector, weightVector)
    +    oldLoss + regularizationConstant * 0.5 * squareNorm
    +  }
    +}
    +
    +/** `L_1` regularization penalty.
    +  *
    +  * The regularization function is the `L1` norm `||w||_1` with `w` being the weight vector.
    +  * The `L_1` penalty can be used to drive a number of the solution coefficients to 0, thereby
    +  * producing sparse solutions.
    +  *
    +  */
    +object L1Regularization extends RegularizationPenalty {
    +
    +  /** Calculates the new weights based on the gradient and regularization penalty
    +    *
    +    * The updated weight `w - learningRate * gradient` is shrunk towards zero
    +    * by applying the proximal operator `signum(w) * max(0.0, abs(w) - shrinkageVal)`
    +    * where `w` is the weight vector, `lambda` is the regularization parameter,
    +    * and `shrinkageVal` is `lambda*learningRate`.
    +    *
    +    * @param weightVector The weights to be updated
    +    * @param gradient The gradient according to which we will update the weights
    +    * @param regularizationConstant The regularization parameter to be applied
    +    * @param learningRate The effective step size for this iteration
    +    * @return Updated weights
    +    */
    +  override def takeStep(
    +      weightVector: Vector,
    +      gradient: Vector,
    +      regularizationConstant: Double,
    +      learningRate: Double)
    +    : Vector = {
    +    // Update weight vector with gradient.
    +    BLAS.axpy(-learningRate, gradient, weightVector)
    +
    +    // Apply proximal operator (soft thresholding)
    +    val shrinkageVal = regularizationConstant * learningRate
    +    var i = 0
    +    while (i < weightVector.size) {
    +      val wi = weightVector(i)
    +      weightVector(i) = scala.math.signum(wi) *
    +        scala.math.max(0.0, scala.math.abs(wi) - shrinkageVal)
    --- End diff --
    
    We can change `scala.math` to `math`.


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63561773
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/GradientDescent.scala ---
    @@ -272,7 +272,7 @@ abstract class GradientDescent extends IterativeSolver {
       * The regularization function is `1/2 ||w||_2^2` with `w` being the weight vector.
       */
     class GradientDescentL2 extends GradientDescent {
    -
    +  //TODO(skavulya): Pass regularization penalty as a parameter
    --- End diff --
    
    I can add it to this PR since it is only affecting SGD.


---
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-1979] Add logistic loss, hinge loss and regulariz...

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

    https://github.com/apache/flink/pull/1985
  
    @tillrohrmann @thvasilo I made the changes you recommended. Please let me know if they look 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 issue #1985: [FLINK-1979] Add logistic loss, hinge loss and regulariza...

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

    https://github.com/apache/flink/pull/1985
  
    While working on L-BFGS, I realized that I need to remove the gradient descent step from RegularizationPenalty. I'll update the PR soon.


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63509544
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/GradientDescent.scala ---
    @@ -364,8 +350,7 @@ class SimpleGradientDescent extends GradientDescent {
           learningRate: Double)
         : Vector = {
         // Update the weight vector
    -    BLAS.axpy(-learningRate, gradient, weightVector)
    -    weightVector
    +    NoRegularization.takeStep(weightVector, gradient, regularizationConstant,learningRate)
    --- End diff --
    
    whitespace


---
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 #1985: [FLINK-1979] Add logistic loss, hinge loss and regulariza...

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

    https://github.com/apache/flink/pull/1985
  
    Thanks @tillrohrmann! 


---
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-1979] Add logistic loss, hinge loss and regulariz...

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

    https://github.com/apache/flink/pull/1985#discussion_r65291353
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/PartialLossFunction.scala ---
    @@ -47,21 +47,106 @@ object SquaredLoss extends PartialLossFunction {
     
       /** Calculates the loss depending on the label and the prediction
         *
    -    * @param prediction
    -    * @param label
    -    * @return
    +    * @param prediction The predicted value
    +    * @param label The true value
    +    * @return The loss
         */
       override def loss(prediction: Double, label: Double): Double = {
         0.5 * (prediction - label) * (prediction - label)
       }
     
       /** Calculates the derivative of the [[PartialLossFunction]]
         *
    -    * @param prediction
    -    * @param label
    -    * @return
    +    * @param prediction The predicted value
    +    * @param label The true value
    +    * @return The derivative of the loss function
         */
       override def derivative(prediction: Double, label: Double): Double = {
         (prediction - label)
       }
     }
    +
    +/** Logistic loss function which can be used with the [[GenericLossFunction]]
    +  *
    +  *
    +  * The [[LogisticLoss]] function implements `log(1 + -exp(prediction*label))`
    +  * for binary classification with label in {-1, 1}
    +  */
    +object LogisticLoss extends PartialLossFunction {
    +
    +  /** Calculates the loss depending on the label and the prediction
    +    *
    +    * @param prediction The predicted value
    +    * @param label The true value
    +    * @return The loss
    +    */
    +  override def loss(prediction: Double, label: Double): Double = {
    +    val z = prediction * label
    +
    +    // based on implementation in scikit-learn
    +    // approximately equal and saves the computation of the log
    +    if (z > 18) {
    +      return math.exp(-z)
    +    }
    +    else if (z < -18) {
    +      return -z
    +    }
    +
    +    math.log(1 + math.exp(-z))
    --- End diff --
    
    Using `return` is not recommended in Scala. Could you change this like following?
    
    ```scala
    if (z > 18) {
      math.exp(-z)
    } else if (z < -18) {
      -z
    } else {
      math.log(1 + math.exp(-z))
    }
    ```


---
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 #1985: [FLINK-1979] Add logistic loss, hinge loss and reg...

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

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


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63519957
  
    --- Diff: flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/optimization/RegularizationPenaltyITSuite.scala ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.optimization
    +
    +import org.apache.flink.ml.math.DenseVector
    +import org.apache.flink.test.util.FlinkTestBase
    +import org.scalatest.{FlatSpec, Matchers}
    +
    +
    +class RegularizationPenaltyITSuite extends FlatSpec with Matchers with FlinkTestBase {
    --- End diff --
    
    @tillrohrmann Do tests like these that only test components without an `ExecutionEnvironment` being created get named differently (i.e. no ITSuite in the class name)


---
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-1979] Add logistic loss, hinge loss and regulariz...

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

    https://github.com/apache/flink/pull/1985
  
    Hi @skavulya, I just quickly reviewed your updated PR and left few comments. They are not critical things but It would be better to fix them. Other things are very good.


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63517587
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/GradientDescent.scala ---
    @@ -272,7 +272,7 @@ abstract class GradientDescent extends IterativeSolver {
       * The regularization function is `1/2 ||w||_2^2` with `w` being the weight vector.
       */
     class GradientDescentL2 extends GradientDescent {
    -
    +  //TODO(skavulya): Pass regularization penalty as a parameter
    --- End diff --
    
    I've mentioned this in the previous PR but adding here for completeness:
    
    I'm in favor of adding the regularization penalty as a parameter for the optimizer.
    
    However that would involve changes that perhaps beyond the scope of this PR, 
    currently with only SGD available we don't have to worry about the applicability of L1/L2 regularization, but should add a note for when L-BFGS get implemented. 
    
    Depending on how much work @skavulya it would be to make the change here, we can choose to have a separate PR for that or include it 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 issue #1985: [FLINK-1979] Add logistic loss, hinge loss and regulariza...

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

    https://github.com/apache/flink/pull/1985
  
    @chiwanpark Decoupling the gradient descent step is complicated for L1 regularization because we are using the proximal gradient method that applies soft thresholding after executing the gradient descent step. I left the regularization penalty as-is. I am thinking of adding an additional method that adds the regularization penalty to gradient without the gradient descent step but I will do it in the L-BFGS PR instead.


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63509514
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/GradientDescent.scala ---
    @@ -321,19 +318,8 @@ class GradientDescentL1 extends GradientDescent {
           regularizationConstant: Double,
           learningRate: Double)
         : Vector = {
    -    // Update weight vector with gradient. L1 regularization has no gradient, the proximal operator
    -    // does the job.
    -    BLAS.axpy(-learningRate, gradient, weightVector)
    -
    -    // Apply proximal operator (soft thresholding)
    -    val shrinkageVal = regularizationConstant * learningRate
    -    var i = 0
    -    while (i < weightVector.size) {
    -      val wi = weightVector(i)
    -      weightVector(i) = scala.math.signum(wi) *
    -        scala.math.max(0.0, scala.math.abs(wi) - shrinkageVal)
    -      i += 1
    -    }
    +
    +    L1Regularization.takeStep(weightVector, gradient, regularizationConstant,learningRate)
    --- End diff --
    
    whitespace missing


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63517610
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/GradientDescent.scala ---
    @@ -272,7 +272,7 @@ abstract class GradientDescent extends IterativeSolver {
       * The regularization function is `1/2 ||w||_2^2` with `w` being the weight vector.
       */
     class GradientDescentL2 extends GradientDescent {
    -
    +  //TODO(skavulya): Pass regularization penalty as a parameter
    --- End diff --
    
    I've mentioned this in the previous PR but adding here for completeness:
    
    I'm in favor of adding the regularization penalty as a parameter for the optimizer.
    
    However that would involve changes that perhaps beyond the scope of this PR, 
    currently with only SGD available we don't have to worry about the applicability of L1/L2 regularization, but should add a note for when L-BFGS get implemented. 
    
    Depending on how much work @skavulya it would be to make the change here, we can choose to have a separate PR for that or include it 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 issue #1985: [FLINK-1979] Add logistic loss, hinge loss and regulariza...

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

    https://github.com/apache/flink/pull/1985
  
    Okay, please ping me when the PR is updated.


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63509381
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/GradientDescent.scala ---
    @@ -272,7 +272,7 @@ abstract class GradientDescent extends IterativeSolver {
       * The regularization function is `1/2 ||w||_2^2` with `w` being the weight vector.
       */
     class GradientDescentL2 extends GradientDescent {
    -
    +  //TODO(skavulya): Pass regularization penalty as a parameter
    --- End diff --
    
    Is this TODO still valid? If so, can we resolve 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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63519390
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/RegularizationPenalty.scala ---
    @@ -0,0 +1,215 @@
    +/*
    + * 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.optimization
    +
    +import org.apache.flink.ml.math.{Vector, BLAS}
    +import org.apache.flink.ml.math.Breeze._
    +import breeze.linalg.{norm => BreezeNorm}
    +
    +/** Represents a type of regularization penalty
    +  *
    +  * Regularization penalties are used to restrict the optimization problem to solutions with
    +  * certain desirable characteristics, such as sparsity for the L1 penalty, or penalizing large
    +  * weights for the L2 penalty.
    +  *
    +  * The regularization term, `R(w)` is added to the objective function, `f(w) = L(w) + lambda*R(w)`
    +  * where lambda is the regularization parameter used to tune the amount of regularization applied.
    +  */
    +trait RegularizationPenalty extends Serializable {
    +
    +  /** Calculates the new weights based on the gradient and regularization penalty
    +    *
    +    * @param weightVector The weights to be updated
    +    * @param gradient The gradient used to update the weights
    +    * @param regularizationConstant The regularization parameter to be applied 
    +    * @param learningRate The effective step size for this iteration
    +    * @return Updated weights
    +    */
    +  def takeStep(
    +      weightVector: Vector,
    +      gradient: Vector,
    +      regularizationConstant: Double,
    +      learningRate: Double)
    +    : Vector
    +
    +  /** Adds regularization to the loss value
    +    *
    +    * @param oldLoss The loss to be updated
    +    * @param weightVector The gradient used to update the loss
    +    * @param regularizationConstant The regularization parameter to be applied
    +    * @return Updated loss
    +    */
    +  def regLoss(oldLoss: Double, weightVector: Vector, regularizationConstant: Double): Double
    +
    +}
    +
    +
    +/** `L_2` regularization penalty.
    +  *
    +  * The regularization function is the square of the L2 norm `1/2*||w||_2^2`
    +  * with `w` being the weight vector. The function penalizes large weights,
    +  * favoring solutions with more small weights rather than few large ones.
    +  */
    +object L2Regularization extends RegularizationPenalty {
    +
    +  /** Calculates the new weights based on the gradient and L2 regularization penalty
    +    *
    +    * The updated weight is `w - learningRate *(gradient + lambda * w)` where
    +    * `w` is the weight vector, and `lambda` is the regularization parameter.
    +    *
    +    * @param weightVector The weights to be updated
    +    * @param gradient The gradient according to which we will update the weights
    +    * @param regularizationConstant The regularization parameter to be applied
    +    * @param learningRate The effective step size for this iteration
    +    * @return Updated weights
    +    */
    +  override def takeStep(
    +      weightVector: Vector,
    +      gradient: Vector,
    +      regularizationConstant: Double,
    +      learningRate: Double)
    +    : Vector = {
    +    // add the gradient of the L2 regularization
    +    BLAS.axpy(regularizationConstant, weightVector, gradient)
    +
    +    // update the weights according to the learning rate
    +    BLAS.axpy(-learningRate, gradient, weightVector)
    +
    +    weightVector
    +  }
    +
    +  /** Adds regularization to the loss value
    +    *
    +    * The updated loss is `l + lambda * 1/2*||w||_2^2` where `l` is the old loss,
    --- End diff --
    
    I would recommend spelling out `loss` here, as in some fonts it's hard tell the letter "l" apart from the number 1,


---
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 #1985: [FLINK-1979] Add logistic loss, hinge loss and regulariza...

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

    https://github.com/apache/flink/pull/1985
  
    @chiwanpark The PR is ready. Let me know if I need to do anything else. 


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63510204
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/PartialLossFunction.scala ---
    @@ -47,21 +47,106 @@ object SquaredLoss extends PartialLossFunction {
     
       /** Calculates the loss depending on the label and the prediction
         *
    -    * @param prediction
    -    * @param label
    -    * @return
    +    * @param prediction The predicted value
    +    * @param label The true value
    +    * @return The loss
         */
       override def loss(prediction: Double, label: Double): Double = {
         0.5 * (prediction - label) * (prediction - label)
       }
     
       /** Calculates the derivative of the [[PartialLossFunction]]
         *
    -    * @param prediction
    -    * @param label
    -    * @return
    +    * @param prediction The predicted value
    +    * @param label The true value
    +    * @return The derivative of the loss function
         */
       override def derivative(prediction: Double, label: Double): Double = {
         (prediction - label)
       }
     }
    +
    +/** Logistic loss function which can be used with the [[GenericLossFunction]]
    +  *
    +  *
    +  * The [[LogisticLoss]] function implements `log(1 + exp(prediction*label))`
    --- End diff --
    
    A minus is missing in the exponent of e.


---
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 #1985: [FLINK-1979] Add logistic loss, hinge loss and regulariza...

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

    https://github.com/apache/flink/pull/1985
  
    Hi @skavulya, sorry for late response. I've checked the updated PR and looks good to me. I wonder whether the name `RegularizationPenalty` is proper because the class calculates lots of values. But it is trivial thing.
    
    If there is no objection in few days, I'll merge this. Thanks! \U0001f44d 


---
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-1979] Add logistic loss, hinge loss and regulariz...

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

    https://github.com/apache/flink/pull/1985#discussion_r65291911
  
    --- Diff: flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/optimization/LossFunctionTest.scala ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.optimization
    +
    +import org.apache.flink.ml.common.{LabeledVector, WeightVector}
    +import org.apache.flink.ml.math.DenseVector
    +import org.scalatest.{Matchers, FlatSpec}
    +import org.apache.flink.test.util.FlinkTestBase
    +
    +
    +class LossFunctionTest extends FlatSpec with Matchers with FlinkTestBase {
    --- End diff --
    
    We don't need to extend `FlinkTestBase` because `LossFunctionTest` do not use Flink cluster.


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#issuecomment-219702326
  
    Thanks for your contribution @skavulya. The changes look really good :-) I especially like your thorough and clear documentation of the different loss functions. Great work.
    
    I had some minor comments. Once fixed, the PR should be good to be 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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63509887
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/PartialLossFunction.scala ---
    @@ -47,21 +47,106 @@ object SquaredLoss extends PartialLossFunction {
     
       /** Calculates the loss depending on the label and the prediction
         *
    -    * @param prediction
    -    * @param label
    -    * @return
    +    * @param prediction The predicted value
    +    * @param label The true value
    +    * @return The loss
         */
       override def loss(prediction: Double, label: Double): Double = {
         0.5 * (prediction - label) * (prediction - label)
       }
     
       /** Calculates the derivative of the [[PartialLossFunction]]
         *
    -    * @param prediction
    -    * @param label
    -    * @return
    +    * @param prediction The predicted value
    +    * @param label The true value
    +    * @return The derivative of the loss function
    --- End diff --
    
    Good code completion, 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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63509482
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/GradientDescent.scala ---
    @@ -306,7 +303,7 @@ object GradientDescentL2 {
       * The regularization function is `||w||_1` with `w` being the weight vector.
       */
     class GradientDescentL1 extends GradientDescent {
    -
    +  //TODO(skavulya): Pass regularization penalty as a parameter
    --- End diff --
    
    TODOs are always good to be resolved :-)


---
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 #1985: [FLINK-1979] Add logistic loss, hinge loss and regulariza...

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

    https://github.com/apache/flink/pull/1985
  
    hi @chiwanpark sorry, I hadn't checked this PR for a while. I merged the latest master. Do you have any preference on an alternative name for RegularizationPenalty? Would Regularizer work 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-1979] Add logistic loss, hinge loss and regulariz...

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

    https://github.com/apache/flink/pull/1985#discussion_r65291461
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/PartialLossFunction.scala ---
    @@ -47,21 +47,106 @@ object SquaredLoss extends PartialLossFunction {
     
       /** Calculates the loss depending on the label and the prediction
         *
    -    * @param prediction
    -    * @param label
    -    * @return
    +    * @param prediction The predicted value
    +    * @param label The true value
    +    * @return The loss
         */
       override def loss(prediction: Double, label: Double): Double = {
         0.5 * (prediction - label) * (prediction - label)
       }
     
       /** Calculates the derivative of the [[PartialLossFunction]]
         *
    -    * @param prediction
    -    * @param label
    -    * @return
    +    * @param prediction The predicted value
    +    * @param label The true value
    +    * @return The derivative of the loss function
         */
       override def derivative(prediction: Double, label: Double): Double = {
         (prediction - label)
       }
     }
    +
    +/** Logistic loss function which can be used with the [[GenericLossFunction]]
    +  *
    +  *
    +  * The [[LogisticLoss]] function implements `log(1 + -exp(prediction*label))`
    +  * for binary classification with label in {-1, 1}
    +  */
    +object LogisticLoss extends PartialLossFunction {
    +
    +  /** Calculates the loss depending on the label and the prediction
    +    *
    +    * @param prediction The predicted value
    +    * @param label The true value
    +    * @return The loss
    +    */
    +  override def loss(prediction: Double, label: Double): Double = {
    +    val z = prediction * label
    +
    +    // based on implementation in scikit-learn
    +    // approximately equal and saves the computation of the log
    +    if (z > 18) {
    +      return math.exp(-z)
    +    }
    +    else if (z < -18) {
    +      return -z
    +    }
    +
    +    math.log(1 + math.exp(-z))
    +  }
    +
    +  /** Calculates the derivative of the loss function with respect to the prediction
    +    *
    +    * @param prediction The predicted value
    +    * @param label The true value
    +    * @return The derivative of the loss function
    +    */
    +  override def derivative(prediction: Double, label: Double): Double = {
    +    val z = prediction * label
    +
    +    // based on implementation in scikit-learn
    +    // approximately equal and saves the computation of the log
    +    if (z > 18) {
    +      return -label * math.exp(-z)
    +    }
    +    else if (z < -18) {
    +      return -label
    +    }
    +
    +    -label/(math.exp(z) + 1)
    +  }
    --- End diff --
    
    As I said above, following is better:
    
    ```scala
    if (z > 18) {
      -label * math.exp(-z)
    } else if (z < -18) {
      -label
    } else {
      -label / (math.exp(z) + 1)
    }
    ```


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63509731
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/LossFunction.scala ---
    @@ -23,8 +23,8 @@ import org.apache.flink.ml.math.BLAS
     
     /** Abstract class that implements some of the functionality for common loss functions
       *
    -  * A loss function determines the loss term $L(w) of the objective function  $f(w) = L(w) +
    -  * \lambda R(w)$ for prediction tasks, the other being regularization, $R(w)$.
    +  * A loss function determines the loss term `L(w)` of the objective function  `f(w) = L(w) +
    +  * lambda*R(w)` for prediction tasks, the other being regularization, `R(w)`.
    --- End diff --
    
    Good catch :-)


---
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 #1985: [FLINK-1979] Add logistic loss, hinge loss and regulariza...

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

    https://github.com/apache/flink/pull/1985
  
    The PR looks good to me. Thanks a lot for your contribution @skavulya. I will merge this PR once the rebased code passes Travis :-)


---
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-1979] Add logistic loss, hinge loss and...

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

    https://github.com/apache/flink/pull/1985#discussion_r63509537
  
    --- Diff: flink-libraries/flink-ml/src/main/scala/org/apache/flink/ml/optimization/GradientDescent.scala ---
    @@ -348,7 +334,7 @@ object GradientDescentL1 {
       * No regularization is applied.
       */
     class SimpleGradientDescent extends GradientDescent {
    -
    +  //TODO(skavulya): Pass regularization penalty as a parameter
    --- End diff --
    
    TODO


---
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 #1985: [FLINK-1979] Add logistic loss, hinge loss and regulariza...

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

    https://github.com/apache/flink/pull/1985
  
    Could this PR be merged ? Maybe this PR blocks to design GLM([FLINK-2013](https://issues.apache.org/jira/browse/FLINK-2013)).


---
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-1979] Add logistic loss, hinge loss and regulariz...

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

    https://github.com/apache/flink/pull/1985#discussion_r65291933
  
    --- Diff: flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/optimization/RegularizationPenaltyTest.scala ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.optimization
    +
    +import org.apache.flink.ml.math.DenseVector
    +import org.apache.flink.test.util.FlinkTestBase
    +import org.scalatest.{FlatSpec, Matchers}
    +
    +
    +class RegularizationPenaltyTest extends FlatSpec with Matchers with FlinkTestBase {
    --- End diff --
    
    Same as above, we don't need to extend `FlinkTestBase`.


---
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 #1985: [FLINK-1979] Add logistic loss, hinge loss and regulariza...

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

    https://github.com/apache/flink/pull/1985
  
    Thanks @chiwanpark. I made the changes you recommended. Please let me know if it looks 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.
---