[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-02-22 Thread Sean R. Owen (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17288806#comment-17288806
 ] 

Sean R. Owen commented on SPARK-34448:
--

So one coarse response is - I'm surprised if the initialization should matter 
_that_ much? Starting the intercept at this value is kind of like starting it 
at the mean of the response in linear regression - probably the best a priori 
guess. That's why I am wondering about convergence (but sounds like it 
converges?) or what scikit does. Given the small data set, is the answer 
under-determined in this case?

I'd have to actually look at your test case to answer those questions, but 
that's what I'm thinking here. Maybe you have already thought it through.

What's a better initial value of the intercept?

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-02-23 Thread Yakov Kerzhner (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17289113#comment-17289113
 ] 

Yakov Kerzhner commented on SPARK-34448:


As I said in the description, I do not believe that the starting point should 
cause this bug; the minimizer should still drift to the proper minimum.  I said 
the fact that the log(odds) was made the starting point seems to suggest that 
whoever wrote the code believed that the intercept should be close to the 
log(odds), which is only true if the data is centered.  If I had to guess, I 
would guess that there is something in the objective function that pulls the 
intercept towards the log(odds).  This would be a bug, as the log(odds) is a 
good approximation for the intercept if and only if the data is centered.  For 
non-centered data, it is completely wrong to have the intercept equal (or be 
close to) the log(odds).  My test shows precisely this, that when the data is 
not centered, spark still returns an intercept equal to the log(odds) (test 
2.b, Intercept: -3.5428941035683303, log(odds): -3.542495168380248, correct 
intercept: -4).  Indeed, even for centered data, (test 1.b), it returns an 
intercept almost equal to the log(odds), (test 1.b. log(odds): 
-3.9876303002978997 Intercept: -3.987260922443554, correct intercept: -4).  So 
we need to dig into the objective function, and whether somewhere in there is a 
term that penalizes the intercept moving away from the log(odds). 

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-02-24 Thread Sean R. Owen (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17290052#comment-17290052
 ] 

Sean R. Owen commented on SPARK-34448:
--

Yes I believe you're definitely correct there's a problem here. [~dbtsai] can I 
add you in here? I think you worked on the LR solver many years ago.

I skimmed the source code in sklearn and looks like the SAG solver starts with 
a 0 intercept:
https://github.com/scikit-learn/scikit-learn/blob/638b7689bbbfae4bcc4592c6f8a43ce86b571f0b/sklearn/linear_model/tests/test_sag.py#L73

Maybe ... this is the issue? I can try porting your test case to Scala to see 
if it fixes it. But the existing test suites seem to pass with a 0 initial 
intercept, at least.

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-02-24 Thread Sean R. Owen (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17290158#comment-17290158
 ] 

Sean R. Owen commented on SPARK-34448:
--

I crudely ported the test setup to a Scala test, and tried a 0 initial 
intercept in the LR implementation. It still gets the -3.5 intercept in the 
case where the 'const_feature' column is added, but -4 without. So, I'm not 
sure that's it.

Let me ping [~podongfeng] or maybe even [~sethah] who have worked on that code 
a bit and might have more of an idea about why the intercept wouldn't quite fit 
right in this case. I'm wondering if there is some issue in 
LogisticAggregator's treatment of the intercept? no idea, this is outside my 
expertise.

https://github.com/apache/spark/blob/3ce4ab545bfc28db7df2c559726b887b0c8c33b7/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala#L244

BTW here's my hacked up test: 

{code}
  test("BLR") {
val centered = false
val regParam = 1.0e-8
val num_distribution_samplings = 1000
val num_rows_per_sampling = 1000
val theta_1 = 0.3f
val theta_2 = 0.2f
val intercept = -4.0f

val (feature1, feature2, target) = generate_blr_data(theta_1, theta_2, 
intercept, centered,
  num_distribution_samplings, num_rows_per_sampling)

val num_rows = num_distribution_samplings * num_rows_per_sampling

val const_feature = Array.fill(num_rows)(1.0f)
(0 until num_rows / 10).foreach { i => const_feature(i) = 0.9f }


val data = (0 until num_rows).map { i =>
  (feature1(i), feature2(i), const_feature(i), target(i))
}

val spark_df = spark.createDataFrame(data).toDF("feature1", "feature2", 
"const_feature", "label").cache()

val vec = new VectorAssembler().setInputCols(Array("feature1", 
"feature2")).setOutputCol(("features"))
val spark_df1 = vec.transform(spark_df).cache()

val lr = new LogisticRegression().
  
setMaxIter(100).setRegParam(regParam).setElasticNetParam(0.5).setFitIntercept(true)
val lrModel = lr.fit(spark_df1)
println("Just the blr data")
println("Coefficients: " + lrModel.coefficients)
println("Intercept: " + lrModel.intercept)

val vec2 = new VectorAssembler().setInputCols(Array("feature1", "feature2", 
"const_feature")).
  setOutputCol(("features"))
val spark_df2 = vec2.transform(spark_df).cache()

val lrModel2 = lr.fit(spark_df2)
println("blr data plus one vector that is filled with 1's and .9's")
println("Coefficients: " + lrModel2.coefficients)
println("Intercept: " + lrModel2.intercept)

  }

  def generate_blr_data(theta_1: Float,
theta_2: Float,
intercept: Float,
centered: Boolean,
num_distribution_samplings: Int,
num_rows_per_sampling: Int): (Array[Float], 
Array[Float], Array[Int]) = {
val random = new Random(12345L)
val uniforms = Array.fill(num_distribution_samplings)(random.nextFloat())
val uniforms2 = Array.fill(num_distribution_samplings)(random.nextFloat())

if (centered) {
  uniforms.transform(f => f - 0.5f)
  uniforms2.transform(f => 2.0f * f - 1.0f)
} else {
  uniforms2.transform(f => f + 1.0f)
}

val h_theta = uniforms.zip(uniforms2).map { case (a, b) => intercept + 
theta_1 * a + theta_2 * b }
val prob = h_theta.map(t => 1.0 / (1.0 + math.exp(-t)))
val array = Array.ofDim[Int](num_distribution_samplings, 
num_rows_per_sampling)
array.indices.foreach { i =>
  (0 until math.round(num_rows_per_sampling * prob(i)).toInt).foreach { j =>
array(i)(j) = 1
  }
}

val num_rows = num_distribution_samplings * num_rows_per_sampling

val feature_1 = uniforms.map(f => 
Array.fill(num_rows_per_sampling)(f)).flatten
val feature_2 = uniforms2.map(f => 
Array.fill(num_rows_per_sampling)(f)).flatten
val target = array.flatten

return (feature_1, feature_2, target)
  }
{code}

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept

[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-02-24 Thread zhengruifeng (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17290642#comment-17290642
 ] 

zhengruifeng commented on SPARK-34448:
--

[~srowen] Thanks for pinging me, I am going to look into this issue

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-02-25 Thread zhengruifeng (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17291378#comment-17291378
 ] 

zhengruifeng commented on SPARK-34448:
--

[~srowen] [~weichenxu123]  [~ykerzhner] 

My findings until now:

1, as to param {{standardization, its name and doc is misleading. No matter 
whether it is true (by default) or false, LR always `standardize` the input 
vectors in a special way (x => x / std(x)), but the transformed vectors are not 
centered;}}

{{2, for the scala testsuite above, I log out the internal gradient and model 
(intercept & coef) at each iteration. I check the objective function and 
gradient, and it seems that they are calculated correctly;}}

{{3, }}{{for the case with const_feature(0.9 & 1.0) above,}}{{ the mean & std 
of three input features are:}}
{code:java}
featuresMean: [0.4999142959117828,1.4847274177074965,0.989976158129]
featuresStd: [0.28501348037270735,0.28375633081273305,0.0302215257344]{code}
{{note that const_feature (its std is 0.03) will be scaled to (30.0 & 33.3).}}

 

*{{I suspect that the underlying solvers (OWLQN/LBFGS/LBFGSB) can not handle a 
feature with such large(>30) values.}}*

{{3.1, Since std vec affects both the internal scaling and regularization, I 
disable regularization by setting regParam 0.0 to see whether this scaling 
matters.}}

{{With *LBFGS* Solver, the issue also exists, the solution with const_feature 
is:
}}
{code:java}
Coefficients: [0.29713531586902586,0.1928976631256973,-0.44332696536594945]
Intercept: -3.548585606117963 {code}
{{ }}

{{Then I manually set std vec to one values:}}
{code:java}
 val featuresStd = Array.fill(featuresMean.length)(1.0){code}
{{Then the optimization procedure behaviors as expectations, and the solution 
is:}}
{code:java}
Coefficients: [0.298868144564205,0.20101389459979044,0.008381706578824933]
Intercept: -4.009204134794202 {code}
 

{{3.2, here I reset the regParam to 0.5, with *OWLQN* Solver, the solution with 
all ones std is:}}
{code:java}
Coefficients: [0.296817926857017,0.19312282148846005,-0.17682584221569103]
Intercept: -3.8124413640824466 {code}
 

{{Compared to previous solution:}}
{code:java}
Coefficients: [0.2997261304455311,0.18830032771483074,-0.44301560942213103]
Intercept: -3.5428941035683303 {code}
{{I think the new solution with unit std vec fits better.}}

 

{{To summary, I guess the internal standardization should center the vectors in 
some way to match existing solver.}}

 

{{TODO:}}

{{1, I will refer to other impls to see how standardization is impled;}}

{{2, I will go on this issue to see what will happen if the vectors are 
centered;}}

{{3, This issue may also exist in LiR/SVC/etc. I will check in the future;}}

 

 

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-02-25 Thread Apache Spark (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17291383#comment-17291383
 ] 

Apache Spark commented on SPARK-34448:
--

User 'zhengruifeng' has created a pull request for this issue:
https://github.com/apache/spark/pull/31657

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-02-25 Thread zhengruifeng (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17291384#comment-17291384
 ] 

zhengruifeng commented on SPARK-34448:
--

My test code and log is here

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-02-26 Thread zhengruifeng (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17291559#comment-17291559
 ] 

zhengruifeng commented on SPARK-34448:
--

1, I just make a simple 
impl(https://github.com/apache/spark/pull/31657/commits/49141bbb178ac28af3263efa31299f8eb835830b)
 that internally center the vectors,

then the solution seems ok.
{code:java}
Coefficients: [0.29886424895473795,0.20097637066670226,0.0081964409252861]
Intercept: -4.0089605363236664 {code}
Moreover, it converge much faster.

 

2, however, if we center the vectors, then a lot of (>24) existing testsuite 
fails. It seems existing scaling was designed on purpose.

 

3, existing scaling (x/std_x) was added in 
https://issues.apache.org/jira/browse/SPARK-7262 , and aimed to keep in line 
with {{glmnet}}. But I am not familiar with {{glmnet}}.

{{In sklearn, linear_model.LogisticRegression}} does not standardize input 
vectors, while other linear models (i.e linear_model.ElasticNet) will 
'subtracting the mean and dividing by the l2-norm'.

 

 

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-02-26 Thread zhengruifeng (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17291562#comment-17291562
 ] 

zhengruifeng commented on SPARK-34448:
--

I am not sure to: 1, center the vector in existing impl (I can image that lots 
of testsuite will needed to change); 2, warn the end users to standardize the 
vectors outside of LR, if abs(mean(x)) is too large (maybe > 1.0?)

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-02-26 Thread Yakov Kerzhner (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17291757#comment-17291757
 ] 

Yakov Kerzhner commented on SPARK-34448:


The faster convergence when using standardization that includes centering makes 
sense as you can ahead of time guess the value of the intercept (it should 
equal the log(odds)).   What I still don't understand is how is it that in the 
case that the data is not centered, the intercept after the minimization is 
almost exactly equal to the log(odds).  This seems extremely strange to me and 
I can't find a mathematical reason for this to be happening.  In the original 
test example, could you print out the x, f(x), grad(x) as the minimizer moves 
from (0, 0, 0, log(odds)) to the minimum?

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-03-01 Thread Apache Spark (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17292779#comment-17292779
 ] 

Apache Spark commented on SPARK-34448:
--

User 'zhengruifeng' has created a pull request for this issue:
https://github.com/apache/spark/pull/31693

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-03-01 Thread Apache Spark (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17292778#comment-17292778
 ] 

Apache Spark commented on SPARK-34448:
--

User 'zhengruifeng' has created a pull request for this issue:
https://github.com/apache/spark/pull/31693

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-03-01 Thread Sean R. Owen (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17292924#comment-17292924
 ] 

Sean R. Owen commented on SPARK-34448:
--

[~ykerzhner] you may be interested in reviewing 
https://github.com/apache/spark/pull/31693

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-03-01 Thread Yakov Kerzhner (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17293017#comment-17293017
 ] 

Yakov Kerzhner commented on SPARK-34448:


I took a look over the weekend.  It seems good, and somewhat matches what I did 
in my test example where I centered before running the fitting.  Unfortunately, 
I am not very well versed in scala, so actually reviewing the code is a bit 
hard.  I appreciate the printouts for the test case in the PR, and I now 
understand why spark was returning the log(odds) for the intercept:  The 
division of a non centered vector with a small std dev creates a vector with 
very large entries that looks roughly like a constant vector.  When the 
minimizer computes the gradient, it assigns far more weight to this big vector 
than it does the intercept, as the magnitude appears more important than the 
fact that it isnt exactly constant.  When the optimizer then moves in the 
direction of the gradient, it finds that the value of the objective function 
actually increased (because of the fact that this big vector isnt exactly 
constant), and backtracks several times.  By the time it has backtracked enough 
to actually get a lower value on the objective function, the movement of the 
intercept is nearly 0.  So essentially, the intercept never moves during the 
entire calibration.  This is also why it takes so much longer (because of all 
the backtracking).  Once things are centered, the entries in the gradient for 
the intercept become dominant compared to the vector that is sort of constant, 
and so the minimizer begins adjusting the intercept, and moves it to the 
correct spot.

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org



[jira] [Commented] (SPARK-34448) Binary logistic regression incorrectly computes the intercept and coefficients when data is not centered

2021-03-01 Thread Yakov Kerzhner (Jira)


[ 
https://issues.apache.org/jira/browse/SPARK-34448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17293027#comment-17293027
 ] 

Yakov Kerzhner commented on SPARK-34448:


I will try to do a code review, but will focus on the comments so that people 
who see this in the future will understand what is happening.

> Binary logistic regression incorrectly computes the intercept and 
> coefficients when data is not centered
> 
>
> Key: SPARK-34448
> URL: https://issues.apache.org/jira/browse/SPARK-34448
> Project: Spark
>  Issue Type: Bug
>  Components: ML, MLlib
>Affects Versions: 2.4.5, 3.0.0
>Reporter: Yakov Kerzhner
>Priority: Major
>  Labels: correctness
>
> I have written up a fairly detailed gist that includes code to reproduce the 
> bug, as well as the output of the code and some commentary:
> [https://gist.github.com/ykerzhner/51358780a6a4cc33266515f17bf98a96]
> To summarize: under certain conditions, the minimization that fits a binary 
> logistic regression contains a bug that pulls the intercept value towards the 
> log(odds) of the target data.  This is mathematically only correct when the 
> data comes from distributions with zero means.  In general, this gives 
> incorrect intercept values, and consequently incorrect coefficients as well.
> As I am not so familiar with the spark code base, I have not been able to 
> find this bug within the spark code itself.  A hint to this bug is here: 
> [https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala#L894-L904]
> based on the code, I don't believe that the features have zero means at this 
> point, and so this heuristic is incorrect.  But an incorrect starting point 
> does not explain this bug.  The minimizer should drift to the correct place.  
> I was not able to find the code of the actual objective function that is 
> being minimized.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org