You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by jk...@apache.org on 2017/08/24 17:18:57 UTC

spark git commit: [SPARK-21681][ML] fix bug of MLOR do not work correctly when featureStd contains zero (backport PR for 2.2)

Repository: spark
Updated Branches:
  refs/heads/branch-2.2 a58536741 -> 2b4bd7910


[SPARK-21681][ML] fix bug of MLOR do not work correctly when featureStd contains zero (backport PR for 2.2)

## What changes were proposed in this pull request?

This is backport PR of https://github.com/apache/spark/pull/18896

fix bug of MLOR do not work correctly when featureStd contains zero

We can reproduce the bug through such dataset (features including zero variance), will generate wrong result (all coefficients becomes 0)
```
    val multinomialDatasetWithZeroVar = {
      val nPoints = 100
      val coefficients = Array(
        -0.57997, 0.912083, -0.371077,
        -0.16624, -0.84355, -0.048509)

      val xMean = Array(5.843, 3.0)
      val xVariance = Array(0.6856, 0.0)  // including zero variance

      val testData = generateMultinomialLogisticInput(
        coefficients, xMean, xVariance, addIntercept = true, nPoints, seed)

      val df = sc.parallelize(testData, 4).toDF().withColumn("weight", lit(1.0))
      df.cache()
      df
    }
```
## How was this patch tested?

testcase added.

Author: WeichenXu <We...@outlook.com>

Closes #19026 from WeichenXu123/fix_mlor_zero_var_bug_2_2.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2b4bd791
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2b4bd791
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2b4bd791

Branch: refs/heads/branch-2.2
Commit: 2b4bd7910fecc8b7b41c7d4388d2a8204c1901e8
Parents: a585367
Author: Weichen Xu <we...@databricks.com>
Authored: Thu Aug 24 10:18:56 2017 -0700
Committer: Joseph K. Bradley <jo...@databricks.com>
Committed: Thu Aug 24 10:18:56 2017 -0700

----------------------------------------------------------------------
 .../ml/classification/LogisticRegression.scala  | 12 ++--
 .../LogisticRegressionSuite.scala               | 75 ++++++++++++++++++++
 2 files changed, 82 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/2b4bd791/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
index 567af04..1de2373 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
@@ -1727,11 +1727,13 @@ private class LogisticAggregator(
 
     val margins = new Array[Double](numClasses)
     features.foreachActive { (index, value) =>
-      val stdValue = value / localFeaturesStd(index)
-      var j = 0
-      while (j < numClasses) {
-        margins(j) += localCoefficients(index * numClasses + j) * stdValue
-        j += 1
+      if (localFeaturesStd(index) != 0.0 && value != 0.0) {
+        val stdValue = value / localFeaturesStd(index)
+        var j = 0
+        while (j < numClasses) {
+          margins(j) += localCoefficients(index * numClasses + j) * stdValue
+          j += 1
+        }
       }
     }
     var i = 0

http://git-wip-us.apache.org/repos/asf/spark/blob/2b4bd791/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
----------------------------------------------------------------------
diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
index 1ffd8dc..8461d64 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
@@ -45,6 +45,7 @@ class LogisticRegressionSuite
   @transient var smallMultinomialDataset: Dataset[_] = _
   @transient var binaryDataset: Dataset[_] = _
   @transient var multinomialDataset: Dataset[_] = _
+  @transient var multinomialDatasetWithZeroVar: Dataset[_] = _
   private val eps: Double = 1e-5
 
   override def beforeAll(): Unit = {
@@ -98,6 +99,23 @@ class LogisticRegressionSuite
       df.cache()
       df
     }
+
+    multinomialDatasetWithZeroVar = {
+      val nPoints = 100
+      val coefficients = Array(
+        -0.57997, 0.912083, -0.371077,
+        -0.16624, -0.84355, -0.048509)
+
+      val xMean = Array(5.843, 3.0)
+      val xVariance = Array(0.6856, 0.0)
+
+      val testData = generateMultinomialLogisticInput(
+        coefficients, xMean, xVariance, addIntercept = true, nPoints, seed)
+
+      val df = sc.parallelize(testData, 4).toDF().withColumn("weight", lit(1.0))
+      df.cache()
+      df
+    }
   }
 
   /**
@@ -111,6 +129,11 @@ class LogisticRegressionSuite
     multinomialDataset.rdd.map { case Row(label: Double, features: Vector, weight: Double) =>
       label + "," + weight + "," + features.toArray.mkString(",")
     }.repartition(1).saveAsTextFile("target/tmp/LogisticRegressionSuite/multinomialDataset")
+    multinomialDatasetWithZeroVar.rdd.map {
+      case Row(label: Double, features: Vector, weight: Double) =>
+        label + "," + weight + "," + features.toArray.mkString(",")
+    }.repartition(1)
+      .saveAsTextFile("target/tmp/LogisticRegressionSuite/multinomialDatasetWithZeroVar")
   }
 
   test("params") {
@@ -1391,6 +1414,58 @@ class LogisticRegressionSuite
     assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps)
   }
 
+  test("multinomial logistic regression with zero variance (SPARK-21681)") {
+    val sqlContext = multinomialDatasetWithZeroVar.sqlContext
+    import sqlContext.implicits._
+    val mlr = new LogisticRegression().setFamily("multinomial").setFitIntercept(true)
+      .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true).setWeightCol("weight")
+
+    val model = mlr.fit(multinomialDatasetWithZeroVar)
+
+    /*
+     Use the following R code to load the data and train the model using glmnet package.
+     library("glmnet")
+     data <- read.csv("path", header=FALSE)
+     label = as.factor(data$V1)
+     w = data$V2
+     features = as.matrix(data.frame(data$V3, data$V4))
+     coefficients = coef(glmnet(features, label, weights=w, family="multinomial",
+     alpha = 0, lambda = 0))
+     coefficients
+     $`0`
+     3 x 1 sparse Matrix of class "dgCMatrix"
+                    s0
+             0.2658824
+     data.V3 0.1881871
+     data.V4 .
+     $`1`
+     3 x 1 sparse Matrix of class "dgCMatrix"
+                      s0
+              0.53604701
+     data.V3 -0.02412645
+     data.V4  .
+     $`2`
+     3 x 1 sparse Matrix of class "dgCMatrix"
+                     s0
+             -0.8019294
+     data.V3 -0.1640607
+     data.V4  .
+    */
+
+    val coefficientsR = new DenseMatrix(3, 2, Array(
+      0.1881871, 0.0,
+      -0.02412645, 0.0,
+      -0.1640607, 0.0), isTransposed = true)
+    val interceptsR = Vectors.dense(0.2658824, 0.53604701, -0.8019294)
+
+    model.coefficientMatrix.colIter.foreach(v => assert(v.toArray.sum ~== 0.0 absTol eps))
+
+    assert(model.coefficientMatrix ~== coefficientsR relTol 0.05)
+    assert(model.coefficientMatrix.toArray.sum ~== 0.0 absTol eps)
+    assert(model.interceptVector ~== interceptsR relTol 0.05)
+    assert(model.interceptVector.toArray.sum ~== 0.0 absTol eps)
+  }
+
   test("multinomial logistic regression with intercept without regularization with bound") {
     // Bound constrained optimization with bound on one side.
     val lowerBoundsOnCoefficients = Matrices.dense(3, 4, Array.fill(12)(1.0))


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org