aboutsummaryrefslogtreecommitdiff
path: root/mllib
diff options
context:
space:
mode:
authorsethah <seth.hendrickson16@gmail.com>2016-09-19 21:33:54 -0700
committerDB Tsai <dbt@netflix.com>2016-09-19 21:33:54 -0700
commit26145a5af9a88053c0eaf280206ca2621c8919f6 (patch)
treeedf2965b4903c10504183006e0b60ee011dc286c /mllib
parente719b1c045ba185d242d21bbfcdee2c84dafc587 (diff)
downloadspark-26145a5af9a88053c0eaf280206ca2621c8919f6.tar.gz
spark-26145a5af9a88053c0eaf280206ca2621c8919f6.tar.bz2
spark-26145a5af9a88053c0eaf280206ca2621c8919f6.zip
[SPARK-17163][ML] Unified LogisticRegression interface
## What changes were proposed in this pull request? Merge `MultinomialLogisticRegression` into `LogisticRegression` and remove `MultinomialLogisticRegression`. Marked as WIP because we should discuss the coefficients API in the model. See discussion below. JIRA: [SPARK-17163](https://issues.apache.org/jira/browse/SPARK-17163) ## How was this patch tested? Merged test suites and added some new unit tests. ## Design ### Switching between binomial and multinomial We default to automatically detecting whether we should run binomial or multinomial lor. We expose a new parameter called `family` which defaults to auto. When "auto" is used, we run normal binomial lor with pivoting if there are 1 or 2 label classes. Otherwise, we run multinomial. If the user explicitly sets the family, then we abide by that setting. In the case where "binomial" is set but multiclass lor is detected, we throw an error. ### coefficients/intercept model API (TODO) This is the biggest design point remaining, IMO. We need to decide how to store the coefficients and intercepts in the model, and in turn how to expose them via the API. Two important points: * We must maintain compatibility with the old API, i.e. we must expose `def coefficients: Vector` and `def intercept: Double` * There are two separate cases: binomial lr where we have a single set of coefficients and a single intercept and multinomial lr where we have `numClasses` sets of coefficients and `numClasses` intercepts. Some options: 1. **Store the binomial coefficients as a `2 x numFeatures` matrix.** This means that we would center the model coefficients before storing them in the model. The BLOR algorithm gives `1 * numFeatures` coefficients, but we would convert them to `2 x numFeatures` coefficients before storing them, effectively doubling the storage in the model. This has the advantage that we can make the code cleaner (i.e. less `if (isMultinomial) ... else ...`) and we don't have to reason about the different cases as much. It has the disadvantage that we double the storage space and we could see small regressions at prediction time since there are 2x the number of operations in the prediction algorithms. Additionally, we still have to produce the uncentered coefficients/intercept via the API, so we will have to either ALSO store the uncentered version, or compute it in `def coefficients: Vector` every time. 2. **Store the binomial coefficients as a `1 x numFeatures` matrix.** We still store the coefficients as a matrix and the intercepts as a vector. When users call `coefficients` we return them a `Vector` that is backed by the same underlying array as the `coefficientMatrix`, so we don't duplicate any data. At prediction time, we use the old prediction methods that are specialized for binary LOR. The benefits here are that we don't store extra data, and we won't see any regressions in performance. The cost of this is that we have separate implementations for predict methods in the binary vs multiclass case. The duplicated code is really not very high, but it's still a bit messy. If we do decide to store the 2x coefficients, we would likely want to see some performance tests to understand the potential regressions. **Update:** We have chosen option 2 ### Threshold/thresholds (TODO) Currently, when `threshold` is set we clear whatever value is in `thresholds` and when `thresholds` is set we clear whatever value is in `threshold`. [SPARK-11543](https://issues.apache.org/jira/browse/SPARK-11543) was created to prefer thresholds over threshold. We should decide if we should implement this behavior now or if we want to do it in a separate JIRA. **Update:** Let's leave it for a follow up PR ## Follow up * Summary model for multiclass logistic regression [SPARK-17139](https://issues.apache.org/jira/browse/SPARK-17139) * Thresholds vs threshold [SPARK-11543](https://issues.apache.org/jira/browse/SPARK-11543) Author: sethah <seth.hendrickson16@gmail.com> Closes #14834 from sethah/SPARK-17163.
Diffstat (limited to 'mllib')
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala476
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/classification/MultinomialLogisticRegression.scala632
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala22
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala6
-rw-r--r--mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala1288
-rw-r--r--mllib/src/test/scala/org/apache/spark/ml/classification/MultinomialLogisticRegressionSuite.scala1056
-rw-r--r--mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala2
-rw-r--r--mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala2
-rw-r--r--mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala2
9 files changed, 1606 insertions, 1880 deletions
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 757d52052d..343d50c790 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
@@ -42,6 +42,7 @@ import org.apache.spark.sql.{DataFrame, Dataset, Row}
import org.apache.spark.sql.functions.{col, lit}
import org.apache.spark.sql.types.DoubleType
import org.apache.spark.storage.StorageLevel
+import org.apache.spark.util.VersionUtils
/**
* Params for logistic regression.
@@ -50,6 +51,8 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas
with HasRegParam with HasElasticNetParam with HasMaxIter with HasFitIntercept with HasTol
with HasStandardization with HasWeightCol with HasThreshold with HasAggregationDepth {
+ import org.apache.spark.ml.classification.LogisticRegression.supportedFamilyNames
+
/**
* Set threshold in binary classification, in range [0, 1].
*
@@ -66,12 +69,37 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas
*
* @group setParam
*/
+ // TODO: Implement SPARK-11543?
def setThreshold(value: Double): this.type = {
if (isSet(thresholds)) clear(thresholds)
set(threshold, value)
}
/**
+ * Param for the name of family which is a description of the label distribution
+ * to be used in the model.
+ * Supported options: "auto", "multinomial", "binomial".
+ * Supported options:
+ * - "auto": Automatically select the family based on the number of classes:
+ * If numClasses == 1 || numClasses == 2, set to "binomial".
+ * Else, set to "multinomial"
+ * - "binomial": Binary logistic regression with pivoting.
+ * - "multinomial": Multinomial logistic (softmax) regression without pivoting.
+ * Default is "auto".
+ *
+ * @group param
+ */
+ @Since("2.1.0")
+ final val family: Param[String] = new Param(this, "family",
+ "The name of family which is a description of the label distribution to be used in the " +
+ s"model. Supported options: ${supportedFamilyNames.mkString(", ")}.",
+ ParamValidators.inArray[String](supportedFamilyNames))
+
+ /** @group getParam */
+ @Since("2.1.0")
+ def getFamily: String = $(family)
+
+ /**
* Get threshold for binary classification.
*
* If [[thresholds]] is set with length 2 (i.e., binary classification),
@@ -154,9 +182,8 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas
}
/**
- * Logistic regression.
- * Currently, this class only supports binary classification. For multiclass classification,
- * use [[MultinomialLogisticRegression]]
+ * Logistic regression. Supports multinomial logistic (softmax) regression and binomial logistic
+ * regression.
*/
@Since("1.2.0")
class LogisticRegression @Since("1.2.0") (
@@ -221,6 +248,16 @@ class LogisticRegression @Since("1.2.0") (
setDefault(fitIntercept -> true)
/**
+ * Sets the value of param [[family]].
+ * Default is "auto".
+ *
+ * @group setParam
+ */
+ @Since("2.1.0")
+ def setFamily(value: String): this.type = set(family, value)
+ setDefault(family -> "auto")
+
+ /**
* Whether to standardize the training features before fitting the model.
* The coefficients of models will be always returned on the original scale,
* so it will be transparent for users. Note that with/without standardization,
@@ -261,6 +298,7 @@ class LogisticRegression @Since("1.2.0") (
* If the dimensions of features or the number of partitions are large,
* this param could be adjusted to a larger size.
* Default is 2.
+ *
* @group expertSetParam
*/
@Since("2.1.0")
@@ -311,8 +349,27 @@ class LogisticRegression @Since("1.2.0") (
val histogram = labelSummarizer.histogram
val numInvalid = labelSummarizer.countInvalid
- val numClasses = histogram.length
val numFeatures = summarizer.mean.size
+ val numFeaturesPlusIntercept = if (getFitIntercept) numFeatures + 1 else numFeatures
+
+ val numClasses = MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match {
+ case Some(n: Int) =>
+ require(n >= histogram.length, s"Specified number of classes $n was " +
+ s"less than the number of unique labels ${histogram.length}.")
+ n
+ case None => histogram.length
+ }
+
+ val isMultinomial = $(family) match {
+ case "binomial" =>
+ require(numClasses == 1 || numClasses == 2, s"Binomial family only supports 1 or 2 " +
+ s"outcome classes but found $numClasses.")
+ false
+ case "multinomial" => true
+ case "auto" => numClasses > 2
+ case other => throw new IllegalArgumentException(s"Unsupported family: $other")
+ }
+ val numCoefficientSets = if (isMultinomial) numClasses else 1
if (isDefined(thresholds)) {
require($(thresholds).length == numClasses, this.getClass.getSimpleName +
@@ -323,7 +380,7 @@ class LogisticRegression @Since("1.2.0") (
instr.logNumClasses(numClasses)
instr.logNumFeatures(numFeatures)
- val (coefficients, intercept, objectiveHistory) = {
+ val (coefficientMatrix, interceptVector, objectiveHistory) = {
if (numInvalid != 0) {
val msg = s"Classification labels should be in [0 to ${numClasses - 1}]. " +
s"Found $numInvalid invalid labels."
@@ -331,24 +388,26 @@ class LogisticRegression @Since("1.2.0") (
throw new SparkException(msg)
}
- val isConstantLabel = histogram.count(_ != 0) == 1
+ val isConstantLabel = histogram.count(_ != 0.0) == 1
- if (numClasses > 2) {
- val msg = s"LogisticRegression with ElasticNet in ML package only supports " +
- s"binary classification. Found $numClasses in the input dataset. Consider using " +
- s"MultinomialLogisticRegression instead."
- logError(msg)
- throw new SparkException(msg)
- } else if ($(fitIntercept) && numClasses == 2 && isConstantLabel) {
- logWarning(s"All labels are one and fitIntercept=true, so the coefficients will be " +
- s"zeros and the intercept will be positive infinity; as a result, " +
- s"training is not needed.")
- (Vectors.sparse(numFeatures, Seq()), Double.PositiveInfinity, Array.empty[Double])
- } else if ($(fitIntercept) && numClasses == 1) {
- logWarning(s"All labels are zero and fitIntercept=true, so the coefficients will be " +
- s"zeros and the intercept will be negative infinity; as a result, " +
- s"training is not needed.")
- (Vectors.sparse(numFeatures, Seq()), Double.NegativeInfinity, Array.empty[Double])
+ if ($(fitIntercept) && isConstantLabel) {
+ logWarning(s"All labels are the same value and fitIntercept=true, so the coefficients " +
+ s"will be zeros. Training is not needed.")
+ val constantLabelIndex = Vectors.dense(histogram).argmax
+ // TODO: use `compressed` after SPARK-17471
+ val coefMatrix = if (numFeatures < numCoefficientSets) {
+ new SparseMatrix(numCoefficientSets, numFeatures,
+ Array.fill(numFeatures + 1)(0), Array.empty[Int], Array.empty[Double])
+ } else {
+ new SparseMatrix(numCoefficientSets, numFeatures, Array.fill(numCoefficientSets + 1)(0),
+ Array.empty[Int], Array.empty[Double], isTransposed = true)
+ }
+ val interceptVec = if (isMultinomial) {
+ Vectors.sparse(numClasses, Seq((constantLabelIndex, Double.PositiveInfinity)))
+ } else {
+ Vectors.dense(if (numClasses == 2) Double.PositiveInfinity else Double.NegativeInfinity)
+ }
+ (coefMatrix, interceptVec, Array.empty[Double])
} else {
if (!$(fitIntercept) && isConstantLabel) {
logWarning(s"All labels belong to a single class and fitIntercept=false. It's a " +
@@ -370,7 +429,8 @@ class LogisticRegression @Since("1.2.0") (
val bcFeaturesStd = instances.context.broadcast(featuresStd)
val costFun = new LogisticCostFun(instances, numClasses, $(fitIntercept),
- $(standardization), bcFeaturesStd, regParamL2, multinomial = false, $(aggregationDepth))
+ $(standardization), bcFeaturesStd, regParamL2, multinomial = isMultinomial,
+ $(aggregationDepth))
val optimizer = if ($(elasticNetParam) == 0.0 || $(regParam) == 0.0) {
new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol))
@@ -378,18 +438,28 @@ class LogisticRegression @Since("1.2.0") (
val standardizationParam = $(standardization)
def regParamL1Fun = (index: Int) => {
// Remove the L1 penalization on the intercept
- if (index == numFeatures) {
+ val isIntercept = $(fitIntercept) && ((index + 1) % numFeaturesPlusIntercept == 0)
+ if (isIntercept) {
0.0
} else {
if (standardizationParam) {
regParamL1
} else {
+ val featureIndex = if ($(fitIntercept)) {
+ index % numFeaturesPlusIntercept
+ } else {
+ index % numFeatures
+ }
// If `standardization` is false, we still standardize the data
// to improve the rate of convergence; as a result, we have to
// perform this reverse standardization by penalizing each component
// differently to get effectively the same objective function when
// the training dataset is not standardized.
- if (featuresStd(index) != 0.0) regParamL1 / featuresStd(index) else 0.0
+ if (featuresStd(featureIndex) != 0.0) {
+ regParamL1 / featuresStd(featureIndex)
+ } else {
+ 0.0
+ }
}
}
}
@@ -397,22 +467,67 @@ class LogisticRegression @Since("1.2.0") (
}
val initialCoefficientsWithIntercept =
- Vectors.zeros(if ($(fitIntercept)) numFeatures + 1 else numFeatures)
-
- if (optInitialModel.isDefined && optInitialModel.get.coefficients.size != numFeatures) {
- val vecSize = optInitialModel.get.coefficients.size
- logWarning(
- s"Initial coefficients will be ignored!! As its size $vecSize did not match the " +
- s"expected size $numFeatures")
+ Vectors.zeros(numCoefficientSets * numFeaturesPlusIntercept)
+
+ val initialModelIsValid = optInitialModel match {
+ case Some(_initialModel) =>
+ val providedCoefs = _initialModel.coefficientMatrix
+ val modelIsValid = (providedCoefs.numRows == numCoefficientSets) &&
+ (providedCoefs.numCols == numFeatures) &&
+ (_initialModel.interceptVector.size == numCoefficientSets) &&
+ (_initialModel.getFitIntercept == $(fitIntercept))
+ if (!modelIsValid) {
+ logWarning(s"Initial coefficients will be ignored! Its dimensions " +
+ s"(${providedCoefs.numRows}, ${providedCoefs.numCols}) did not match the " +
+ s"expected size ($numCoefficientSets, $numFeatures)")
+ }
+ modelIsValid
+ case None => false
}
- if (optInitialModel.isDefined && optInitialModel.get.coefficients.size == numFeatures) {
- val initialCoefficientsWithInterceptArray = initialCoefficientsWithIntercept.toArray
- optInitialModel.get.coefficients.foreachActive { case (index, value) =>
- initialCoefficientsWithInterceptArray(index) = value
+ if (initialModelIsValid) {
+ val initialCoefWithInterceptArray = initialCoefficientsWithIntercept.toArray
+ val providedCoef = optInitialModel.get.coefficientMatrix
+ providedCoef.foreachActive { (row, col, value) =>
+ val flatIndex = row * numFeaturesPlusIntercept + col
+ // We need to scale the coefficients since they will be trained in the scaled space
+ initialCoefWithInterceptArray(flatIndex) = value * featuresStd(col)
}
if ($(fitIntercept)) {
- initialCoefficientsWithInterceptArray(numFeatures) == optInitialModel.get.intercept
+ optInitialModel.get.interceptVector.foreachActive { (index, value) =>
+ val coefIndex = (index + 1) * numFeaturesPlusIntercept - 1
+ initialCoefWithInterceptArray(coefIndex) = value
+ }
+ }
+ } else if ($(fitIntercept) && isMultinomial) {
+ /*
+ For multinomial logistic regression, when we initialize the coefficients as zeros,
+ it will converge faster if we initialize the intercepts such that
+ it follows the distribution of the labels.
+ {{{
+ P(1) = \exp(b_1) / Z
+ ...
+ P(K) = \exp(b_K) / Z
+ where Z = \sum_{k=1}^{K} \exp(b_k)
+ }}}
+ Since this doesn't have a unique solution, one of the solutions that satisfies the
+ above equations is
+ {{{
+ \exp(b_k) = count_k * \exp(\lambda)
+ b_k = \log(count_k) * \lambda
+ }}}
+ \lambda is a free parameter, so choose the phase \lambda such that the
+ mean is centered. This yields
+ {{{
+ b_k = \log(count_k)
+ b_k' = b_k - \mean(b_k)
+ }}}
+ */
+ val rawIntercepts = histogram.map(c => math.log(c + 1)) // add 1 for smoothing
+ val rawMean = rawIntercepts.sum / rawIntercepts.length
+ rawIntercepts.indices.foreach { i =>
+ initialCoefficientsWithIntercept.toArray(i * numFeaturesPlusIntercept + numFeatures) =
+ rawIntercepts(i) - rawMean
}
} else if ($(fitIntercept)) {
/*
@@ -446,6 +561,7 @@ class LogisticRegression @Since("1.2.0") (
state = states.next()
arrayBuilder += state.adjustedValue
}
+ bcFeaturesStd.destroy(blocking = false)
if (state == null) {
val msg = s"${optimizer.getClass.getName} failed."
@@ -460,33 +576,85 @@ class LogisticRegression @Since("1.2.0") (
as a result, no scaling is needed.
*/
val rawCoefficients = state.x.toArray.clone()
- var i = 0
- while (i < numFeatures) {
- rawCoefficients(i) *= { if (featuresStd(i) != 0.0) 1.0 / featuresStd(i) else 0.0 }
- i += 1
+ val coefficientArray = Array.tabulate(numCoefficientSets * numFeatures) { i =>
+ // flatIndex will loop though rawCoefficients, and skip the intercept terms.
+ val flatIndex = if ($(fitIntercept)) i + i / numFeatures else i
+ val featureIndex = i % numFeatures
+ if (featuresStd(featureIndex) != 0.0) {
+ rawCoefficients(flatIndex) / featuresStd(featureIndex)
+ } else {
+ 0.0
+ }
+ }
+
+ if ($(regParam) == 0.0 && isMultinomial) {
+ /*
+ When no regularization is applied, the multinomial coefficients lack identifiability
+ because we do not use a pivot class. We can add any constant value to the coefficients
+ and get the same likelihood. So here, we choose the mean centered coefficients for
+ reproducibility. This method follows the approach in glmnet, described here:
+
+ Friedman, et al. "Regularization Paths for Generalized Linear Models via
+ Coordinate Descent," https://core.ac.uk/download/files/153/6287975.pdf
+ */
+ val coefficientMean = coefficientArray.sum / coefficientArray.length
+ coefficientArray.indices.foreach { i => coefficientArray(i) -= coefficientMean}
}
- bcFeaturesStd.destroy(blocking = false)
- if ($(fitIntercept)) {
- (Vectors.dense(rawCoefficients.dropRight(1)).compressed, rawCoefficients.last,
- arrayBuilder.result())
+ val denseCoefficientMatrix =
+ new DenseMatrix(numCoefficientSets, numFeatures, coefficientArray, isTransposed = true)
+ // TODO: use `denseCoefficientMatrix.compressed` after SPARK-17471
+ val compressedCoefficientMatrix = if (isMultinomial) {
+ denseCoefficientMatrix
} else {
- (Vectors.dense(rawCoefficients).compressed, 0.0, arrayBuilder.result())
+ val compressedVector = Vectors.dense(coefficientArray).compressed
+ compressedVector match {
+ case dv: DenseVector => denseCoefficientMatrix
+ case sv: SparseVector =>
+ new SparseMatrix(1, numFeatures, Array(0, sv.indices.length), sv.indices, sv.values,
+ isTransposed = true)
+ }
}
+
+ val interceptsArray: Array[Double] = if ($(fitIntercept)) {
+ Array.tabulate(numCoefficientSets) { i =>
+ val coefIndex = (i + 1) * numFeaturesPlusIntercept - 1
+ rawCoefficients(coefIndex)
+ }
+ } else {
+ Array[Double]()
+ }
+ val interceptVector = if (interceptsArray.nonEmpty && isMultinomial) {
+ // The intercepts are never regularized, so we always center the mean.
+ val interceptMean = interceptsArray.sum / numClasses
+ interceptsArray.indices.foreach { i => interceptsArray(i) -= interceptMean }
+ Vectors.dense(interceptsArray)
+ } else if (interceptsArray.length == 1) {
+ Vectors.dense(interceptsArray)
+ } else {
+ Vectors.sparse(numCoefficientSets, Seq())
+ }
+ (compressedCoefficientMatrix, interceptVector.compressed, arrayBuilder.result())
}
}
if (handlePersistence) instances.unpersist()
- val model = copyValues(new LogisticRegressionModel(uid, coefficients, intercept))
- val (summaryModel, probabilityColName) = model.findSummaryModelAndProbabilityCol()
- val logRegSummary = new BinaryLogisticRegressionTrainingSummary(
- summaryModel.transform(dataset),
- probabilityColName,
- $(labelCol),
- $(featuresCol),
- objectiveHistory)
- val m = model.setSummary(logRegSummary)
+ val model = copyValues(new LogisticRegressionModel(uid, coefficientMatrix, interceptVector,
+ numClasses, isMultinomial))
+ // TODO: implement summary model for multinomial case
+ val m = if (!isMultinomial) {
+ val (summaryModel, probabilityColName) = model.findSummaryModelAndProbabilityCol()
+ val logRegSummary = new BinaryLogisticRegressionTrainingSummary(
+ summaryModel.transform(dataset),
+ probabilityColName,
+ $(labelCol),
+ $(featuresCol),
+ objectiveHistory)
+ model.setSummary(logRegSummary)
+ } else {
+ model
+ }
instr.logSuccess(m)
m
}
@@ -500,6 +668,9 @@ object LogisticRegression extends DefaultParamsReadable[LogisticRegression] {
@Since("1.6.0")
override def load(path: String): LogisticRegression = super.load(path)
+
+ private[classification] val supportedFamilyNames =
+ Array("auto", "binomial", "multinomial").map(_.toLowerCase)
}
/**
@@ -508,11 +679,59 @@ object LogisticRegression extends DefaultParamsReadable[LogisticRegression] {
@Since("1.4.0")
class LogisticRegressionModel private[spark] (
@Since("1.4.0") override val uid: String,
- @Since("2.0.0") val coefficients: Vector,
- @Since("1.3.0") val intercept: Double)
+ @Since("2.1.0") val coefficientMatrix: Matrix,
+ @Since("2.1.0") val interceptVector: Vector,
+ @Since("1.3.0") override val numClasses: Int,
+ private val isMultinomial: Boolean)
extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel]
with LogisticRegressionParams with MLWritable {
+ require(coefficientMatrix.numRows == interceptVector.size, s"Dimension mismatch! Expected " +
+ s"coefficientMatrix.numRows == interceptVector.size, but ${coefficientMatrix.numRows} != " +
+ s"${interceptVector.size}")
+
+ private[spark] def this(uid: String, coefficients: Vector, intercept: Double) =
+ this(uid, new DenseMatrix(1, coefficients.size, coefficients.toArray, isTransposed = true),
+ Vectors.dense(intercept), 2, isMultinomial = false)
+
+ /**
+ * A vector of model coefficients for "binomial" logistic regression. If this model was trained
+ * using the "multinomial" family then an exception is thrown.
+ * @return Vector
+ */
+ @Since("2.0.0")
+ def coefficients: Vector = if (isMultinomial) {
+ throw new SparkException("Multinomial models contain a matrix of coefficients, use " +
+ "coefficientMatrix instead.")
+ } else {
+ _coefficients
+ }
+
+ // convert to appropriate vector representation without replicating data
+ private lazy val _coefficients: Vector = {
+ require(coefficientMatrix.isTransposed,
+ "LogisticRegressionModel coefficients should be row major.")
+ coefficientMatrix match {
+ case dm: DenseMatrix => Vectors.dense(dm.values)
+ case sm: SparseMatrix => Vectors.sparse(coefficientMatrix.numCols, sm.rowIndices, sm.values)
+ }
+ }
+
+ /**
+ * The model intercept for "binomial" logistic regression. If this model was fit with the
+ * "multinomial" family then an exception is thrown.
+ * @return Double
+ */
+ @Since("1.3.0")
+ def intercept: Double = if (isMultinomial) {
+ throw new SparkException("Multinomial models contain a vector of intercepts, use " +
+ "interceptVector instead.")
+ } else {
+ _intercept
+ }
+
+ private lazy val _intercept = interceptVector.toArray.head
+
@Since("1.5.0")
override def setThreshold(value: Double): this.type = super.setThreshold(value)
@@ -527,7 +746,14 @@ class LogisticRegressionModel private[spark] (
/** Margin (rawPrediction) for class label 1. For binary classification only. */
private val margin: Vector => Double = (features) => {
- BLAS.dot(features, coefficients) + intercept
+ BLAS.dot(features, _coefficients) + _intercept
+ }
+
+ /** Margin (rawPrediction) for each class label. */
+ private val margins: Vector => Vector = (features) => {
+ val m = interceptVector.toDense.copy
+ BLAS.gemv(1.0, coefficientMatrix, features, 1.0, m)
+ m
}
/** Score (probability) for class label 1. For binary classification only. */
@@ -537,10 +763,7 @@ class LogisticRegressionModel private[spark] (
}
@Since("1.6.0")
- override val numFeatures: Int = coefficients.size
-
- @Since("1.3.0")
- override val numClasses: Int = 2
+ override val numFeatures: Int = coefficientMatrix.numCols
private var trainingSummary: Option[LogisticRegressionTrainingSummary] = None
@@ -595,7 +818,9 @@ class LogisticRegressionModel private[spark] (
* Predict label for the given feature vector.
* The behavior of this can be adjusted using [[thresholds]].
*/
- override protected def predict(features: Vector): Double = {
+ override protected def predict(features: Vector): Double = if (isMultinomial) {
+ super.predict(features)
+ } else {
// Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden.
if (score(features) > getThreshold) 1 else 0
}
@@ -603,13 +828,47 @@ class LogisticRegressionModel private[spark] (
override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = {
rawPrediction match {
case dv: DenseVector =>
- var i = 0
- val size = dv.size
- while (i < size) {
- dv.values(i) = 1.0 / (1.0 + math.exp(-dv.values(i)))
- i += 1
+ if (isMultinomial) {
+ val size = dv.size
+ val values = dv.values
+
+ // get the maximum margin
+ val maxMarginIndex = rawPrediction.argmax
+ val maxMargin = rawPrediction(maxMarginIndex)
+
+ if (maxMargin == Double.PositiveInfinity) {
+ var k = 0
+ while (k < size) {
+ values(k) = if (k == maxMarginIndex) 1.0 else 0.0
+ k += 1
+ }
+ } else {
+ val sum = {
+ var temp = 0.0
+ var k = 0
+ while (k < numClasses) {
+ values(k) = if (maxMargin > 0) {
+ math.exp(values(k) - maxMargin)
+ } else {
+ math.exp(values(k))
+ }
+ temp += values(k)
+ k += 1
+ }
+ temp
+ }
+ BLAS.scal(1 / sum, dv)
+ }
+ dv
+ } else {
+ var i = 0
+ val size = dv.size
+ while (i < size) {
+ dv.values(i) = 1.0 / (1.0 + math.exp(-dv.values(i)))
+ i += 1
+ }
+ dv
}
- dv
case sv: SparseVector =>
throw new RuntimeException("Unexpected error in LogisticRegressionModel:" +
" raw2probabilitiesInPlace encountered SparseVector")
@@ -617,33 +876,46 @@ class LogisticRegressionModel private[spark] (
}
override protected def predictRaw(features: Vector): Vector = {
- val m = margin(features)
- Vectors.dense(-m, m)
+ if (isMultinomial) {
+ margins(features)
+ } else {
+ val m = margin(features)
+ Vectors.dense(-m, m)
+ }
}
@Since("1.4.0")
override def copy(extra: ParamMap): LogisticRegressionModel = {
- val newModel = copyValues(new LogisticRegressionModel(uid, coefficients, intercept), extra)
+ val newModel = copyValues(new LogisticRegressionModel(uid, coefficientMatrix, interceptVector,
+ numClasses, isMultinomial), extra)
if (trainingSummary.isDefined) newModel.setSummary(trainingSummary.get)
newModel.setParent(parent)
}
override protected def raw2prediction(rawPrediction: Vector): Double = {
- // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden.
- val t = getThreshold
- val rawThreshold = if (t == 0.0) {
- Double.NegativeInfinity
- } else if (t == 1.0) {
- Double.PositiveInfinity
+ if (isMultinomial) {
+ super.raw2prediction(rawPrediction)
} else {
- math.log(t / (1.0 - t))
+ // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden.
+ val t = getThreshold
+ val rawThreshold = if (t == 0.0) {
+ Double.NegativeInfinity
+ } else if (t == 1.0) {
+ Double.PositiveInfinity
+ } else {
+ math.log(t / (1.0 - t))
+ }
+ if (rawPrediction(1) > rawThreshold) 1 else 0
}
- if (rawPrediction(1) > rawThreshold) 1 else 0
}
override protected def probability2prediction(probability: Vector): Double = {
- // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden.
- if (probability(1) > getThreshold) 1 else 0
+ if (isMultinomial) {
+ super.probability2prediction(probability)
+ } else {
+ // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden.
+ if (probability(1) > getThreshold) 1 else 0
+ }
}
/**
@@ -676,39 +948,53 @@ object LogisticRegressionModel extends MLReadable[LogisticRegressionModel] {
private case class Data(
numClasses: Int,
numFeatures: Int,
- intercept: Double,
- coefficients: Vector)
+ interceptVector: Vector,
+ coefficientMatrix: Matrix,
+ isMultinomial: Boolean)
override protected def saveImpl(path: String): Unit = {
// Save metadata and Params
DefaultParamsWriter.saveMetadata(instance, path, sc)
// Save model data: numClasses, numFeatures, intercept, coefficients
- val data = Data(instance.numClasses, instance.numFeatures, instance.intercept,
- instance.coefficients)
+ val data = Data(instance.numClasses, instance.numFeatures, instance.interceptVector,
+ instance.coefficientMatrix, instance.isMultinomial)
val dataPath = new Path(path, "data").toString
sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath)
}
}
- private class LogisticRegressionModelReader
- extends MLReader[LogisticRegressionModel] {
+ private class LogisticRegressionModelReader extends MLReader[LogisticRegressionModel] {
/** Checked against metadata when loading model */
private val className = classOf[LogisticRegressionModel].getName
override def load(path: String): LogisticRegressionModel = {
val metadata = DefaultParamsReader.loadMetadata(path, sc, className)
+ val (major, minor) = VersionUtils.majorMinorVersion(metadata.sparkVersion)
val dataPath = new Path(path, "data").toString
val data = sparkSession.read.format("parquet").load(dataPath)
- // We will need numClasses, numFeatures in the future for multinomial logreg support.
- // TODO: remove numClasses and numFeatures fields?
- val Row(numClasses: Int, numFeatures: Int, intercept: Double, coefficients: Vector) =
- MLUtils.convertVectorColumnsToML(data, "coefficients")
- .select("numClasses", "numFeatures", "intercept", "coefficients")
- .head()
- val model = new LogisticRegressionModel(metadata.uid, coefficients, intercept)
+ val model = if (major.toInt < 2 || (major.toInt == 2 && minor.toInt == 0)) {
+ // 2.0 and before
+ val Row(numClasses: Int, numFeatures: Int, intercept: Double, coefficients: Vector) =
+ MLUtils.convertVectorColumnsToML(data, "coefficients")
+ .select("numClasses", "numFeatures", "intercept", "coefficients")
+ .head()
+ val coefficientMatrix =
+ new DenseMatrix(1, coefficients.size, coefficients.toArray, isTransposed = true)
+ val interceptVector = Vectors.dense(intercept)
+ new LogisticRegressionModel(metadata.uid, coefficientMatrix,
+ interceptVector, numClasses, isMultinomial = false)
+ } else {
+ // 2.1+
+ val Row(numClasses: Int, numFeatures: Int, interceptVector: Vector,
+ coefficientMatrix: Matrix, isMultinomial: Boolean) = data
+ .select("numClasses", "numFeatures", "interceptVector", "coefficientMatrix",
+ "isMultinomial").head()
+ new LogisticRegressionModel(metadata.uid, coefficientMatrix, interceptVector,
+ numClasses, isMultinomial)
+ }
DefaultParamsReader.getAndSetParams(model, metadata)
model
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultinomialLogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultinomialLogisticRegression.scala
deleted file mode 100644
index 006f57c0ce..0000000000
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultinomialLogisticRegression.scala
+++ /dev/null
@@ -1,632 +0,0 @@
-/*
- * 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.spark.ml.classification
-
-import scala.collection.mutable
-
-import breeze.linalg.{DenseVector => BDV}
-import breeze.optimize.{CachedDiffFunction, LBFGS => BreezeLBFGS, OWLQN => BreezeOWLQN}
-import org.apache.hadoop.fs.Path
-
-import org.apache.spark.SparkException
-import org.apache.spark.annotation.{Experimental, Since}
-import org.apache.spark.internal.Logging
-import org.apache.spark.ml.feature.Instance
-import org.apache.spark.ml.linalg._
-import org.apache.spark.ml.param._
-import org.apache.spark.ml.param.shared._
-import org.apache.spark.ml.util._
-import org.apache.spark.mllib.linalg.VectorImplicits._
-import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{Dataset, Row}
-import org.apache.spark.sql.functions.{col, lit}
-import org.apache.spark.sql.types.DoubleType
-import org.apache.spark.storage.StorageLevel
-
-/**
- * Params for multinomial logistic (softmax) regression.
- */
-private[classification] trait MultinomialLogisticRegressionParams
- extends ProbabilisticClassifierParams with HasRegParam with HasElasticNetParam with HasMaxIter
- with HasFitIntercept with HasTol with HasStandardization with HasWeightCol
- with HasAggregationDepth {
-
- /**
- * Set thresholds in multiclass (or binary) classification to adjust the probability of
- * predicting each class. Array must have length equal to the number of classes, with values >= 0.
- * The class with largest value p/t is predicted, where p is the original probability of that
- * class and t is the class' threshold.
- *
- * @group setParam
- */
- def setThresholds(value: Array[Double]): this.type = {
- set(thresholds, value)
- }
-
- /**
- * Get thresholds for binary or multiclass classification.
- *
- * @group getParam
- */
- override def getThresholds: Array[Double] = {
- $(thresholds)
- }
-}
-
-/**
- * :: Experimental ::
- * Multinomial Logistic (softmax) regression.
- */
-@Since("2.1.0")
-@Experimental
-class MultinomialLogisticRegression @Since("2.1.0") (
- @Since("2.1.0") override val uid: String)
- extends ProbabilisticClassifier[Vector,
- MultinomialLogisticRegression, MultinomialLogisticRegressionModel]
- with MultinomialLogisticRegressionParams with DefaultParamsWritable with Logging {
-
- @Since("2.1.0")
- def this() = this(Identifiable.randomUID("mlogreg"))
-
- /**
- * Set the regularization parameter.
- * Default is 0.0.
- *
- * @group setParam
- */
- @Since("2.1.0")
- def setRegParam(value: Double): this.type = set(regParam, value)
- setDefault(regParam -> 0.0)
-
- /**
- * Set the ElasticNet mixing parameter.
- * For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.
- * For 0 < alpha < 1, the penalty is a combination of L1 and L2.
- * Default is 0.0 which is an L2 penalty.
- *
- * @group setParam
- */
- @Since("2.1.0")
- def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value)
- setDefault(elasticNetParam -> 0.0)
-
- /**
- * Set the maximum number of iterations.
- * Default is 100.
- *
- * @group setParam
- */
- @Since("2.1.0")
- def setMaxIter(value: Int): this.type = set(maxIter, value)
- setDefault(maxIter -> 100)
-
- /**
- * Set the convergence tolerance of iterations.
- * Smaller value will lead to higher accuracy with the cost of more iterations.
- * Default is 1E-6.
- *
- * @group setParam
- */
- @Since("2.1.0")
- def setTol(value: Double): this.type = set(tol, value)
- setDefault(tol -> 1E-6)
-
- /**
- * Whether to fit an intercept term.
- * Default is true.
- *
- * @group setParam
- */
- @Since("2.1.0")
- def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value)
- setDefault(fitIntercept -> true)
-
- /**
- * Whether to standardize the training features before fitting the model.
- * The coefficients of models will be always returned on the original scale,
- * so it will be transparent for users. Note that with/without standardization,
- * the models should always converge to the same solution when no regularization
- * is applied. In R's GLMNET package, the default behavior is true as well.
- * Default is true.
- *
- * @group setParam
- */
- @Since("2.1.0")
- def setStandardization(value: Boolean): this.type = set(standardization, value)
- setDefault(standardization -> true)
-
- /**
- * Sets the value of param [[weightCol]].
- * If this is not set or empty, we treat all instance weights as 1.0.
- * Default is not set, so all instances have weight one.
- *
- * @group setParam
- */
- @Since("2.1.0")
- def setWeightCol(value: String): this.type = set(weightCol, value)
-
- @Since("2.1.0")
- override def setThresholds(value: Array[Double]): this.type = super.setThresholds(value)
-
- /**
- * Suggested depth for treeAggregate (>= 2).
- * If the dimensions of features or the number of partitions are large,
- * this param could be adjusted to a larger size.
- * Default is 2.
- * @group expertSetParam
- */
- @Since("2.1.0")
- def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value)
- setDefault(aggregationDepth -> 2)
-
- override protected[spark] def train(dataset: Dataset[_]): MultinomialLogisticRegressionModel = {
- val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol))
- val instances: RDD[Instance] =
- dataset.select(col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd.map {
- case Row(label: Double, weight: Double, features: Vector) =>
- Instance(label, weight, features)
- }
-
- val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE
- if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK)
-
- val instr = Instrumentation.create(this, instances)
- instr.logParams(regParam, elasticNetParam, standardization, thresholds,
- maxIter, tol, fitIntercept)
-
- val (summarizer, labelSummarizer) = {
- val seqOp = (c: (MultivariateOnlineSummarizer, MultiClassSummarizer),
- instance: Instance) =>
- (c._1.add(instance.features, instance.weight), c._2.add(instance.label, instance.weight))
-
- val combOp = (c1: (MultivariateOnlineSummarizer, MultiClassSummarizer),
- c2: (MultivariateOnlineSummarizer, MultiClassSummarizer)) =>
- (c1._1.merge(c2._1), c1._2.merge(c2._2))
-
- instances.treeAggregate(
- new MultivariateOnlineSummarizer, new MultiClassSummarizer)(seqOp, combOp)
- }
-
- val histogram = labelSummarizer.histogram
- val numInvalid = labelSummarizer.countInvalid
- val numFeatures = summarizer.mean.size
- val numFeaturesPlusIntercept = if (getFitIntercept) numFeatures + 1 else numFeatures
-
- val numClasses = MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match {
- case Some(n: Int) =>
- require(n >= histogram.length, s"Specified number of classes $n was " +
- s"less than the number of unique labels ${histogram.length}")
- n
- case None => histogram.length
- }
-
- instr.logNumClasses(numClasses)
- instr.logNumFeatures(numFeatures)
-
- val (coefficients, intercepts, objectiveHistory) = {
- if (numInvalid != 0) {
- val msg = s"Classification labels should be in {0 to ${numClasses - 1} " +
- s"Found $numInvalid invalid labels."
- logError(msg)
- throw new SparkException(msg)
- }
-
- val isConstantLabel = histogram.count(_ != 0) == 1
-
- if ($(fitIntercept) && isConstantLabel) {
- // we want to produce a model that will always predict the constant label so all the
- // coefficients will be zero, and the constant label class intercept will be +inf
- val constantLabelIndex = Vectors.dense(histogram).argmax
- (Matrices.sparse(numClasses, numFeatures, Array.fill(numFeatures + 1)(0),
- Array.empty[Int], Array.empty[Double]),
- Vectors.sparse(numClasses, Seq((constantLabelIndex, Double.PositiveInfinity))),
- Array.empty[Double])
- } else {
- if (!$(fitIntercept) && isConstantLabel) {
- logWarning(s"All labels belong to a single class and fitIntercept=false. It's" +
- s"a dangerous ground, so the algorithm may not converge.")
- }
-
- val featuresStd = summarizer.variance.toArray.map(math.sqrt)
- val featuresMean = summarizer.mean.toArray
- if (!$(fitIntercept) && (0 until numFeatures).exists { i =>
- featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) {
- logWarning("Fitting MultinomialLogisticRegressionModel 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)
-
- val bcFeaturesStd = instances.context.broadcast(featuresStd)
- val costFun = new LogisticCostFun(instances, numClasses, $(fitIntercept),
- $(standardization), bcFeaturesStd, regParamL2, multinomial = true, $(aggregationDepth))
-
- val optimizer = if ($(elasticNetParam) == 0.0 || $(regParam) == 0.0) {
- new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol))
- } else {
- val standardizationParam = $(standardization)
- def regParamL1Fun = (index: Int) => {
- // Remove the L1 penalization on the intercept
- val isIntercept = $(fitIntercept) && ((index + 1) % numFeaturesPlusIntercept == 0)
- if (isIntercept) {
- 0.0
- } else {
- if (standardizationParam) {
- regParamL1
- } else {
- val featureIndex = if ($(fitIntercept)) {
- index % numFeaturesPlusIntercept
- } else {
- index % numFeatures
- }
- // If `standardization` is false, we still standardize the data
- // to improve the rate of convergence; as a result, we have to
- // perform this reverse standardization by penalizing each component
- // differently to get effectively the same objective function when
- // the training dataset is not standardized.
- if (featuresStd(featureIndex) != 0.0) {
- regParamL1 / featuresStd(featureIndex)
- } else {
- 0.0
- }
- }
- }
- }
- new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, regParamL1Fun, $(tol))
- }
-
- val initialCoefficientsWithIntercept = Vectors.zeros(numClasses * numFeaturesPlusIntercept)
-
- if ($(fitIntercept)) {
- /*
- For multinomial logistic regression, when we initialize the coefficients as zeros,
- it will converge faster if we initialize the intercepts such that
- it follows the distribution of the labels.
- {{{
- P(1) = \exp(b_1) / Z
- ...
- P(K) = \exp(b_K) / Z
- where Z = \sum_{k=1}^{K} \exp(b_k)
- }}}
- Since this doesn't have a unique solution, one of the solutions that satisfies the
- above equations is
- {{{
- \exp(b_k) = count_k * \exp(\lambda)
- b_k = \log(count_k) * \lambda
- }}}
- \lambda is a free parameter, so choose the phase \lambda such that the
- mean is centered. This yields
- {{{
- b_k = \log(count_k)
- b_k' = b_k - \mean(b_k)
- }}}
- */
- val rawIntercepts = histogram.map(c => math.log(c + 1)) // add 1 for smoothing
- val rawMean = rawIntercepts.sum / rawIntercepts.length
- rawIntercepts.indices.foreach { i =>
- initialCoefficientsWithIntercept.toArray(i * numFeaturesPlusIntercept + numFeatures) =
- rawIntercepts(i) - rawMean
- }
- }
-
- val states = optimizer.iterations(new CachedDiffFunction(costFun),
- initialCoefficientsWithIntercept.asBreeze.toDenseVector)
-
- /*
- Note that in Multinomial Logistic Regression, the objective history
- (loss + regularization) is log-likelihood which is invariant under feature
- standardization. As a result, the objective history from optimizer is the same as the
- one in the original space.
- */
- val arrayBuilder = mutable.ArrayBuilder.make[Double]
- var state: optimizer.State = null
- while (states.hasNext) {
- state = states.next()
- arrayBuilder += state.adjustedValue
- }
-
- if (state == null) {
- val msg = s"${optimizer.getClass.getName} failed."
- logError(msg)
- throw new SparkException(msg)
- }
- bcFeaturesStd.destroy(blocking = false)
-
- /*
- The coefficients are trained in the scaled space; we're converting them back to
- the original space.
- Note that the intercept in scaled space and original space is the same;
- as a result, no scaling is needed.
- */
- val rawCoefficients = state.x.toArray
- val interceptsArray: Array[Double] = if ($(fitIntercept)) {
- Array.tabulate(numClasses) { i =>
- val coefIndex = (i + 1) * numFeaturesPlusIntercept - 1
- rawCoefficients(coefIndex)
- }
- } else {
- Array.empty
- }
-
- val coefficientArray: Array[Double] = Array.tabulate(numClasses * numFeatures) { i =>
- // flatIndex will loop though rawCoefficients, and skip the intercept terms.
- val flatIndex = if ($(fitIntercept)) i + i / numFeatures else i
- val featureIndex = i % numFeatures
- if (featuresStd(featureIndex) != 0.0) {
- rawCoefficients(flatIndex) / featuresStd(featureIndex)
- } else {
- 0.0
- }
- }
- val coefficientMatrix =
- new DenseMatrix(numClasses, numFeatures, coefficientArray, isTransposed = true)
-
- /*
- When no regularization is applied, the coefficients lack identifiability because
- we do not use a pivot class. We can add any constant value to the coefficients and
- get the same likelihood. So here, we choose the mean centered coefficients for
- reproducibility. This method follows the approach in glmnet, described here:
-
- Friedman, et al. "Regularization Paths for Generalized Linear Models via
- Coordinate Descent," https://core.ac.uk/download/files/153/6287975.pdf
- */
- if ($(regParam) == 0.0) {
- val coefficientMean = coefficientMatrix.values.sum / (numClasses * numFeatures)
- coefficientMatrix.update(_ - coefficientMean)
- }
- /*
- The intercepts are never regularized, so we always center the mean.
- */
- val interceptVector = if (interceptsArray.nonEmpty) {
- val interceptMean = interceptsArray.sum / numClasses
- interceptsArray.indices.foreach { i => interceptsArray(i) -= interceptMean }
- Vectors.dense(interceptsArray)
- } else {
- Vectors.sparse(numClasses, Seq())
- }
-
- (coefficientMatrix, interceptVector, arrayBuilder.result())
- }
- }
-
- if (handlePersistence) instances.unpersist()
-
- val model = copyValues(
- new MultinomialLogisticRegressionModel(uid, coefficients, intercepts, numClasses))
- instr.logSuccess(model)
- model
- }
-
- @Since("2.1.0")
- override def copy(extra: ParamMap): MultinomialLogisticRegression = defaultCopy(extra)
-}
-
-@Since("2.1.0")
-object MultinomialLogisticRegression extends DefaultParamsReadable[MultinomialLogisticRegression] {
-
- @Since("2.1.0")
- override def load(path: String): MultinomialLogisticRegression = super.load(path)
-}
-
-/**
- * :: Experimental ::
- * Model produced by [[MultinomialLogisticRegression]].
- */
-@Since("2.1.0")
-@Experimental
-class MultinomialLogisticRegressionModel private[spark] (
- @Since("2.1.0") override val uid: String,
- @Since("2.1.0") val coefficients: Matrix,
- @Since("2.1.0") val intercepts: Vector,
- @Since("2.1.0") val numClasses: Int)
- extends ProbabilisticClassificationModel[Vector, MultinomialLogisticRegressionModel]
- with MultinomialLogisticRegressionParams with MLWritable {
-
- @Since("2.1.0")
- override def setThresholds(value: Array[Double]): this.type = super.setThresholds(value)
-
- @Since("2.1.0")
- override def getThresholds: Array[Double] = super.getThresholds
-
- @Since("2.1.0")
- override val numFeatures: Int = coefficients.numCols
-
- /** Margin (rawPrediction) for each class label. */
- private val margins: Vector => Vector = (features) => {
- val m = intercepts.toDense.copy
- BLAS.gemv(1.0, coefficients, features, 1.0, m)
- m
- }
-
- /** Score (probability) for each class label. */
- private val scores: Vector => Vector = (features) => {
- val m = margins(features)
- val maxMarginIndex = m.argmax
- val marginArray = m.toArray
- val maxMargin = marginArray(maxMarginIndex)
-
- // adjust margins for overflow
- val sum = {
- var temp = 0.0
- var k = 0
- while (k < numClasses) {
- marginArray(k) = if (maxMargin > 0) {
- math.exp(marginArray(k) - maxMargin)
- } else {
- math.exp(marginArray(k))
- }
- temp += marginArray(k)
- k += 1
- }
- temp
- }
-
- val scores = Vectors.dense(marginArray)
- BLAS.scal(1 / sum, scores)
- scores
- }
-
- /**
- * Predict label for the given feature vector.
- * The behavior of this can be adjusted using [[thresholds]].
- */
- override protected def predict(features: Vector): Double = {
- if (isDefined(thresholds)) {
- val thresholds: Array[Double] = getThresholds
- val probabilities = scores(features).toArray
- var argMax = 0
- var max = Double.NegativeInfinity
- var i = 0
- while (i < numClasses) {
- if (thresholds(i) == 0.0) {
- max = Double.PositiveInfinity
- argMax = i
- } else {
- val scaled = probabilities(i) / thresholds(i)
- if (scaled > max) {
- max = scaled
- argMax = i
- }
- }
- i += 1
- }
- argMax
- } else {
- scores(features).argmax
- }
- }
-
- override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = {
- rawPrediction match {
- case dv: DenseVector =>
- val size = dv.size
- val values = dv.values
-
- // get the maximum margin
- val maxMarginIndex = rawPrediction.argmax
- val maxMargin = rawPrediction(maxMarginIndex)
-
- if (maxMargin == Double.PositiveInfinity) {
- var k = 0
- while (k < size) {
- values(k) = if (k == maxMarginIndex) 1.0 else 0.0
- k += 1
- }
- } else {
- val sum = {
- var temp = 0.0
- var k = 0
- while (k < numClasses) {
- values(k) = if (maxMargin > 0) {
- math.exp(values(k) - maxMargin)
- } else {
- math.exp(values(k))
- }
- temp += values(k)
- k += 1
- }
- temp
- }
- BLAS.scal(1 / sum, dv)
- }
- dv
- case sv: SparseVector =>
- throw new RuntimeException("Unexpected error in MultinomialLogisticRegressionModel:" +
- " raw2probabilitiesInPlace encountered SparseVector")
- }
- }
-
- override protected def predictRaw(features: Vector): Vector = margins(features)
-
- @Since("2.1.0")
- override def copy(extra: ParamMap): MultinomialLogisticRegressionModel = {
- val newModel =
- copyValues(
- new MultinomialLogisticRegressionModel(uid, coefficients, intercepts, numClasses), extra)
- newModel.setParent(parent)
- }
-
- /**
- * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance.
- *
- * This does not save the [[parent]] currently.
- */
- @Since("2.1.0")
- override def write: MLWriter =
- new MultinomialLogisticRegressionModel.MultinomialLogisticRegressionModelWriter(this)
-}
-
-
-@Since("2.1.0")
-object MultinomialLogisticRegressionModel extends MLReadable[MultinomialLogisticRegressionModel] {
-
- @Since("2.1.0")
- override def read: MLReader[MultinomialLogisticRegressionModel] =
- new MultinomialLogisticRegressionModelReader
-
- @Since("2.1.0")
- override def load(path: String): MultinomialLogisticRegressionModel = super.load(path)
-
- /** [[MLWriter]] instance for [[MultinomialLogisticRegressionModel]] */
- private[MultinomialLogisticRegressionModel]
- class MultinomialLogisticRegressionModelWriter(instance: MultinomialLogisticRegressionModel)
- extends MLWriter with Logging {
-
- private case class Data(
- numClasses: Int,
- numFeatures: Int,
- intercepts: Vector,
- coefficients: Matrix)
-
- override protected def saveImpl(path: String): Unit = {
- // Save metadata and Params
- DefaultParamsWriter.saveMetadata(instance, path, sc)
- // Save model data: numClasses, numFeatures, intercept, coefficients
- val data = Data(instance.numClasses, instance.numFeatures, instance.intercepts,
- instance.coefficients)
- val dataPath = new Path(path, "data").toString
- sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath)
- }
- }
-
- private class MultinomialLogisticRegressionModelReader
- extends MLReader[MultinomialLogisticRegressionModel] {
-
- /** Checked against metadata when loading model */
- private val className = classOf[MultinomialLogisticRegressionModel].getName
-
- override def load(path: String): MultinomialLogisticRegressionModel = {
- val metadata = DefaultParamsReader.loadMetadata(path, sc, className)
-
- val dataPath = new Path(path, "data").toString
- val data = sqlContext.read.format("parquet").load(dataPath)
- .select("numClasses", "numFeatures", "intercepts", "coefficients").head()
- val numClasses = data.getAs[Int](data.fieldIndex("numClasses"))
- val intercepts = data.getAs[Vector](data.fieldIndex("intercepts"))
- val coefficients = data.getAs[Matrix](data.fieldIndex("coefficients"))
- val model =
- new MultinomialLogisticRegressionModel(metadata.uid, coefficients, intercepts, numClasses)
-
- DefaultParamsReader.getAndSetParams(model, metadata)
- model
- }
- }
-}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala
index 19df8f7edd..1b6e77542c 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala
@@ -201,11 +201,25 @@ abstract class ProbabilisticClassificationModel[
probability.argmax
} else {
val thresholds: Array[Double] = getThresholds
- val scaledProbability: Array[Double] =
- probability.toArray.zip(thresholds).map { case (p, t) =>
- if (t == 0.0) Double.PositiveInfinity else p / t
+ val probabilities = probability.toArray
+ var argMax = 0
+ var max = Double.NegativeInfinity
+ var i = 0
+ val probabilitySize = probability.size
+ while (i < probabilitySize) {
+ if (thresholds(i) == 0.0) {
+ max = Double.PositiveInfinity
+ argMax = i
+ } else {
+ val scaled = probabilities(i) / thresholds(i)
+ if (scaled > max) {
+ max = scaled
+ argMax = i
+ }
}
- Vectors.dense(scaledProbability).argmax
+ i += 1
+ }
+ argMax
}
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
index e4cbf5acbc..d851b98334 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
@@ -19,6 +19,7 @@ package org.apache.spark.mllib.classification
import org.apache.spark.SparkContext
import org.apache.spark.annotation.Since
+import org.apache.spark.ml.linalg.DenseMatrix
import org.apache.spark.ml.util.Identifiable
import org.apache.spark.mllib.classification.impl.GLMClassificationModel
import org.apache.spark.mllib.linalg.{DenseVector, Vector, Vectors}
@@ -430,8 +431,9 @@ class LogisticRegressionWithLBFGS
lr.setStandardization(useFeatureScaling)
if (userSuppliedWeights) {
val uid = Identifiable.randomUID("logreg-static")
- lr.setInitialModel(new org.apache.spark.ml.classification.LogisticRegressionModel(
- uid, initialWeights.asML, 1.0))
+ lr.setInitialModel(new org.apache.spark.ml.classification.LogisticRegressionModel(uid,
+ new DenseMatrix(1, initialWeights.size, initialWeights.toArray),
+ Vectors.dense(1.0).asML, 2, false))
}
lr.setFitIntercept(addIntercept)
lr.setMaxIter(optimizer.getNumIterations())
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 a1b48539c4..2623759f24 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
@@ -22,28 +22,49 @@ import scala.language.existentials
import scala.util.Random
import scala.util.control.Breaks._
-import org.apache.spark.SparkFunSuite
+import org.apache.spark.{SparkException, SparkFunSuite}
+import org.apache.spark.ml.attribute.NominalAttribute
import org.apache.spark.ml.classification.LogisticRegressionSuite._
-import org.apache.spark.ml.feature.{Instance, LabeledPoint}
-import org.apache.spark.ml.linalg.{Vector, Vectors}
+import org.apache.spark.ml.feature.LabeledPoint
+import org.apache.spark.ml.linalg.{DenseMatrix, Matrices, SparseMatrix, SparseVector, Vector, Vectors}
import org.apache.spark.ml.param.ParamsSuite
import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils}
import org.apache.spark.ml.util.TestingUtils._
import org.apache.spark.mllib.util.MLlibTestSparkContext
-import org.apache.spark.sql.{DataFrame, Dataset, Row}
+import org.apache.spark.sql.{Dataset, Row}
import org.apache.spark.sql.functions.lit
class LogisticRegressionSuite
extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
- @transient var dataset: Dataset[_] = _
- @transient var binaryDataset: DataFrame = _
+ @transient var smallBinaryDataset: Dataset[_] = _
+ @transient var smallMultinomialDataset: Dataset[_] = _
+ @transient var binaryDataset: Dataset[_] = _
+ @transient var multinomialDataset: Dataset[_] = _
private val eps: Double = 1e-5
override def beforeAll(): Unit = {
super.beforeAll()
- dataset = spark.createDataFrame(generateLogisticInput(1.0, 1.0, nPoints = 100, seed = 42))
+ smallBinaryDataset =
+ spark.createDataFrame(generateLogisticInput(1.0, 1.0, nPoints = 100, seed = 42))
+
+ smallMultinomialDataset = {
+ val nPoints = 100
+ val coefficients = Array(
+ -0.57997, 0.912083, -0.371077,
+ -0.16624, -0.84355, -0.048509)
+
+ val xMean = Array(5.843, 3.057)
+ val xVariance = Array(0.6856, 0.1899)
+
+ val testData = generateMultinomialLogisticInput(
+ coefficients, xMean, xVariance, addIntercept = true, nPoints, 42)
+
+ val df = spark.createDataFrame(sc.parallelize(testData, 4))
+ df.cache()
+ df
+ }
binaryDataset = {
val nPoints = 10000
@@ -57,6 +78,23 @@ class LogisticRegressionSuite
spark.createDataFrame(sc.parallelize(testData, 4))
}
+
+ multinomialDataset = {
+ val nPoints = 10000
+ val coefficients = Array(
+ -0.57997, 0.912083, -0.371077, -0.819866, 2.688191,
+ -0.16624, -0.84355, -0.048509, -0.301789, 4.170682)
+
+ val xMean = Array(5.843, 3.057, 3.758, 1.199)
+ val xVariance = Array(0.6856, 0.1899, 3.116, 0.581)
+
+ val testData = generateMultinomialLogisticInput(
+ coefficients, xMean, xVariance, addIntercept = true, nPoints, 42)
+
+ val df = spark.createDataFrame(sc.parallelize(testData, 4))
+ df.cache()
+ df
+ }
}
/**
@@ -67,6 +105,9 @@ class LogisticRegressionSuite
binaryDataset.rdd.map { case Row(label: Double, features: Vector) =>
label + "," + features.toArray.mkString(",")
}.repartition(1).saveAsTextFile("target/tmp/LogisticRegressionSuite/binaryDataset")
+ multinomialDataset.rdd.map { case Row(label: Double, features: Vector) =>
+ label + "," + features.toArray.mkString(",")
+ }.repartition(1).saveAsTextFile("target/tmp/LogisticRegressionSuite/multinomialDataset")
}
test("params") {
@@ -82,11 +123,12 @@ class LogisticRegressionSuite
assert(lr.getPredictionCol === "prediction")
assert(lr.getRawPredictionCol === "rawPrediction")
assert(lr.getProbabilityCol === "probability")
+ assert(lr.getFamily === "auto")
assert(!lr.isDefined(lr.weightCol))
assert(lr.getFitIntercept)
assert(lr.getStandardization)
- val model = lr.fit(dataset)
- model.transform(dataset)
+ val model = lr.fit(smallBinaryDataset)
+ model.transform(smallBinaryDataset)
.select("label", "probability", "prediction", "rawPrediction")
.collect()
assert(model.getThreshold === 0.5)
@@ -100,17 +142,17 @@ class LogisticRegressionSuite
test("empty probabilityCol") {
val lr = new LogisticRegression().setProbabilityCol("")
- val model = lr.fit(dataset)
+ val model = lr.fit(smallBinaryDataset)
assert(model.hasSummary)
// Validate that we re-insert a probability column for evaluation
val fieldNames = model.summary.predictions.schema.fieldNames
- assert(dataset.schema.fieldNames.toSet.subsetOf(
+ assert(smallBinaryDataset.schema.fieldNames.toSet.subsetOf(
fieldNames.toSet))
assert(fieldNames.exists(s => s.startsWith("probability_")))
}
test("setThreshold, getThreshold") {
- val lr = new LogisticRegression
+ val lr = new LogisticRegression().setFamily("binomial")
// default
assert(lr.getThreshold === 0.5, "LogisticRegression.threshold should default to 0.5")
withClue("LogisticRegression should not have thresholds set by default.") {
@@ -127,7 +169,7 @@ class LogisticRegressionSuite
lr.setThreshold(0.5)
assert(lr.getThresholds === Array(0.5, 0.5))
// Set via thresholds
- val lr2 = new LogisticRegression
+ val lr2 = new LogisticRegression().setFamily("binomial")
lr2.setThresholds(Array(0.3, 0.7))
val expectedThreshold = 1.0 / (1.0 + 0.3 / 0.7)
assert(lr2.getThreshold ~== expectedThreshold relTol 1E-7)
@@ -141,19 +183,72 @@ class LogisticRegressionSuite
// thresholds and threshold must be consistent: values
withClue("fit with ParamMap should throw error if threshold, thresholds do not match.") {
intercept[IllegalArgumentException] {
- val lr2model = lr2.fit(dataset,
+ val lr2model = lr2.fit(smallBinaryDataset,
lr2.thresholds -> Array(0.3, 0.7), lr2.threshold -> (expectedThreshold / 2.0))
lr2model.getThreshold
}
}
}
+ test("thresholds prediction") {
+ val blr = new LogisticRegression().setFamily("binomial")
+ val binaryModel = blr.fit(smallBinaryDataset)
+
+ binaryModel.setThreshold(1.0)
+ val binaryZeroPredictions =
+ binaryModel.transform(smallBinaryDataset).select("prediction").collect()
+ assert(binaryZeroPredictions.forall(_.getDouble(0) === 0.0))
+
+ binaryModel.setThreshold(0.0)
+ val binaryOnePredictions =
+ binaryModel.transform(smallBinaryDataset).select("prediction").collect()
+ assert(binaryOnePredictions.forall(_.getDouble(0) === 1.0))
+
+
+ val mlr = new LogisticRegression().setFamily("multinomial")
+ val model = mlr.fit(smallMultinomialDataset)
+ val basePredictions = model.transform(smallMultinomialDataset).select("prediction").collect()
+
+ // should predict all zeros
+ model.setThresholds(Array(1, 1000, 1000))
+ val zeroPredictions = model.transform(smallMultinomialDataset).select("prediction").collect()
+ assert(zeroPredictions.forall(_.getDouble(0) === 0.0))
+
+ // should predict all ones
+ model.setThresholds(Array(1000, 1, 1000))
+ val onePredictions = model.transform(smallMultinomialDataset).select("prediction").collect()
+ assert(onePredictions.forall(_.getDouble(0) === 1.0))
+
+ // should predict all twos
+ model.setThresholds(Array(1000, 1000, 1))
+ val twoPredictions = model.transform(smallMultinomialDataset).select("prediction").collect()
+ assert(twoPredictions.forall(_.getDouble(0) === 2.0))
+
+ // constant threshold scaling is the same as no thresholds
+ model.setThresholds(Array(1000, 1000, 1000))
+ val scaledPredictions = model.transform(smallMultinomialDataset).select("prediction").collect()
+ assert(scaledPredictions.zip(basePredictions).forall { case (scaled, base) =>
+ scaled.getDouble(0) === base.getDouble(0)
+ })
+
+ // force it to use the predict method
+ model.setRawPredictionCol("").setProbabilityCol("").setThresholds(Array(0, 1, 1))
+ val predictionsWithPredict =
+ model.transform(smallMultinomialDataset).select("prediction").collect()
+ assert(predictionsWithPredict.forall(_.getDouble(0) === 0.0))
+ }
+
test("logistic regression doesn't fit intercept when fitIntercept is off") {
- val lr = new LogisticRegression
+ val lr = new LogisticRegression().setFamily("binomial")
lr.setFitIntercept(false)
- val model = lr.fit(dataset)
+ val model = lr.fit(smallBinaryDataset)
assert(model.intercept === 0.0)
+ val mlr = new LogisticRegression().setFamily("multinomial")
+ mlr.setFitIntercept(false)
+ val mlrModel = mlr.fit(smallMultinomialDataset)
+ assert(mlrModel.interceptVector === Vectors.sparse(3, Seq()))
+
// copied model must have the same parent.
MLTestingUtils.checkCopy(model)
}
@@ -165,7 +260,7 @@ class LogisticRegressionSuite
.setRegParam(1.0)
.setThreshold(0.6)
.setProbabilityCol("myProbability")
- val model = lr.fit(dataset)
+ val model = lr.fit(smallBinaryDataset)
val parent = model.parent.asInstanceOf[LogisticRegression]
assert(parent.getMaxIter === 10)
assert(parent.getRegParam === 1.0)
@@ -174,16 +269,16 @@ class LogisticRegressionSuite
// Modify model params, and check that the params worked.
model.setThreshold(1.0)
- val predAllZero = model.transform(dataset)
+ val predAllZero = model.transform(smallBinaryDataset)
.select("prediction", "myProbability")
.collect()
.map { case Row(pred: Double, prob: Vector) => pred }
assert(predAllZero.forall(_ === 0),
s"With threshold=1.0, expected predictions to be all 0, but only" +
- s" ${predAllZero.count(_ === 0)} of ${dataset.count()} were 0.")
+ s" ${predAllZero.count(_ === 0)} of ${smallBinaryDataset.count()} were 0.")
// Call transform with params, and check that the params worked.
val predNotAllZero =
- model.transform(dataset, model.threshold -> 0.0,
+ model.transform(smallBinaryDataset, model.threshold -> 0.0,
model.probabilityCol -> "myProb")
.select("prediction", "myProb")
.collect()
@@ -192,7 +287,7 @@ class LogisticRegressionSuite
// Call fit() with new params, and check as many params as we can.
lr.setThresholds(Array(0.6, 0.4))
- val model2 = lr.fit(dataset, lr.maxIter -> 5, lr.regParam -> 0.1,
+ val model2 = lr.fit(smallBinaryDataset, lr.maxIter -> 5, lr.regParam -> 0.1,
lr.probabilityCol -> "theProb")
val parent2 = model2.parent.asInstanceOf[LogisticRegression]
assert(parent2.getMaxIter === 5)
@@ -202,17 +297,82 @@ class LogisticRegressionSuite
assert(model2.getProbabilityCol === "theProb")
}
- test("logistic regression: Predictor, Classifier methods") {
- val spark = this.spark
- val lr = new LogisticRegression
+ test("multinomial logistic regression: Predictor, Classifier methods") {
+ val sqlContext = smallMultinomialDataset.sqlContext
+ import sqlContext.implicits._
+ val mlr = new LogisticRegression().setFamily("multinomial")
+
+ val model = mlr.fit(smallMultinomialDataset)
+ assert(model.numClasses === 3)
+ val numFeatures = smallMultinomialDataset.select("features").first().getAs[Vector](0).size
+ assert(model.numFeatures === numFeatures)
+
+ val results = model.transform(smallMultinomialDataset)
+ // check that raw prediction is coefficients dot features + intercept
+ results.select("rawPrediction", "features").collect().foreach {
+ case Row(raw: Vector, features: Vector) =>
+ assert(raw.size === 3)
+ val margins = Array.tabulate(3) { k =>
+ var margin = 0.0
+ features.foreachActive { (index, value) =>
+ margin += value * model.coefficientMatrix(k, index)
+ }
+ margin += model.interceptVector(k)
+ margin
+ }
+ assert(raw ~== Vectors.dense(margins) relTol eps)
+ }
+
+ // Compare rawPrediction with probability
+ results.select("rawPrediction", "probability").collect().foreach {
+ case Row(raw: Vector, prob: Vector) =>
+ assert(raw.size === 3)
+ assert(prob.size === 3)
+ val max = raw.toArray.max
+ val subtract = if (max > 0) max else 0.0
+ val sum = raw.toArray.map(x => math.exp(x - subtract)).sum
+ val probFromRaw0 = math.exp(raw(0) - subtract) / sum
+ val probFromRaw1 = math.exp(raw(1) - subtract) / sum
+ assert(prob(0) ~== probFromRaw0 relTol eps)
+ assert(prob(1) ~== probFromRaw1 relTol eps)
+ assert(prob(2) ~== 1.0 - probFromRaw1 - probFromRaw0 relTol eps)
+ }
+
+ // Compare prediction with probability
+ results.select("prediction", "probability").collect().foreach {
+ case Row(pred: Double, prob: Vector) =>
+ val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2
+ assert(pred == predFromProb)
+ }
+
+ // force it to use probability2prediction
+ model.setProbabilityCol("")
+ val resultsUsingProb2Predict =
+ model.transform(smallMultinomialDataset).select("prediction").as[Double].collect()
+ resultsUsingProb2Predict.zip(results.select("prediction").as[Double].collect()).foreach {
+ case (pred1, pred2) => assert(pred1 === pred2)
+ }
+
+ // force it to use predict
+ model.setRawPredictionCol("").setProbabilityCol("")
+ val resultsUsingPredict =
+ model.transform(smallMultinomialDataset).select("prediction").as[Double].collect()
+ resultsUsingPredict.zip(results.select("prediction").as[Double].collect()).foreach {
+ case (pred1, pred2) => assert(pred1 === pred2)
+ }
+ }
+
+ test("binary logistic regression: Predictor, Classifier methods") {
+ val sqlContext = smallBinaryDataset.sqlContext
+ import sqlContext.implicits._
+ val lr = new LogisticRegression().setFamily("binomial")
- val model = lr.fit(dataset)
+ val model = lr.fit(smallBinaryDataset)
assert(model.numClasses === 2)
- val numFeatures = dataset.select("features").first().getAs[Vector](0).size
+ val numFeatures = smallBinaryDataset.select("features").first().getAs[Vector](0).size
assert(model.numFeatures === numFeatures)
- val threshold = model.getThreshold
- val results = model.transform(dataset)
+ val results = model.transform(smallBinaryDataset)
// Compare rawPrediction with probability
results.select("rawPrediction", "probability").collect().foreach {
@@ -230,6 +390,63 @@ class LogisticRegressionSuite
val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2
assert(pred == predFromProb)
}
+
+ // force it to use probability2prediction
+ model.setProbabilityCol("")
+ val resultsUsingProb2Predict =
+ model.transform(smallBinaryDataset).select("prediction").as[Double].collect()
+ resultsUsingProb2Predict.zip(results.select("prediction").as[Double].collect()).foreach {
+ case (pred1, pred2) => assert(pred1 === pred2)
+ }
+
+ // force it to use predict
+ model.setRawPredictionCol("").setProbabilityCol("")
+ val resultsUsingPredict =
+ model.transform(smallBinaryDataset).select("prediction").as[Double].collect()
+ resultsUsingPredict.zip(results.select("prediction").as[Double].collect()).foreach {
+ case (pred1, pred2) => assert(pred1 === pred2)
+ }
+ }
+
+ test("coefficients and intercept methods") {
+ val mlr = new LogisticRegression().setMaxIter(1).setFamily("multinomial")
+ val mlrModel = mlr.fit(smallMultinomialDataset)
+ val thrownCoef = intercept[SparkException] {
+ mlrModel.coefficients
+ }
+ val thrownIntercept = intercept[SparkException] {
+ mlrModel.intercept
+ }
+ assert(thrownCoef.getMessage().contains("use coefficientMatrix instead"))
+ assert(thrownIntercept.getMessage().contains("use interceptVector instead"))
+
+ val blr = new LogisticRegression().setMaxIter(1).setFamily("binomial")
+ val blrModel = blr.fit(smallBinaryDataset)
+ assert(blrModel.coefficients.size === 1)
+ assert(blrModel.intercept !== 0.0)
+ }
+
+ test("overflow prediction for multiclass") {
+ val model = new LogisticRegressionModel("mLogReg",
+ Matrices.dense(3, 2, Array(0.0, 0.0, 0.0, 1.0, 2.0, 3.0)),
+ Vectors.dense(0.0, 0.0, 0.0), 3, true)
+ val overFlowData = spark.createDataFrame(Seq(
+ LabeledPoint(1.0, Vectors.dense(0.0, 1000.0)),
+ LabeledPoint(1.0, Vectors.dense(0.0, -1.0))
+ ))
+ val results = model.transform(overFlowData).select("rawPrediction", "probability").collect()
+
+ // probabilities are correct when margins have to be adjusted
+ val raw1 = results(0).getAs[Vector](0)
+ val prob1 = results(0).getAs[Vector](1)
+ assert(raw1 === Vectors.dense(1000.0, 2000.0, 3000.0))
+ assert(prob1 ~== Vectors.dense(0.0, 0.0, 1.0) absTol eps)
+
+ // probabilities are correct when margins don't have to be adjusted
+ val raw2 = results(1).getAs[Vector](0)
+ val prob2 = results(1).getAs[Vector](1)
+ assert(raw2 === Vectors.dense(-1.0, -2.0, -3.0))
+ assert(prob2 ~== Vectors.dense(0.66524096, 0.24472847, 0.09003057) relTol eps)
}
test("MultiClassSummarizer") {
@@ -427,7 +644,9 @@ class LogisticRegressionSuite
val coefficientsR2 = Vectors.dense(0.0, 0.0, -0.1665453, 0.0)
assert(model2.intercept ~== interceptR2 relTol 1E-2)
- assert(model2.coefficients ~= coefficientsR2 absTol 1E-3)
+ assert(model2.coefficients ~== coefficientsR2 absTol 1E-3)
+ // TODO: move this to a standalone test of compression after SPARK-17471
+ assert(model2.coefficients.isInstanceOf[SparseVector])
}
test("binary logistic regression without intercept with L1 regularization") {
@@ -768,6 +987,7 @@ class LogisticRegressionSuite
assert(model2.coefficients ~= coefficientsTheory absTol 1E-6)
/*
+ TODO: why is this needed? The correctness of L1 regularization is already checked elsewhere
Using the following R code to load the data and train the model using glmnet package.
library("glmnet")
@@ -792,16 +1012,759 @@ class LogisticRegressionSuite
assert(model1.coefficients ~== coefficientsR absTol 1E-6)
}
+ test("multinomial logistic regression with intercept with strong L1 regularization") {
+ val trainer1 = (new LogisticRegression).setFitIntercept(true)
+ .setElasticNetParam(1.0).setRegParam(6.0).setStandardization(true)
+ val trainer2 = (new LogisticRegression).setFitIntercept(true)
+ .setElasticNetParam(1.0).setRegParam(6.0).setStandardization(false)
+
+ val sqlContext = multinomialDataset.sqlContext
+ import sqlContext.implicits._
+ val model1 = trainer1.fit(multinomialDataset)
+ val model2 = trainer2.fit(multinomialDataset)
+
+ val histogram = multinomialDataset.as[LabeledPoint].rdd.map(_.label)
+ .treeAggregate(new MultiClassSummarizer)(
+ seqOp = (c, v) => (c, v) match {
+ case (classSummarizer: MultiClassSummarizer, label: Double) => classSummarizer.add(label)
+ },
+ combOp = (c1, c2) => (c1, c2) match {
+ case (classSummarizer1: MultiClassSummarizer, classSummarizer2: MultiClassSummarizer) =>
+ classSummarizer1.merge(classSummarizer2)
+ }).histogram
+ val numFeatures = multinomialDataset.as[LabeledPoint].first().features.size
+ val numClasses = histogram.length
+
+ /*
+ For multinomial logistic regression with strong L1 regularization, all the coefficients
+ will be zeros. As a result, the intercepts will be proportional to the log counts in the
+ histogram.
+ {{{
+ \exp(b_k) = count_k * \exp(\lambda)
+ b_k = \log(count_k) * \lambda
+ }}}
+ \lambda is a free parameter, so choose the phase \lambda such that the
+ mean is centered. This yields
+ {{{
+ b_k = \log(count_k)
+ b_k' = b_k - \mean(b_k)
+ }}}
+ */
+ val rawInterceptsTheory = histogram.map(c => math.log(c + 1)) // add 1 for smoothing
+ val rawMean = rawInterceptsTheory.sum / rawInterceptsTheory.length
+ val interceptsTheory = Vectors.dense(rawInterceptsTheory.map(_ - rawMean))
+ val coefficientsTheory = new DenseMatrix(numClasses, numFeatures,
+ Array.fill[Double](numClasses * numFeatures)(0.0), isTransposed = true)
+
+ assert(model1.interceptVector ~== interceptsTheory relTol 1E-3)
+ assert(model1.coefficientMatrix ~= coefficientsTheory absTol 1E-6)
+
+ assert(model2.interceptVector ~== interceptsTheory relTol 1E-3)
+ assert(model2.coefficientMatrix ~= coefficientsTheory absTol 1E-6)
+ }
+
+ test("multinomial logistic regression with intercept without regularization") {
+
+ val trainer1 = (new LogisticRegression).setFitIntercept(true)
+ .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true).setMaxIter(100)
+ val trainer2 = (new LogisticRegression).setFitIntercept(true)
+ .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(false)
+
+ val model1 = trainer1.fit(multinomialDataset)
+ val model2 = trainer2.fit(multinomialDataset)
+
+ /*
+ Using 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)
+ > features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+ > coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0, lambda = 0))
+ > coefficients
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ -2.24493379
+ V2 0.25096771
+ V3 -0.03915938
+ V4 0.14766639
+ V5 0.36810817
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ 0.3778931
+ V2 -0.3327489
+ V3 0.8893666
+ V4 -0.2306948
+ V5 -0.4442330
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ 1.86704066
+ V2 0.08178121
+ V3 -0.85020722
+ V4 0.08302840
+ V5 0.07612480
+ */
+
+ val coefficientsR = new DenseMatrix(3, 4, Array(
+ 0.2509677, -0.0391594, 0.1476664, 0.3681082,
+ -0.3327489, 0.8893666, -0.2306948, -0.4442330,
+ 0.0817812, -0.8502072, 0.0830284, 0.0761248), isTransposed = true)
+ val interceptsR = Vectors.dense(-2.2449338, 0.3778931, 1.8670407)
+
+ assert(model1.coefficientMatrix ~== coefficientsR relTol 0.05)
+ assert(model1.coefficientMatrix.toArray.sum ~== 0.0 absTol eps)
+ assert(model1.interceptVector ~== interceptsR relTol 0.05)
+ assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ assert(model2.coefficientMatrix ~== coefficientsR relTol 0.05)
+ assert(model2.coefficientMatrix.toArray.sum ~== 0.0 absTol eps)
+ assert(model2.interceptVector ~== interceptsR relTol 0.05)
+ assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ }
+
+ test("multinomial logistic regression without intercept without regularization") {
+
+ val trainer1 = (new LogisticRegression).setFitIntercept(false)
+ .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true)
+ val trainer2 = (new LogisticRegression).setFitIntercept(false)
+ .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(false)
+
+ val model1 = trainer1.fit(multinomialDataset)
+ val model2 = trainer2.fit(multinomialDataset)
+
+ /*
+ Using 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)
+ features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+ coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0, lambda = 0,
+ intercept=F))
+ > coefficients
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 0.06992464
+ V3 -0.36562784
+ V4 0.12142680
+ V5 0.32052211
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 -0.3036269
+ V3 0.9449630
+ V4 -0.2271038
+ V5 -0.4364839
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 0.2337022
+ V3 -0.5793351
+ V4 0.1056770
+ V5 0.1159618
+ */
+
+ val coefficientsR = new DenseMatrix(3, 4, Array(
+ 0.0699246, -0.3656278, 0.1214268, 0.3205221,
+ -0.3036269, 0.9449630, -0.2271038, -0.4364839,
+ 0.2337022, -0.5793351, 0.1056770, 0.1159618), isTransposed = true)
+
+ assert(model1.coefficientMatrix ~== coefficientsR relTol 0.05)
+ assert(model1.coefficientMatrix.toArray.sum ~== 0.0 absTol eps)
+ assert(model1.interceptVector.toArray === Array.fill(3)(0.0))
+ assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ assert(model2.coefficientMatrix ~== coefficientsR relTol 0.05)
+ assert(model2.coefficientMatrix.toArray.sum ~== 0.0 absTol eps)
+ assert(model2.interceptVector.toArray === Array.fill(3)(0.0))
+ assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ }
+
+ test("multinomial logistic regression with intercept with L1 regularization") {
+
+ // use tighter constraints because OWL-QN solver takes longer to converge
+ val trainer1 = (new LogisticRegression).setFitIntercept(true)
+ .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(true)
+ .setMaxIter(300).setTol(1e-10)
+ val trainer2 = (new LogisticRegression).setFitIntercept(true)
+ .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(false)
+ .setMaxIter(300).setTol(1e-10)
+
+ val model1 = trainer1.fit(multinomialDataset)
+ val model2 = trainer2.fit(multinomialDataset)
+
+ /*
+ 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)
+ features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+ coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 1,
+ lambda = 0.05, standardization=T))
+ coefficients = coef(glmnet(features, label, family="multinomial", alpha = 1, lambda = 0.05,
+ standardization=F))
+ > coefficientsStd
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ -0.68988825
+ V2 .
+ V3 .
+ V4 .
+ V5 0.09404023
+
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ -0.2303499
+ V2 -0.1232443
+ V3 0.3258380
+ V4 -0.1564688
+ V5 -0.2053965
+
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ 0.9202381
+ V2 .
+ V3 -0.4803856
+ V4 .
+ V5 .
+
+ > coefficients
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ -0.44893320
+ V2 .
+ V3 .
+ V4 0.01933812
+ V5 0.03666044
+
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ 0.7376760
+ V2 -0.0577182
+ V3 .
+ V4 -0.2081718
+ V5 -0.1304592
+
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ -0.2887428
+ V2 .
+ V3 .
+ V4 .
+ V5 .
+ */
+
+ val coefficientsRStd = new DenseMatrix(3, 4, Array(
+ 0.0, 0.0, 0.0, 0.09404023,
+ -0.1232443, 0.3258380, -0.1564688, -0.2053965,
+ 0.0, -0.4803856, 0.0, 0.0), isTransposed = true)
+ val interceptsRStd = Vectors.dense(-0.68988825, -0.2303499, 0.9202381)
+
+ val coefficientsR = new DenseMatrix(3, 4, Array(
+ 0.0, 0.0, 0.01933812, 0.03666044,
+ -0.0577182, 0.0, -0.2081718, -0.1304592,
+ 0.0, 0.0, 0.0, 0.0), isTransposed = true)
+ val interceptsR = Vectors.dense(-0.44893320, 0.7376760, -0.2887428)
+
+ assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.02)
+ assert(model1.interceptVector ~== interceptsRStd relTol 0.1)
+ assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ assert(model2.coefficientMatrix ~== coefficientsR absTol 0.02)
+ assert(model2.interceptVector ~== interceptsR relTol 0.1)
+ assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ }
+
+ test("multinomial logistic regression without intercept with L1 regularization") {
+ val trainer1 = (new LogisticRegression).setFitIntercept(false)
+ .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(true)
+ val trainer2 = (new LogisticRegression).setFitIntercept(false)
+ .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(false)
+
+ val model1 = trainer1.fit(multinomialDataset)
+ val model2 = trainer2.fit(multinomialDataset)
+ /*
+ 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)
+ features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+ coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 1,
+ lambda = 0.05, intercept=F, standardization=T))
+ coefficients = coef(glmnet(features, label, family="multinomial", alpha = 1, lambda = 0.05,
+ intercept=F, standardization=F))
+ > coefficientsStd
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 .
+ V3 .
+ V4 .
+ V5 0.01525105
+
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 -0.1502410
+ V3 0.5134658
+ V4 -0.1601146
+ V5 -0.2500232
+
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 0.003301875
+ V3 .
+ V4 .
+ V5 .
+
+ > coefficients
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 .
+ V3 .
+ V4 .
+ V5 .
+
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 .
+ V3 0.1943624
+ V4 -0.1902577
+ V5 -0.1028789
+
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 .
+ V3 .
+ V4 .
+ V5 .
+ */
+
+ val coefficientsRStd = new DenseMatrix(3, 4, Array(
+ 0.0, 0.0, 0.0, 0.01525105,
+ -0.1502410, 0.5134658, -0.1601146, -0.2500232,
+ 0.003301875, 0.0, 0.0, 0.0), isTransposed = true)
+
+ val coefficientsR = new DenseMatrix(3, 4, Array(
+ 0.0, 0.0, 0.0, 0.0,
+ 0.0, 0.1943624, -0.1902577, -0.1028789,
+ 0.0, 0.0, 0.0, 0.0), isTransposed = true)
+
+ assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01)
+ assert(model1.interceptVector.toArray === Array.fill(3)(0.0))
+ assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ assert(model2.coefficientMatrix ~== coefficientsR absTol 0.01)
+ assert(model2.interceptVector.toArray === Array.fill(3)(0.0))
+ assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ }
+
+ test("multinomial logistic regression with intercept with L2 regularization") {
+ val trainer1 = (new LogisticRegression).setFitIntercept(true)
+ .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(true)
+ val trainer2 = (new LogisticRegression).setFitIntercept(true)
+ .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(false)
+
+ val model1 = trainer1.fit(multinomialDataset)
+ val model2 = trainer2.fit(multinomialDataset)
+ /*
+ 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)
+ features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+ coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 0,
+ lambda = 0.1, intercept=T, standardization=T))
+ coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0,
+ lambda = 0.1, intercept=T, standardization=F))
+ > coefficientsStd
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ -1.70040424
+ V2 0.17576070
+ V3 0.01527894
+ V4 0.10216108
+ V5 0.26099531
+
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ 0.2438590
+ V2 -0.2238875
+ V3 0.5967610
+ V4 -0.1555496
+ V5 -0.3010479
+
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ 1.45654525
+ V2 0.04812679
+ V3 -0.61203992
+ V4 0.05338850
+ V5 0.04005258
+
+ > coefficients
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ -1.65488543
+ V2 0.15715048
+ V3 0.01992903
+ V4 0.12428858
+ V5 0.22130317
+
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ 1.1297533
+ V2 -0.1974768
+ V3 0.2776373
+ V4 -0.1869445
+ V5 -0.2510320
+
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ 0.52513212
+ V2 0.04032627
+ V3 -0.29756637
+ V4 0.06265594
+ V5 0.02972883
+ */
+
+ val coefficientsRStd = new DenseMatrix(3, 4, Array(
+ 0.17576070, 0.01527894, 0.10216108, 0.26099531,
+ -0.2238875, 0.5967610, -0.1555496, -0.3010479,
+ 0.04812679, -0.61203992, 0.05338850, 0.04005258), isTransposed = true)
+ val interceptsRStd = Vectors.dense(-1.70040424, 0.2438590, 1.45654525)
+
+ val coefficientsR = new DenseMatrix(3, 4, Array(
+ 0.15715048, 0.01992903, 0.12428858, 0.22130317,
+ -0.1974768, 0.2776373, -0.1869445, -0.2510320,
+ 0.04032627, -0.29756637, 0.06265594, 0.02972883), isTransposed = true)
+ val interceptsR = Vectors.dense(-1.65488543, 1.1297533, 0.52513212)
+
+ assert(model1.coefficientMatrix ~== coefficientsRStd relTol 0.05)
+ assert(model1.interceptVector ~== interceptsRStd relTol 0.05)
+ assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ assert(model2.coefficientMatrix ~== coefficientsR relTol 0.05)
+ assert(model2.interceptVector ~== interceptsR relTol 0.05)
+ assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ }
+
+ test("multinomial logistic regression without intercept with L2 regularization") {
+ val trainer1 = (new LogisticRegression).setFitIntercept(false)
+ .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(true)
+ val trainer2 = (new LogisticRegression).setFitIntercept(false)
+ .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(false)
+
+ val model1 = trainer1.fit(multinomialDataset)
+ val model2 = trainer2.fit(multinomialDataset)
+ /*
+ 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)
+ features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+ coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 0,
+ lambda = 0.1, intercept=F, standardization=T))
+ coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0,
+ lambda = 0.1, intercept=F, standardization=F))
+ > coefficientsStd
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 0.03904171
+ V3 -0.23354322
+ V4 0.08288096
+ V5 0.22706393
+
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 -0.2061848
+ V3 0.6341398
+ V4 -0.1530059
+ V5 -0.2958455
+
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 0.16714312
+ V3 -0.40059658
+ V4 0.07012496
+ V5 0.06878158
+ > coefficients
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 -0.005704542
+ V3 -0.144466409
+ V4 0.092080736
+ V5 0.182927657
+
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 -0.08469036
+ V3 0.38996748
+ V4 -0.16468436
+ V5 -0.22522976
+
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 0.09039490
+ V3 -0.24550107
+ V4 0.07260362
+ V5 0.04230210
+ */
+ val coefficientsRStd = new DenseMatrix(3, 4, Array(
+ 0.03904171, -0.23354322, 0.08288096, 0.2270639,
+ -0.2061848, 0.6341398, -0.1530059, -0.2958455,
+ 0.16714312, -0.40059658, 0.07012496, 0.06878158), isTransposed = true)
+
+ val coefficientsR = new DenseMatrix(3, 4, Array(
+ -0.005704542, -0.144466409, 0.092080736, 0.182927657,
+ -0.08469036, 0.38996748, -0.16468436, -0.22522976,
+ 0.0903949, -0.24550107, 0.07260362, 0.0423021), isTransposed = true)
+
+ assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01)
+ assert(model1.interceptVector.toArray === Array.fill(3)(0.0))
+ assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ assert(model2.coefficientMatrix ~== coefficientsR absTol 0.01)
+ assert(model2.interceptVector.toArray === Array.fill(3)(0.0))
+ assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ }
+
+ test("multinomial logistic regression with intercept with elasticnet regularization") {
+ val trainer1 = (new LogisticRegression).setFitIntercept(true)
+ .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(true)
+ .setMaxIter(300).setTol(1e-10)
+ val trainer2 = (new LogisticRegression).setFitIntercept(true)
+ .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(false)
+ .setMaxIter(300).setTol(1e-10)
+
+ val model1 = trainer1.fit(multinomialDataset)
+ val model2 = trainer2.fit(multinomialDataset)
+ /*
+ 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)
+ features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+ coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 0.5,
+ lambda = 0.1, intercept=T, standardization=T))
+ coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0.5,
+ lambda = 0.1, intercept=T, standardization=F))
+ > coefficientsStd
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ -0.5521819483
+ V2 0.0003092611
+ V3 .
+ V4 .
+ V5 0.0913818490
+
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ -0.27531989
+ V2 -0.09790029
+ V3 0.28502034
+ V4 -0.12416487
+ V5 -0.16513373
+
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ 0.8275018
+ V2 .
+ V3 -0.4044859
+ V4 .
+ V5 .
+
+ > coefficients
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ -0.39876213
+ V2 .
+ V3 .
+ V4 0.02547520
+ V5 0.03893991
+
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ 0.61089869
+ V2 -0.04224269
+ V3 .
+ V4 -0.18923970
+ V5 -0.09104249
+
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ -0.2121366
+ V2 .
+ V3 .
+ V4 .
+ V5 .
+ */
+
+ val coefficientsRStd = new DenseMatrix(3, 4, Array(
+ 0.0003092611, 0.0, 0.0, 0.091381849,
+ -0.09790029, 0.28502034, -0.12416487, -0.16513373,
+ 0.0, -0.4044859, 0.0, 0.0), isTransposed = true)
+ val interceptsRStd = Vectors.dense(-0.5521819483, -0.27531989, 0.8275018)
+
+ val coefficientsR = new DenseMatrix(3, 4, Array(
+ 0.0, 0.0, 0.0254752, 0.03893991,
+ -0.04224269, 0.0, -0.1892397, -0.09104249,
+ 0.0, 0.0, 0.0, 0.0), isTransposed = true)
+ val interceptsR = Vectors.dense(-0.39876213, 0.61089869, -0.2121366)
+
+ assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01)
+ assert(model1.interceptVector ~== interceptsRStd absTol 0.01)
+ assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ assert(model2.coefficientMatrix ~== coefficientsR absTol 0.01)
+ assert(model2.interceptVector ~== interceptsR absTol 0.01)
+ assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ }
+
+ test("multinomial logistic regression without intercept with elasticnet regularization") {
+ val trainer1 = (new LogisticRegression).setFitIntercept(false)
+ .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(true)
+ .setMaxIter(300).setTol(1e-10)
+ val trainer2 = (new LogisticRegression).setFitIntercept(false)
+ .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(false)
+ .setMaxIter(300).setTol(1e-10)
+
+ val model1 = trainer1.fit(multinomialDataset)
+ val model2 = trainer2.fit(multinomialDataset)
+ /*
+ 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)
+ features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+ coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 0.5,
+ lambda = 0.1, intercept=F, standardization=T))
+ coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0.5,
+ lambda = 0.1, intercept=F, standardization=F))
+ > coefficientsStd
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 .
+ V3 .
+ V4 .
+ V5 0.03543706
+
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 -0.1187387
+ V3 0.4025482
+ V4 -0.1270969
+ V5 -0.1918386
+
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 0.00774365
+ V3 .
+ V4 .
+ V5 .
+
+ > coefficients
+ $`0`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 .
+ V3 .
+ V4 .
+ V5 .
+
+ $`1`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 .
+ V3 0.14666497
+ V4 -0.16570638
+ V5 -0.05982875
+
+ $`2`
+ 5 x 1 sparse Matrix of class "dgCMatrix"
+ s0
+ .
+ V2 .
+ V3 .
+ V4 .
+ V5 .
+ */
+ val coefficientsRStd = new DenseMatrix(3, 4, Array(
+ 0.0, 0.0, 0.0, 0.03543706,
+ -0.1187387, 0.4025482, -0.1270969, -0.1918386,
+ 0.0, 0.0, 0.0, 0.00774365), isTransposed = true)
+
+ val coefficientsR = new DenseMatrix(3, 4, Array(
+ 0.0, 0.0, 0.0, 0.0,
+ 0.0, 0.14666497, -0.16570638, -0.05982875,
+ 0.0, 0.0, 0.0, 0.0), isTransposed = true)
+
+ assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01)
+ assert(model1.interceptVector.toArray === Array.fill(3)(0.0))
+ assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ assert(model2.coefficientMatrix ~== coefficientsR absTol 0.01)
+ assert(model2.interceptVector.toArray === Array.fill(3)(0.0))
+ assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps)
+ }
+
test("evaluate on test set") {
+ // TODO: add for multiclass when model summary becomes available
// Evaluate on test set should be same as that of the transformed training data.
val lr = new LogisticRegression()
.setMaxIter(10)
.setRegParam(1.0)
.setThreshold(0.6)
- val model = lr.fit(dataset)
+ val model = lr.fit(smallBinaryDataset)
val summary = model.summary.asInstanceOf[BinaryLogisticRegressionSummary]
- val sameSummary = model.evaluate(dataset).asInstanceOf[BinaryLogisticRegressionSummary]
+ val sameSummary =
+ model.evaluate(smallBinaryDataset).asInstanceOf[BinaryLogisticRegressionSummary]
assert(summary.areaUnderROC === sameSummary.areaUnderROC)
assert(summary.roc.collect() === sameSummary.roc.collect())
assert(summary.pr.collect === sameSummary.pr.collect())
@@ -818,7 +1781,7 @@ class LogisticRegressionSuite
.setMaxIter(10)
.setRegParam(1.0)
.setThreshold(0.6)
- val model = lr.fit(dataset)
+ val model = lr.fit(smallBinaryDataset)
assert(
model.summary
.objectiveHistory
@@ -827,67 +1790,118 @@ class LogisticRegressionSuite
}
- test("binary logistic regression with weighted samples") {
- val (dataset, weightedDataset) = {
- val nPoints = 1000
- val coefficients = Array(-0.57997, 0.912083, -0.371077, -0.819866, 2.688191)
- val xMean = Array(5.843, 3.057, 3.758, 1.199)
- val xVariance = Array(0.6856, 0.1899, 3.116, 0.581)
- val testData =
- generateMultinomialLogisticInput(coefficients, xMean, xVariance, true, nPoints, 42)
-
- // Let's over-sample the positive samples twice.
- val data1 = testData.flatMap { case labeledPoint: LabeledPoint =>
- if (labeledPoint.label == 1.0) {
- Iterator(labeledPoint, labeledPoint)
- } else {
- Iterator(labeledPoint)
- }
- }
+ test("binary logistic regression with weighted data") {
+ val numClasses = 2
+ val numPoints = 40
+ val outlierData = MLTestingUtils.genClassificationInstancesWithWeightedOutliers(spark,
+ numClasses, numPoints)
+ val testData = spark.createDataFrame(Array.tabulate[LabeledPoint](numClasses) { i =>
+ LabeledPoint(i.toDouble, Vectors.dense(i.toDouble))
+ })
+ val lr = new LogisticRegression().setFamily("binomial").setWeightCol("weight")
+ val model = lr.fit(outlierData)
+ val results = model.transform(testData).select("label", "prediction").collect()
+
+ // check that the predictions are the one to one mapping
+ results.foreach { case Row(label: Double, pred: Double) =>
+ assert(label === pred)
+ }
+ val (overSampledData, weightedData) =
+ MLTestingUtils.genEquivalentOversampledAndWeightedInstances(outlierData, "label", "features",
+ 42L)
+ val weightedModel = lr.fit(weightedData)
+ val overSampledModel = lr.setWeightCol("").fit(overSampledData)
+ assert(weightedModel.coefficientMatrix ~== overSampledModel.coefficientMatrix relTol 0.01)
+ }
- val rnd = new Random(8392)
- val data2 = testData.flatMap { case LabeledPoint(label: Double, features: Vector) =>
- if (rnd.nextGaussian() > 0.0) {
- if (label == 1.0) {
- Iterator(
- Instance(label, 1.2, features),
- Instance(label, 0.8, features),
- Instance(0.0, 0.0, features))
- } else {
- Iterator(
- Instance(label, 0.3, features),
- Instance(1.0, 0.0, features),
- Instance(label, 0.1, features),
- Instance(label, 0.6, features))
- }
- } else {
- if (label == 1.0) {
- Iterator(Instance(label, 2.0, features))
- } else {
- Iterator(Instance(label, 1.0, features))
- }
- }
- }
+ test("multinomial logistic regression with weighted data") {
+ val numClasses = 5
+ val numPoints = 40
+ val outlierData = MLTestingUtils.genClassificationInstancesWithWeightedOutliers(spark,
+ numClasses, numPoints)
+ val testData = spark.createDataFrame(Array.tabulate[LabeledPoint](numClasses) { i =>
+ LabeledPoint(i.toDouble, Vectors.dense(i.toDouble))
+ })
+ val mlr = new LogisticRegression().setFamily("multinomial").setWeightCol("weight")
+ val model = mlr.fit(outlierData)
+ val results = model.transform(testData).select("label", "prediction").collect()
+
+ // check that the predictions are the one to one mapping
+ results.foreach { case Row(label: Double, pred: Double) =>
+ assert(label === pred)
+ }
+ val (overSampledData, weightedData) =
+ MLTestingUtils.genEquivalentOversampledAndWeightedInstances(outlierData, "label", "features",
+ 42L)
+ val weightedModel = mlr.fit(weightedData)
+ val overSampledModel = mlr.setWeightCol("").fit(overSampledData)
+ assert(weightedModel.coefficientMatrix ~== overSampledModel.coefficientMatrix relTol 0.01)
+ }
- (spark.createDataFrame(sc.parallelize(data1, 4)),
- spark.createDataFrame(sc.parallelize(data2, 4)))
+ test("set family") {
+ val lr = new LogisticRegression().setMaxIter(1)
+ // don't set anything for binary classification
+ val model1 = lr.fit(binaryDataset)
+ assert(model1.coefficientMatrix.numRows === 1 && model1.coefficientMatrix.numCols === 4)
+ assert(model1.interceptVector.size === 1)
+
+ // set to multinomial for binary classification
+ val model2 = lr.setFamily("multinomial").fit(binaryDataset)
+ assert(model2.coefficientMatrix.numRows === 2 && model2.coefficientMatrix.numCols === 4)
+ assert(model2.interceptVector.size === 2)
+
+ // set to binary for binary classification
+ val model3 = lr.setFamily("binomial").fit(binaryDataset)
+ assert(model3.coefficientMatrix.numRows === 1 && model3.coefficientMatrix.numCols === 4)
+ assert(model3.interceptVector.size === 1)
+
+ // don't set anything for multiclass classification
+ val mlr = new LogisticRegression().setMaxIter(1)
+ val model4 = mlr.fit(multinomialDataset)
+ assert(model4.coefficientMatrix.numRows === 3 && model4.coefficientMatrix.numCols === 4)
+ assert(model4.interceptVector.size === 3)
+
+ // set to binary for multiclass classification
+ mlr.setFamily("binomial")
+ val thrown = intercept[IllegalArgumentException] {
+ mlr.fit(multinomialDataset)
}
+ assert(thrown.getMessage.contains("Binomial family only supports 1 or 2 outcome classes"))
- val trainer1a = (new LogisticRegression).setFitIntercept(true)
- .setRegParam(0.0).setStandardization(true)
- val trainer1b = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight")
- .setRegParam(0.0).setStandardization(true)
- val model1a0 = trainer1a.fit(dataset)
- val model1a1 = trainer1a.fit(weightedDataset)
- val model1b = trainer1b.fit(weightedDataset)
- assert(model1a0.coefficients !~= model1a1.coefficients absTol 1E-3)
- assert(model1a0.intercept !~= model1a1.intercept absTol 1E-3)
- assert(model1a0.coefficients ~== model1b.coefficients absTol 1E-3)
- assert(model1a0.intercept ~== model1b.intercept absTol 1E-3)
+ // set to multinomial for multiclass
+ mlr.setFamily("multinomial")
+ val model5 = mlr.fit(multinomialDataset)
+ assert(model5.coefficientMatrix.numRows === 3 && model5.coefficientMatrix.numCols === 4)
+ assert(model5.interceptVector.size === 3)
}
- test("logistic regression with all labels the same") {
- val sameLabels = dataset
+ test("set initial model") {
+ val lr = new LogisticRegression().setFamily("binomial")
+ val model1 = lr.fit(smallBinaryDataset)
+ val lr2 = new LogisticRegression().setInitialModel(model1).setMaxIter(5).setFamily("binomial")
+ val model2 = lr2.fit(smallBinaryDataset)
+ val predictions1 = model1.transform(smallBinaryDataset).select("prediction").collect()
+ val predictions2 = model2.transform(smallBinaryDataset).select("prediction").collect()
+ predictions1.zip(predictions2).foreach { case (Row(p1: Double), Row(p2: Double)) =>
+ assert(p1 === p2)
+ }
+ assert(model2.summary.totalIterations === 1)
+
+ val lr3 = new LogisticRegression().setFamily("multinomial")
+ val model3 = lr3.fit(smallMultinomialDataset)
+ val lr4 = new LogisticRegression()
+ .setInitialModel(model3).setMaxIter(5).setFamily("multinomial")
+ val model4 = lr4.fit(smallMultinomialDataset)
+ val predictions3 = model3.transform(smallMultinomialDataset).select("prediction").collect()
+ val predictions4 = model4.transform(smallMultinomialDataset).select("prediction").collect()
+ predictions3.zip(predictions4).foreach { case (Row(p1: Double), Row(p2: Double)) =>
+ assert(p1 === p2)
+ }
+ // TODO: check that it converges in a single iteration when model summary is available
+ }
+
+ test("binary logistic regression with all labels the same") {
+ val sameLabels = smallBinaryDataset
.withColumn("zeroLabel", lit(0.0))
.withColumn("oneLabel", lit(1.0))
@@ -895,6 +1909,7 @@ class LogisticRegressionSuite
val lrIntercept = new LogisticRegression()
.setFitIntercept(true)
.setMaxIter(3)
+ .setFamily("binomial")
val allZeroInterceptModel = lrIntercept
.setLabelCol("zeroLabel")
@@ -914,6 +1929,7 @@ class LogisticRegressionSuite
val lrNoIntercept = new LogisticRegression()
.setFitIntercept(false)
.setMaxIter(3)
+ .setFamily("binomial")
val allZeroNoInterceptModel = lrNoIntercept
.setLabelCol("zeroLabel")
@@ -928,6 +1944,102 @@ class LogisticRegressionSuite
assert(allOneNoInterceptModel.summary.totalIterations > 0)
}
+ test("multiclass logistic regression with all labels the same") {
+ val constantData = spark.createDataFrame(Seq(
+ LabeledPoint(4.0, Vectors.dense(0.0)),
+ LabeledPoint(4.0, Vectors.dense(1.0)),
+ LabeledPoint(4.0, Vectors.dense(2.0)))
+ )
+ val mlr = new LogisticRegression().setFamily("multinomial")
+ val model = mlr.fit(constantData)
+ val results = model.transform(constantData)
+ results.select("rawPrediction", "probability", "prediction").collect().foreach {
+ case Row(raw: Vector, prob: Vector, pred: Double) =>
+ assert(raw === Vectors.dense(Array(0.0, 0.0, 0.0, 0.0, Double.PositiveInfinity)))
+ assert(prob === Vectors.dense(Array(0.0, 0.0, 0.0, 0.0, 1.0)))
+ assert(pred === 4.0)
+ }
+
+ // force the model to be trained with only one class
+ val constantZeroData = spark.createDataFrame(Seq(
+ LabeledPoint(0.0, Vectors.dense(0.0)),
+ LabeledPoint(0.0, Vectors.dense(1.0)),
+ LabeledPoint(0.0, Vectors.dense(2.0)))
+ )
+ val modelZeroLabel = mlr.setFitIntercept(false).fit(constantZeroData)
+ val resultsZero = modelZeroLabel.transform(constantZeroData)
+ resultsZero.select("rawPrediction", "probability", "prediction").collect().foreach {
+ case Row(raw: Vector, prob: Vector, pred: Double) =>
+ assert(prob === Vectors.dense(Array(1.0)))
+ assert(pred === 0.0)
+ }
+
+ // ensure that the correct value is predicted when numClasses passed through metadata
+ val labelMeta = NominalAttribute.defaultAttr.withName("label").withNumValues(6).toMetadata()
+ val constantDataWithMetadata = constantData
+ .select(constantData("label").as("label", labelMeta), constantData("features"))
+ val modelWithMetadata = mlr.setFitIntercept(true).fit(constantDataWithMetadata)
+ val resultsWithMetadata = modelWithMetadata.transform(constantDataWithMetadata)
+ resultsWithMetadata.select("rawPrediction", "probability", "prediction").collect().foreach {
+ case Row(raw: Vector, prob: Vector, pred: Double) =>
+ assert(raw === Vectors.dense(Array(0.0, 0.0, 0.0, 0.0, Double.PositiveInfinity, 0.0)))
+ assert(prob === Vectors.dense(Array(0.0, 0.0, 0.0, 0.0, 1.0, 0.0)))
+ assert(pred === 4.0)
+ }
+ // TODO: check num iters is zero when it become available in the model
+ }
+
+ test("compressed storage") {
+ val moreClassesThanFeatures = spark.createDataFrame(Seq(
+ LabeledPoint(4.0, Vectors.dense(0.0, 0.0, 0.0)),
+ LabeledPoint(4.0, Vectors.dense(1.0, 1.0, 1.0)),
+ LabeledPoint(4.0, Vectors.dense(2.0, 2.0, 2.0)))
+ )
+ val mlr = new LogisticRegression().setFamily("multinomial")
+ val model = mlr.fit(moreClassesThanFeatures)
+ assert(model.coefficientMatrix.isInstanceOf[SparseMatrix])
+ assert(model.coefficientMatrix.asInstanceOf[SparseMatrix].colPtrs.length === 4)
+ val moreFeaturesThanClasses = spark.createDataFrame(Seq(
+ LabeledPoint(1.0, Vectors.dense(0.0, 0.0, 0.0)),
+ LabeledPoint(1.0, Vectors.dense(1.0, 1.0, 1.0)),
+ LabeledPoint(1.0, Vectors.dense(2.0, 2.0, 2.0)))
+ )
+ val model2 = mlr.fit(moreFeaturesThanClasses)
+ assert(model2.coefficientMatrix.isInstanceOf[SparseMatrix])
+ assert(model2.coefficientMatrix.asInstanceOf[SparseMatrix].colPtrs.length === 3)
+
+ val blr = new LogisticRegression().setFamily("binomial")
+ val blrModel = blr.fit(moreFeaturesThanClasses)
+ assert(blrModel.coefficientMatrix.isInstanceOf[SparseMatrix])
+ assert(blrModel.coefficientMatrix.asInstanceOf[SparseMatrix].colPtrs.length === 2)
+ }
+
+ test("numClasses specified in metadata/inferred") {
+ val lr = new LogisticRegression().setMaxIter(1).setFamily("multinomial")
+
+ // specify more classes than unique label values
+ val labelMeta = NominalAttribute.defaultAttr.withName("label").withNumValues(4).toMetadata()
+ val df = smallMultinomialDataset.select(smallMultinomialDataset("label").as("label", labelMeta),
+ smallMultinomialDataset("features"))
+ val model1 = lr.fit(df)
+ assert(model1.numClasses === 4)
+ assert(model1.interceptVector.size === 4)
+
+ // specify two classes when there are really three
+ val labelMeta1 = NominalAttribute.defaultAttr.withName("label").withNumValues(2).toMetadata()
+ val df1 = smallMultinomialDataset
+ .select(smallMultinomialDataset("label").as("label", labelMeta1),
+ smallMultinomialDataset("features"))
+ val thrown = intercept[IllegalArgumentException] {
+ lr.fit(df1)
+ }
+ assert(thrown.getMessage.contains("less than the number of unique labels"))
+
+ // lr should infer the number of classes if not specified
+ val model3 = lr.fit(smallMultinomialDataset)
+ assert(model3.numClasses === 3)
+ }
+
test("read/write") {
def checkModelData(model: LogisticRegressionModel, model2: LogisticRegressionModel): Unit = {
assert(model.intercept === model2.intercept)
@@ -936,7 +2048,7 @@ class LogisticRegressionSuite
assert(model.numFeatures === model2.numFeatures)
}
val lr = new LogisticRegression()
- testEstimatorAndModelReadWrite(lr, dataset, LogisticRegressionSuite.allParamSettings,
+ testEstimatorAndModelReadWrite(lr, smallBinaryDataset, LogisticRegressionSuite.allParamSettings,
checkModelData)
}
diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/MultinomialLogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/MultinomialLogisticRegressionSuite.scala
deleted file mode 100644
index 0913fe559c..0000000000
--- a/mllib/src/test/scala/org/apache/spark/ml/classification/MultinomialLogisticRegressionSuite.scala
+++ /dev/null
@@ -1,1056 +0,0 @@
-/*
- * 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.spark.ml.classification
-
-import scala.language.existentials
-
-import org.apache.spark.SparkFunSuite
-import org.apache.spark.ml.attribute.NominalAttribute
-import org.apache.spark.ml.classification.LogisticRegressionSuite._
-import org.apache.spark.ml.feature.LabeledPoint
-import org.apache.spark.ml.linalg._
-import org.apache.spark.ml.param.ParamsSuite
-import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils}
-import org.apache.spark.ml.util.TestingUtils._
-import org.apache.spark.mllib.util.MLlibTestSparkContext
-import org.apache.spark.sql.{DataFrame, Dataset, Row}
-
-class MultinomialLogisticRegressionSuite
- extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
-
- @transient var dataset: Dataset[_] = _
- @transient var multinomialDataset: DataFrame = _
- private val eps: Double = 1e-5
-
- override def beforeAll(): Unit = {
- super.beforeAll()
-
- dataset = {
- val nPoints = 100
- val coefficients = Array(
- -0.57997, 0.912083, -0.371077,
- -0.16624, -0.84355, -0.048509)
-
- val xMean = Array(5.843, 3.057)
- val xVariance = Array(0.6856, 0.1899)
-
- val testData = generateMultinomialLogisticInput(
- coefficients, xMean, xVariance, addIntercept = true, nPoints, 42)
-
- val df = spark.createDataFrame(sc.parallelize(testData, 4))
- df.cache()
- df
- }
-
- multinomialDataset = {
- val nPoints = 10000
- val coefficients = Array(
- -0.57997, 0.912083, -0.371077, -0.819866, 2.688191,
- -0.16624, -0.84355, -0.048509, -0.301789, 4.170682)
-
- val xMean = Array(5.843, 3.057, 3.758, 1.199)
- val xVariance = Array(0.6856, 0.1899, 3.116, 0.581)
-
- val testData = generateMultinomialLogisticInput(
- coefficients, xMean, xVariance, addIntercept = true, nPoints, 42)
-
- val df = spark.createDataFrame(sc.parallelize(testData, 4))
- df.cache()
- df
- }
- }
-
- /**
- * Enable the ignored test to export the dataset into CSV format,
- * so we can validate the training accuracy compared with R's glmnet package.
- */
- ignore("export test data into CSV format") {
- val rdd = multinomialDataset.rdd.map { case Row(label: Double, features: Vector) =>
- label + "," + features.toArray.mkString(",")
- }.repartition(1)
- rdd.saveAsTextFile("target/tmp/MultinomialLogisticRegressionSuite/multinomialDataset")
- }
-
- test("params") {
- ParamsSuite.checkParams(new MultinomialLogisticRegression)
- val model = new MultinomialLogisticRegressionModel("mLogReg",
- Matrices.dense(2, 1, Array(0.0, 0.0)), Vectors.dense(0.0, 0.0), 2)
- ParamsSuite.checkParams(model)
- }
-
- test("multinomial logistic regression: default params") {
- val mlr = new MultinomialLogisticRegression
- assert(mlr.getLabelCol === "label")
- assert(mlr.getFeaturesCol === "features")
- assert(mlr.getPredictionCol === "prediction")
- assert(mlr.getRawPredictionCol === "rawPrediction")
- assert(mlr.getProbabilityCol === "probability")
- assert(!mlr.isDefined(mlr.weightCol))
- assert(!mlr.isDefined(mlr.thresholds))
- assert(mlr.getFitIntercept)
- assert(mlr.getStandardization)
- val model = mlr.fit(dataset)
- model.transform(dataset)
- .select("label", "probability", "prediction", "rawPrediction")
- .collect()
- assert(model.getFeaturesCol === "features")
- assert(model.getPredictionCol === "prediction")
- assert(model.getRawPredictionCol === "rawPrediction")
- assert(model.getProbabilityCol === "probability")
- assert(model.intercepts !== Vectors.dense(0.0, 0.0))
- assert(model.hasParent)
- }
-
- test("multinomial logistic regression with intercept without regularization") {
-
- val trainer1 = (new MultinomialLogisticRegression).setFitIntercept(true)
- .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true).setMaxIter(100)
- val trainer2 = (new MultinomialLogisticRegression).setFitIntercept(true)
- .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(false)
-
- val model1 = trainer1.fit(multinomialDataset)
- val model2 = trainer2.fit(multinomialDataset)
-
- /*
- Using 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)
- > features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
- > coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0, lambda = 0))
- > coefficients
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- -2.24493379
- V2 0.25096771
- V3 -0.03915938
- V4 0.14766639
- V5 0.36810817
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- 0.3778931
- V2 -0.3327489
- V3 0.8893666
- V4 -0.2306948
- V5 -0.4442330
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- 1.86704066
- V2 0.08178121
- V3 -0.85020722
- V4 0.08302840
- V5 0.07612480
- */
-
- val coefficientsR = new DenseMatrix(3, 4, Array(
- 0.2509677, -0.0391594, 0.1476664, 0.3681082,
- -0.3327489, 0.8893666, -0.2306948, -0.4442330,
- 0.0817812, -0.8502072, 0.0830284, 0.0761248), isTransposed = true)
- val interceptsR = Vectors.dense(-2.2449338, 0.3778931, 1.8670407)
-
- assert(model1.coefficients ~== coefficientsR relTol 0.05)
- assert(model1.coefficients.toArray.sum ~== 0.0 absTol eps)
- assert(model1.intercepts ~== interceptsR relTol 0.05)
- assert(model1.intercepts.toArray.sum ~== 0.0 absTol eps)
- assert(model2.coefficients ~== coefficientsR relTol 0.05)
- assert(model2.coefficients.toArray.sum ~== 0.0 absTol eps)
- assert(model2.intercepts ~== interceptsR relTol 0.05)
- assert(model2.intercepts.toArray.sum ~== 0.0 absTol eps)
- }
-
- test("multinomial logistic regression without intercept without regularization") {
-
- val trainer1 = (new MultinomialLogisticRegression).setFitIntercept(false)
- .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true)
- val trainer2 = (new MultinomialLogisticRegression).setFitIntercept(false)
- .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(false)
-
- val model1 = trainer1.fit(multinomialDataset)
- val model2 = trainer2.fit(multinomialDataset)
-
- /*
- Using 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)
- features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
- coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0, lambda = 0,
- intercept=F))
- > coefficients
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 0.06992464
- V3 -0.36562784
- V4 0.12142680
- V5 0.32052211
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 -0.3036269
- V3 0.9449630
- V4 -0.2271038
- V5 -0.4364839
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 0.2337022
- V3 -0.5793351
- V4 0.1056770
- V5 0.1159618
- */
-
- val coefficientsR = new DenseMatrix(3, 4, Array(
- 0.0699246, -0.3656278, 0.1214268, 0.3205221,
- -0.3036269, 0.9449630, -0.2271038, -0.4364839,
- 0.2337022, -0.5793351, 0.1056770, 0.1159618), isTransposed = true)
-
- assert(model1.coefficients ~== coefficientsR relTol 0.05)
- assert(model1.coefficients.toArray.sum ~== 0.0 absTol eps)
- assert(model1.intercepts.toArray === Array.fill(3)(0.0))
- assert(model1.intercepts.toArray.sum ~== 0.0 absTol eps)
- assert(model2.coefficients ~== coefficientsR relTol 0.05)
- assert(model2.coefficients.toArray.sum ~== 0.0 absTol eps)
- assert(model2.intercepts.toArray === Array.fill(3)(0.0))
- assert(model2.intercepts.toArray.sum ~== 0.0 absTol eps)
- }
-
- test("multinomial logistic regression with intercept with L1 regularization") {
-
- // use tighter constraints because OWL-QN solver takes longer to converge
- val trainer1 = (new MultinomialLogisticRegression).setFitIntercept(true)
- .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(true)
- .setMaxIter(300).setTol(1e-10)
- val trainer2 = (new MultinomialLogisticRegression).setFitIntercept(true)
- .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(false)
- .setMaxIter(300).setTol(1e-10)
-
- val model1 = trainer1.fit(multinomialDataset)
- val model2 = trainer2.fit(multinomialDataset)
-
- /*
- 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)
- features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
- coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 1,
- lambda = 0.05, standardization=T))
- coefficients = coef(glmnet(features, label, family="multinomial", alpha = 1, lambda = 0.05,
- standardization=F))
- > coefficientsStd
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- -0.68988825
- V2 .
- V3 .
- V4 .
- V5 0.09404023
-
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- -0.2303499
- V2 -0.1232443
- V3 0.3258380
- V4 -0.1564688
- V5 -0.2053965
-
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- 0.9202381
- V2 .
- V3 -0.4803856
- V4 .
- V5 .
-
- > coefficients
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- -0.44893320
- V2 .
- V3 .
- V4 0.01933812
- V5 0.03666044
-
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- 0.7376760
- V2 -0.0577182
- V3 .
- V4 -0.2081718
- V5 -0.1304592
-
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- -0.2887428
- V2 .
- V3 .
- V4 .
- V5 .
- */
-
- val coefficientsRStd = new DenseMatrix(3, 4, Array(
- 0.0, 0.0, 0.0, 0.09404023,
- -0.1232443, 0.3258380, -0.1564688, -0.2053965,
- 0.0, -0.4803856, 0.0, 0.0), isTransposed = true)
- val interceptsRStd = Vectors.dense(-0.68988825, -0.2303499, 0.9202381)
-
- val coefficientsR = new DenseMatrix(3, 4, Array(
- 0.0, 0.0, 0.01933812, 0.03666044,
- -0.0577182, 0.0, -0.2081718, -0.1304592,
- 0.0, 0.0, 0.0, 0.0), isTransposed = true)
- val interceptsR = Vectors.dense(-0.44893320, 0.7376760, -0.2887428)
-
- assert(model1.coefficients ~== coefficientsRStd absTol 0.02)
- assert(model1.intercepts ~== interceptsRStd relTol 0.1)
- assert(model1.intercepts.toArray.sum ~== 0.0 absTol eps)
- assert(model2.coefficients ~== coefficientsR absTol 0.02)
- assert(model2.intercepts ~== interceptsR relTol 0.1)
- assert(model2.intercepts.toArray.sum ~== 0.0 absTol eps)
- }
-
- test("multinomial logistic regression without intercept with L1 regularization") {
- val trainer1 = (new MultinomialLogisticRegression).setFitIntercept(false)
- .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(true)
- val trainer2 = (new MultinomialLogisticRegression).setFitIntercept(false)
- .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(false)
-
- val model1 = trainer1.fit(multinomialDataset)
- val model2 = trainer2.fit(multinomialDataset)
- /*
- 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)
- features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
- coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 1,
- lambda = 0.05, intercept=F, standardization=T))
- coefficients = coef(glmnet(features, label, family="multinomial", alpha = 1, lambda = 0.05,
- intercept=F, standardization=F))
- > coefficientsStd
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 .
- V3 .
- V4 .
- V5 0.01525105
-
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 -0.1502410
- V3 0.5134658
- V4 -0.1601146
- V5 -0.2500232
-
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 0.003301875
- V3 .
- V4 .
- V5 .
-
- > coefficients
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 .
- V3 .
- V4 .
- V5 .
-
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 .
- V3 0.1943624
- V4 -0.1902577
- V5 -0.1028789
-
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 .
- V3 .
- V4 .
- V5 .
- */
-
- val coefficientsRStd = new DenseMatrix(3, 4, Array(
- 0.0, 0.0, 0.0, 0.01525105,
- -0.1502410, 0.5134658, -0.1601146, -0.2500232,
- 0.003301875, 0.0, 0.0, 0.0), isTransposed = true)
-
- val coefficientsR = new DenseMatrix(3, 4, Array(
- 0.0, 0.0, 0.0, 0.0,
- 0.0, 0.1943624, -0.1902577, -0.1028789,
- 0.0, 0.0, 0.0, 0.0), isTransposed = true)
-
- assert(model1.coefficients ~== coefficientsRStd absTol 0.01)
- assert(model1.intercepts.toArray === Array.fill(3)(0.0))
- assert(model1.intercepts.toArray.sum ~== 0.0 absTol eps)
- assert(model2.coefficients ~== coefficientsR absTol 0.01)
- assert(model2.intercepts.toArray === Array.fill(3)(0.0))
- assert(model2.intercepts.toArray.sum ~== 0.0 absTol eps)
- }
-
- test("multinomial logistic regression with intercept with L2 regularization") {
- val trainer1 = (new MultinomialLogisticRegression).setFitIntercept(true)
- .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(true)
- val trainer2 = (new MultinomialLogisticRegression).setFitIntercept(true)
- .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(false)
-
- val model1 = trainer1.fit(multinomialDataset)
- val model2 = trainer2.fit(multinomialDataset)
- /*
- 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)
- features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
- coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 0,
- lambda = 0.1, intercept=T, standardization=T))
- coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0,
- lambda = 0.1, intercept=T, standardization=F))
- > coefficientsStd
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- -1.70040424
- V2 0.17576070
- V3 0.01527894
- V4 0.10216108
- V5 0.26099531
-
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- 0.2438590
- V2 -0.2238875
- V3 0.5967610
- V4 -0.1555496
- V5 -0.3010479
-
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- 1.45654525
- V2 0.04812679
- V3 -0.61203992
- V4 0.05338850
- V5 0.04005258
-
- > coefficients
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- -1.65488543
- V2 0.15715048
- V3 0.01992903
- V4 0.12428858
- V5 0.22130317
-
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- 1.1297533
- V2 -0.1974768
- V3 0.2776373
- V4 -0.1869445
- V5 -0.2510320
-
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- 0.52513212
- V2 0.04032627
- V3 -0.29756637
- V4 0.06265594
- V5 0.02972883
- */
-
- val coefficientsRStd = new DenseMatrix(3, 4, Array(
- 0.17576070, 0.01527894, 0.10216108, 0.26099531,
- -0.2238875, 0.5967610, -0.1555496, -0.3010479,
- 0.04812679, -0.61203992, 0.05338850, 0.04005258), isTransposed = true)
- val interceptsRStd = Vectors.dense(-1.70040424, 0.2438590, 1.45654525)
-
- val coefficientsR = new DenseMatrix(3, 4, Array(
- 0.15715048, 0.01992903, 0.12428858, 0.22130317,
- -0.1974768, 0.2776373, -0.1869445, -0.2510320,
- 0.04032627, -0.29756637, 0.06265594, 0.02972883), isTransposed = true)
- val interceptsR = Vectors.dense(-1.65488543, 1.1297533, 0.52513212)
-
- assert(model1.coefficients ~== coefficientsRStd relTol 0.05)
- assert(model1.intercepts ~== interceptsRStd relTol 0.05)
- assert(model1.intercepts.toArray.sum ~== 0.0 absTol eps)
- assert(model2.coefficients ~== coefficientsR relTol 0.05)
- assert(model2.intercepts ~== interceptsR relTol 0.05)
- assert(model2.intercepts.toArray.sum ~== 0.0 absTol eps)
- }
-
- test("multinomial logistic regression without intercept with L2 regularization") {
- val trainer1 = (new MultinomialLogisticRegression).setFitIntercept(false)
- .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(true)
- val trainer2 = (new MultinomialLogisticRegression).setFitIntercept(false)
- .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(false)
-
- val model1 = trainer1.fit(multinomialDataset)
- val model2 = trainer2.fit(multinomialDataset)
- /*
- 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)
- features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
- coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 0,
- lambda = 0.1, intercept=F, standardization=T))
- coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0,
- lambda = 0.1, intercept=F, standardization=F))
- > coefficientsStd
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 0.03904171
- V3 -0.23354322
- V4 0.08288096
- V5 0.22706393
-
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 -0.2061848
- V3 0.6341398
- V4 -0.1530059
- V5 -0.2958455
-
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 0.16714312
- V3 -0.40059658
- V4 0.07012496
- V5 0.06878158
- > coefficients
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 -0.005704542
- V3 -0.144466409
- V4 0.092080736
- V5 0.182927657
-
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 -0.08469036
- V3 0.38996748
- V4 -0.16468436
- V5 -0.22522976
-
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 0.09039490
- V3 -0.24550107
- V4 0.07260362
- V5 0.04230210
- */
- val coefficientsRStd = new DenseMatrix(3, 4, Array(
- 0.03904171, -0.23354322, 0.08288096, 0.2270639,
- -0.2061848, 0.6341398, -0.1530059, -0.2958455,
- 0.16714312, -0.40059658, 0.07012496, 0.06878158), isTransposed = true)
-
- val coefficientsR = new DenseMatrix(3, 4, Array(
- -0.005704542, -0.144466409, 0.092080736, 0.182927657,
- -0.08469036, 0.38996748, -0.16468436, -0.22522976,
- 0.0903949, -0.24550107, 0.07260362, 0.0423021), isTransposed = true)
-
- assert(model1.coefficients ~== coefficientsRStd absTol 0.01)
- assert(model1.intercepts.toArray === Array.fill(3)(0.0))
- assert(model1.intercepts.toArray.sum ~== 0.0 absTol eps)
- assert(model2.coefficients ~== coefficientsR absTol 0.01)
- assert(model2.intercepts.toArray === Array.fill(3)(0.0))
- assert(model2.intercepts.toArray.sum ~== 0.0 absTol eps)
- }
-
- test("multinomial logistic regression with intercept with elasticnet regularization") {
- val trainer1 = (new MultinomialLogisticRegression).setFitIntercept(true)
- .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(true)
- .setMaxIter(300).setTol(1e-10)
- val trainer2 = (new MultinomialLogisticRegression).setFitIntercept(true)
- .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(false)
- .setMaxIter(300).setTol(1e-10)
-
- val model1 = trainer1.fit(multinomialDataset)
- val model2 = trainer2.fit(multinomialDataset)
- /*
- 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)
- features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
- coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 0.5,
- lambda = 0.1, intercept=T, standardization=T))
- coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0.5,
- lambda = 0.1, intercept=T, standardization=F))
- > coefficientsStd
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- -0.5521819483
- V2 0.0003092611
- V3 .
- V4 .
- V5 0.0913818490
-
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- -0.27531989
- V2 -0.09790029
- V3 0.28502034
- V4 -0.12416487
- V5 -0.16513373
-
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- 0.8275018
- V2 .
- V3 -0.4044859
- V4 .
- V5 .
-
- > coefficients
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- -0.39876213
- V2 .
- V3 .
- V4 0.02547520
- V5 0.03893991
-
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- 0.61089869
- V2 -0.04224269
- V3 .
- V4 -0.18923970
- V5 -0.09104249
-
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- -0.2121366
- V2 .
- V3 .
- V4 .
- V5 .
- */
-
- val coefficientsRStd = new DenseMatrix(3, 4, Array(
- 0.0003092611, 0.0, 0.0, 0.091381849,
- -0.09790029, 0.28502034, -0.12416487, -0.16513373,
- 0.0, -0.4044859, 0.0, 0.0), isTransposed = true)
- val interceptsRStd = Vectors.dense(-0.5521819483, -0.27531989, 0.8275018)
-
- val coefficientsR = new DenseMatrix(3, 4, Array(
- 0.0, 0.0, 0.0254752, 0.03893991,
- -0.04224269, 0.0, -0.1892397, -0.09104249,
- 0.0, 0.0, 0.0, 0.0), isTransposed = true)
- val interceptsR = Vectors.dense(-0.39876213, 0.61089869, -0.2121366)
-
- assert(model1.coefficients ~== coefficientsRStd absTol 0.01)
- assert(model1.intercepts ~== interceptsRStd absTol 0.01)
- assert(model1.intercepts.toArray.sum ~== 0.0 absTol eps)
- assert(model2.coefficients ~== coefficientsR absTol 0.01)
- assert(model2.intercepts ~== interceptsR absTol 0.01)
- assert(model2.intercepts.toArray.sum ~== 0.0 absTol eps)
- }
-
- test("multinomial logistic regression without intercept with elasticnet regularization") {
- val trainer1 = (new MultinomialLogisticRegression).setFitIntercept(false)
- .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(true)
- .setMaxIter(300).setTol(1e-10)
- val trainer2 = (new MultinomialLogisticRegression).setFitIntercept(false)
- .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(false)
- .setMaxIter(300).setTol(1e-10)
-
- val model1 = trainer1.fit(multinomialDataset)
- val model2 = trainer2.fit(multinomialDataset)
- /*
- 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)
- features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
- coefficientsStd = coef(glmnet(features, label, family="multinomial", alpha = 0.5,
- lambda = 0.1, intercept=F, standardization=T))
- coefficients = coef(glmnet(features, label, family="multinomial", alpha = 0.5,
- lambda = 0.1, intercept=F, standardization=F))
- > coefficientsStd
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 .
- V3 .
- V4 .
- V5 0.03543706
-
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 -0.1187387
- V3 0.4025482
- V4 -0.1270969
- V5 -0.1918386
-
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 0.00774365
- V3 .
- V4 .
- V5 .
-
- > coefficients
- $`0`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 .
- V3 .
- V4 .
- V5 .
-
- $`1`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 .
- V3 0.14666497
- V4 -0.16570638
- V5 -0.05982875
-
- $`2`
- 5 x 1 sparse Matrix of class "dgCMatrix"
- s0
- .
- V2 .
- V3 .
- V4 .
- V5 .
- */
- val coefficientsRStd = new DenseMatrix(3, 4, Array(
- 0.0, 0.0, 0.0, 0.03543706,
- -0.1187387, 0.4025482, -0.1270969, -0.1918386,
- 0.0, 0.0, 0.0, 0.00774365), isTransposed = true)
-
- val coefficientsR = new DenseMatrix(3, 4, Array(
- 0.0, 0.0, 0.0, 0.0,
- 0.0, 0.14666497, -0.16570638, -0.05982875,
- 0.0, 0.0, 0.0, 0.0), isTransposed = true)
-
- assert(model1.coefficients ~== coefficientsRStd absTol 0.01)
- assert(model1.intercepts.toArray === Array.fill(3)(0.0))
- assert(model1.intercepts.toArray.sum ~== 0.0 absTol eps)
- assert(model2.coefficients ~== coefficientsR absTol 0.01)
- assert(model2.intercepts.toArray === Array.fill(3)(0.0))
- assert(model2.intercepts.toArray.sum ~== 0.0 absTol eps)
- }
-
- /*
- test("multinomial logistic regression with intercept with strong L1 regularization") {
- // TODO: implement this test to check that the priors on the intercepts are correct
- // TODO: when initial model becomes available
- }
- */
-
- test("prediction") {
- val model = new MultinomialLogisticRegressionModel("mLogReg",
- Matrices.dense(3, 2, Array(0.0, 0.0, 0.0, 1.0, 2.0, 3.0)),
- Vectors.dense(0.0, 0.0, 0.0), 3)
- val overFlowData = spark.createDataFrame(Seq(
- LabeledPoint(1.0, Vectors.dense(0.0, 1000.0)),
- LabeledPoint(1.0, Vectors.dense(0.0, -1.0))
- ))
- val results = model.transform(overFlowData).select("rawPrediction", "probability").collect()
-
- // probabilities are correct when margins have to be adjusted
- val raw1 = results(0).getAs[Vector](0)
- val prob1 = results(0).getAs[Vector](1)
- assert(raw1 === Vectors.dense(1000.0, 2000.0, 3000.0))
- assert(prob1 ~== Vectors.dense(0.0, 0.0, 1.0) absTol eps)
-
- // probabilities are correct when margins don't have to be adjusted
- val raw2 = results(1).getAs[Vector](0)
- val prob2 = results(1).getAs[Vector](1)
- assert(raw2 === Vectors.dense(-1.0, -2.0, -3.0))
- assert(prob2 ~== Vectors.dense(0.66524096, 0.24472847, 0.09003057) relTol eps)
- }
-
- test("multinomial logistic regression: Predictor, Classifier methods") {
- val mlr = new MultinomialLogisticRegression
-
- val model = mlr.fit(dataset)
- assert(model.numClasses === 3)
- val numFeatures = dataset.select("features").first().getAs[Vector](0).size
- assert(model.numFeatures === numFeatures)
-
- val results = model.transform(dataset)
- // check that raw prediction is coefficients dot features + intercept
- results.select("rawPrediction", "features").collect().foreach {
- case Row(raw: Vector, features: Vector) =>
- assert(raw.size === 3)
- val margins = Array.tabulate(3) { k =>
- var margin = 0.0
- features.foreachActive { (index, value) =>
- margin += value * model.coefficients(k, index)
- }
- margin += model.intercepts(k)
- margin
- }
- assert(raw ~== Vectors.dense(margins) relTol eps)
- }
-
- // Compare rawPrediction with probability
- results.select("rawPrediction", "probability").collect().foreach {
- case Row(raw: Vector, prob: Vector) =>
- assert(raw.size === 3)
- assert(prob.size === 3)
- val max = raw.toArray.max
- val subtract = if (max > 0) max else 0.0
- val sum = raw.toArray.map(x => math.exp(x - subtract)).sum
- val probFromRaw0 = math.exp(raw(0) - subtract) / sum
- val probFromRaw1 = math.exp(raw(1) - subtract) / sum
- assert(prob(0) ~== probFromRaw0 relTol eps)
- assert(prob(1) ~== probFromRaw1 relTol eps)
- assert(prob(2) ~== 1.0 - probFromRaw1 - probFromRaw0 relTol eps)
- }
-
- // Compare prediction with probability
- results.select("prediction", "probability").collect().foreach {
- case Row(pred: Double, prob: Vector) =>
- val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2
- assert(pred == predFromProb)
- }
- }
-
- test("multinomial logistic regression coefficients should be centered") {
- val mlr = new MultinomialLogisticRegression().setMaxIter(1)
- val model = mlr.fit(dataset)
- assert(model.intercepts.toArray.sum ~== 0.0 absTol 1e-6)
- assert(model.coefficients.toArray.sum ~== 0.0 absTol 1e-6)
- }
-
- test("numClasses specified in metadata/inferred") {
- val mlr = new MultinomialLogisticRegression().setMaxIter(1)
-
- // specify more classes than unique label values
- val labelMeta = NominalAttribute.defaultAttr.withName("label").withNumValues(4).toMetadata()
- val df = dataset.select(dataset("label").as("label", labelMeta), dataset("features"))
- val model1 = mlr.fit(df)
- assert(model1.numClasses === 4)
- assert(model1.intercepts.size === 4)
-
- // specify two classes when there are really three
- val labelMeta1 = NominalAttribute.defaultAttr.withName("label").withNumValues(2).toMetadata()
- val df1 = dataset.select(dataset("label").as("label", labelMeta1), dataset("features"))
- val thrown = intercept[IllegalArgumentException] {
- mlr.fit(df1)
- }
- assert(thrown.getMessage.contains("less than the number of unique labels"))
-
- // mlr should infer the number of classes if not specified
- val model3 = mlr.fit(dataset)
- assert(model3.numClasses === 3)
- }
-
- test("all labels the same") {
- val constantData = spark.createDataFrame(Seq(
- LabeledPoint(4.0, Vectors.dense(0.0)),
- LabeledPoint(4.0, Vectors.dense(1.0)),
- LabeledPoint(4.0, Vectors.dense(2.0)))
- )
- val mlr = new MultinomialLogisticRegression
- val model = mlr.fit(constantData)
- val results = model.transform(constantData)
- results.select("rawPrediction", "probability", "prediction").collect().foreach {
- case Row(raw: Vector, prob: Vector, pred: Double) =>
- assert(raw === Vectors.dense(Array(0.0, 0.0, 0.0, 0.0, Double.PositiveInfinity)))
- assert(prob === Vectors.dense(Array(0.0, 0.0, 0.0, 0.0, 1.0)))
- assert(pred === 4.0)
- }
-
- // force the model to be trained with only one class
- val constantZeroData = spark.createDataFrame(Seq(
- LabeledPoint(0.0, Vectors.dense(0.0)),
- LabeledPoint(0.0, Vectors.dense(1.0)),
- LabeledPoint(0.0, Vectors.dense(2.0)))
- )
- val modelZeroLabel = mlr.setFitIntercept(false).fit(constantZeroData)
- val resultsZero = modelZeroLabel.transform(constantZeroData)
- resultsZero.select("rawPrediction", "probability", "prediction").collect().foreach {
- case Row(raw: Vector, prob: Vector, pred: Double) =>
- assert(prob === Vectors.dense(Array(1.0)))
- assert(pred === 0.0)
- }
-
- // ensure that the correct value is predicted when numClasses passed through metadata
- val labelMeta = NominalAttribute.defaultAttr.withName("label").withNumValues(6).toMetadata()
- val constantDataWithMetadata = constantData
- .select(constantData("label").as("label", labelMeta), constantData("features"))
- val modelWithMetadata = mlr.setFitIntercept(true).fit(constantDataWithMetadata)
- val resultsWithMetadata = modelWithMetadata.transform(constantDataWithMetadata)
- resultsWithMetadata.select("rawPrediction", "probability", "prediction").collect().foreach {
- case Row(raw: Vector, prob: Vector, pred: Double) =>
- assert(raw === Vectors.dense(Array(0.0, 0.0, 0.0, 0.0, Double.PositiveInfinity, 0.0)))
- assert(prob === Vectors.dense(Array(0.0, 0.0, 0.0, 0.0, 1.0, 0.0)))
- assert(pred === 4.0)
- }
- // TODO: check num iters is zero when it become available in the model
- }
-
- test("weighted data") {
- val numClasses = 5
- val numPoints = 40
- val outlierData = MLTestingUtils.genClassificationInstancesWithWeightedOutliers(spark,
- numClasses, numPoints)
- val testData = spark.createDataFrame(Array.tabulate[LabeledPoint](numClasses) { i =>
- LabeledPoint(i.toDouble, Vectors.dense(i.toDouble))
- })
- val mlr = new MultinomialLogisticRegression().setWeightCol("weight")
- val model = mlr.fit(outlierData)
- val results = model.transform(testData).select("label", "prediction").collect()
-
- // check that the predictions are the one to one mapping
- results.foreach { case Row(label: Double, pred: Double) =>
- assert(label === pred)
- }
- val (overSampledData, weightedData) =
- MLTestingUtils.genEquivalentOversampledAndWeightedInstances(outlierData, "label", "features",
- 42L)
- val weightedModel = mlr.fit(weightedData)
- val overSampledModel = mlr.setWeightCol("").fit(overSampledData)
- assert(weightedModel.coefficients ~== overSampledModel.coefficients relTol 0.01)
- }
-
- test("thresholds prediction") {
- val mlr = new MultinomialLogisticRegression
- val model = mlr.fit(dataset)
- val basePredictions = model.transform(dataset).select("prediction").collect()
-
- // should predict all zeros
- model.setThresholds(Array(1, 1000, 1000))
- val zeroPredictions = model.transform(dataset).select("prediction").collect()
- assert(zeroPredictions.forall(_.getDouble(0) === 0.0))
-
- // should predict all ones
- model.setThresholds(Array(1000, 1, 1000))
- val onePredictions = model.transform(dataset).select("prediction").collect()
- assert(onePredictions.forall(_.getDouble(0) === 1.0))
-
- // should predict all twos
- model.setThresholds(Array(1000, 1000, 1))
- val twoPredictions = model.transform(dataset).select("prediction").collect()
- assert(twoPredictions.forall(_.getDouble(0) === 2.0))
-
- // constant threshold scaling is the same as no thresholds
- model.setThresholds(Array(1000, 1000, 1000))
- val scaledPredictions = model.transform(dataset).select("prediction").collect()
- assert(scaledPredictions.zip(basePredictions).forall { case (scaled, base) =>
- scaled.getDouble(0) === base.getDouble(0)
- })
- }
-
- test("read/write") {
- def checkModelData(
- model: MultinomialLogisticRegressionModel,
- model2: MultinomialLogisticRegressionModel): Unit = {
- assert(model.intercepts === model2.intercepts)
- assert(model.coefficients.toArray === model2.coefficients.toArray)
- assert(model.numClasses === model2.numClasses)
- assert(model.numFeatures === model2.numFeatures)
- }
- val mlr = new MultinomialLogisticRegression()
- testEstimatorAndModelReadWrite(mlr, dataset,
- MultinomialLogisticRegressionSuite.allParamSettings,
- checkModelData)
- }
-
- test("should support all NumericType labels and not support other types") {
- val mlr = new MultinomialLogisticRegression().setMaxIter(1)
- MLTestingUtils
- .checkNumericTypes[MultinomialLogisticRegressionModel, MultinomialLogisticRegression](
- mlr, spark) { (expected, actual) =>
- assert(expected.intercepts === actual.intercepts)
- assert(expected.coefficients.toArray === actual.coefficients.toArray)
- }
- }
-}
-
-object MultinomialLogisticRegressionSuite {
-
- /**
- * Mapping from all Params to valid settings which differ from the defaults.
- * This is useful for tests which need to exercise all Params, such as save/load.
- * This excludes input columns to simplify some tests.
- */
- val allParamSettings: Map[String, Any] = ProbabilisticClassifierSuite.allParamSettings ++ Map(
- "probabilityCol" -> "myProbability",
- "thresholds" -> Array(0.4, 0.6),
- "regParam" -> 0.01,
- "elasticNetParam" -> 0.1,
- "maxIter" -> 2, // intentionally small
- "fitIntercept" -> true,
- "tol" -> 0.8,
- "standardization" -> false
- )
-}
diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala
index 361dd74cb0..99dd5854ff 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala
@@ -22,7 +22,7 @@ import org.apache.spark.ml.attribute.NominalAttribute
import org.apache.spark.ml.classification.LogisticRegressionSuite._
import org.apache.spark.ml.feature.LabeledPoint
import org.apache.spark.ml.feature.StringIndexer
-import org.apache.spark.ml.linalg.Vectors
+import org.apache.spark.ml.linalg.{DenseMatrix, Vectors}
import org.apache.spark.ml.param.{ParamMap, ParamsSuite}
import org.apache.spark.ml.util.{DefaultReadWriteTest, MetadataUtils, MLTestingUtils}
import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS
diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala
index 30bd390381..750dc5bf01 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala
@@ -23,7 +23,7 @@ import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressio
import org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInput
import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, Evaluator, RegressionEvaluator}
import org.apache.spark.ml.feature.HashingTF
-import org.apache.spark.ml.linalg.Vectors
+import org.apache.spark.ml.linalg.{DenseMatrix, Vectors}
import org.apache.spark.ml.param.{ParamMap, ParamPair}
import org.apache.spark.ml.param.shared.HasInputCol
import org.apache.spark.ml.regression.LinearRegression
diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala
index c1e9c2fc1d..9971371e47 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala
@@ -22,7 +22,7 @@ import org.apache.spark.ml.{Estimator, Model}
import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel}
import org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInput
import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, Evaluator, RegressionEvaluator}
-import org.apache.spark.ml.linalg.Vectors
+import org.apache.spark.ml.linalg.{DenseMatrix, Vectors}
import org.apache.spark.ml.param.ParamMap
import org.apache.spark.ml.param.shared.HasInputCol
import org.apache.spark.ml.regression.LinearRegression