[SPARK-13590][ML][DOC] Document spark.ml LiR, LoR and AFTSurvivalRegression behavior difference

## What changes were proposed in this pull request?
When fitting ```LinearRegressionModel```(by "l-bfgs" solver) and ```LogisticRegressionModel``` w/o intercept on dataset with constant nonzero column, spark.ml produce same model as R glmnet but different from LIBSVM.

When fitting ```AFTSurvivalRegressionModel``` w/o intercept on dataset with constant nonzero column, spark.ml produce different model compared with R survival::survreg.

We should output a warning message and clarify in document for this condition.

## How was this patch tested?
Document change, no unit test.

cc mengxr

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #12731 from yanboliang/spark-13590.
This commit is contained in:
Yanbo Liang 2016-06-07 15:25:36 -07:00
parent 890baaca50
commit 6ecedf39b4
4 changed files with 28 additions and 1 deletions

View file

@ -62,6 +62,8 @@ For more background and more details about the implementation, refer to the docu
> The current implementation of logistic regression in `spark.ml` only supports binary classes. Support for multiclass regression will be added in the future.
> When fitting LogisticRegressionModel without intercept on dataset with constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero columns. This behavior is the same as R glmnet but different from LIBSVM.
**Example**
The following example shows how to train a logistic regression model
@ -351,6 +353,8 @@ Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classificat
The interface for working with linear regression models and model
summaries is similar to the logistic regression case.
> When fitting LinearRegressionModel without intercept on dataset with constant nonzero column by "l-bfgs" solver, Spark MLlib outputs zero coefficients for constant nonzero columns. This behavior is the same as R glmnet but different from LIBSVM.
**Example**
The following
@ -666,6 +670,8 @@ The optimization algorithm underlying the implementation is L-BFGS.
The implementation matches the result from R's survival function
[survreg](https://stat.ethz.ch/R-manual/R-devel/library/survival/html/survreg.html)
> When fitting AFTSurvivalRegressionModel without intercept on dataset with constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero columns. This behavior is different from R survival::survreg.
**Example**
<div class="codetabs">

View file

@ -333,6 +333,13 @@ class LogisticRegression @Since("1.2.0") (
val featuresMean = summarizer.mean.toArray
val featuresStd = summarizer.variance.toArray.map(math.sqrt)
if (!$(fitIntercept) && (0 until numFeatures).exists { i =>
featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) {
logWarning("Fitting LogisticRegressionModel without intercept on dataset with " +
"constant nonzero column, Spark MLlib outputs zero coefficients for constant " +
"nonzero columns. This behavior is the same as R glmnet but different from LIBSVM.")
}
val regParamL1 = $(elasticNetParam) * $(regParam)
val regParamL2 = (1.0 - $(elasticNetParam)) * $(regParam)

View file

@ -209,11 +209,18 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S
}
val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt)
val numFeatures = featuresStd.size
if (!$(fitIntercept) && (0 until numFeatures).exists { i =>
featuresStd(i) == 0.0 && featuresSummarizer.mean(i) != 0.0 }) {
logWarning("Fitting AFTSurvivalRegressionModel without intercept on dataset with " +
"constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero " +
"columns. This behavior is different from R survival::survreg.")
}
val costFun = new AFTCostFun(instances, $(fitIntercept), featuresStd)
val optimizer = new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol))
val numFeatures = featuresStd.size
/*
The parameters vector has three parts:
the first element: Double, log(sigma), the log of scale parameter

View file

@ -267,6 +267,13 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String
val featuresMean = featuresSummarizer.mean.toArray
val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt)
if (!$(fitIntercept) && (0 until numFeatures).exists { i =>
featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) {
logWarning("Fitting LinearRegressionModel without intercept on dataset with " +
"constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero " +
"columns. This behavior is the same as R glmnet but different from LIBSVM.")
}
// Since we implicitly do the feature scaling when we compute the cost function
// to improve the convergence, the effective regParam will be changed.
val effectiveRegParam = $(regParam) / yStd