Skip to content

Commit cf29828

Browse files
sethahNick Pentreath
authored andcommitted
[SPARK-20988][ML] Logistic regression uses aggregator hierarchy
## What changes were proposed in this pull request? This change pulls the `LogisticAggregator` class out of LogisticRegression.scala and makes it extend `DifferentiableLossAggregator`. It also changes logistic regression to use the generic `RDDLossFunction` instead of having its own. Other minor changes: * L2Regularization accepts `Option[Int => Double]` for features standard deviation * L2Regularization uses `Vector` type instead of Array * Some tests added to LeastSquaresAggregator ## How was this patch tested? Unit test suites are added. Author: sethah <[email protected]> Closes #18305 from sethah/SPARK-20988.
1 parent ae4ea5f commit cf29828

File tree

11 files changed

+752
-537
lines changed

11 files changed

+752
-537
lines changed

mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala

Lines changed: 20 additions & 472 deletions
Large diffs are not rendered by default.
Lines changed: 364 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,364 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.spark.ml.optim.aggregator
18+
19+
import org.apache.spark.broadcast.Broadcast
20+
import org.apache.spark.internal.Logging
21+
import org.apache.spark.ml.feature.Instance
22+
import org.apache.spark.ml.linalg.{DenseVector, Vector}
23+
import org.apache.spark.mllib.util.MLUtils
24+
25+
/**
26+
* LogisticAggregator computes the gradient and loss for binary or multinomial logistic (softmax)
27+
* loss function, as used in classification for instances in sparse or dense vector in an online
28+
* fashion.
29+
*
30+
* Two LogisticAggregators can be merged together to have a summary of loss and gradient of
31+
* the corresponding joint dataset.
32+
*
33+
* For improving the convergence rate during the optimization process and also to prevent against
34+
* features with very large variances exerting an overly large influence during model training,
35+
* packages like R's GLMNET perform the scaling to unit variance and remove the mean in order to
36+
* reduce the condition number. The model is then trained in this scaled space, but returns the
37+
* coefficients in the original scale. See page 9 in
38+
* http://cran.r-project.org/web/packages/glmnet/glmnet.pdf
39+
*
40+
* However, we don't want to apply the [[org.apache.spark.ml.feature.StandardScaler]] on the
41+
* training dataset, and then cache the standardized dataset since it will create a lot of overhead.
42+
* As a result, we perform the scaling implicitly when we compute the objective function (though
43+
* we do not subtract the mean).
44+
*
45+
* Note that there is a difference between multinomial (softmax) and binary loss. The binary case
46+
* uses one outcome class as a "pivot" and regresses the other class against the pivot. In the
47+
* multinomial case, the softmax loss function is used to model each class probability
48+
* independently. Using softmax loss produces `K` sets of coefficients, while using a pivot class
49+
* produces `K - 1` sets of coefficients (a single coefficient vector in the binary case). In the
50+
* binary case, we can say that the coefficients are shared between the positive and negative
51+
* classes. When regularization is applied, multinomial (softmax) loss will produce a result
52+
* different from binary loss since the positive and negative don't share the coefficients while the
53+
* binary regression shares the coefficients between positive and negative.
54+
*
55+
* The following is a mathematical derivation for the multinomial (softmax) loss.
56+
*
57+
* The probability of the multinomial outcome $y$ taking on any of the K possible outcomes is:
58+
*
59+
* <blockquote>
60+
* $$
61+
* P(y_i=0|\vec{x}_i, \beta) = \frac{e^{\vec{x}_i^T \vec{\beta}_0}}{\sum_{k=0}^{K-1}
62+
* e^{\vec{x}_i^T \vec{\beta}_k}} \\
63+
* P(y_i=1|\vec{x}_i, \beta) = \frac{e^{\vec{x}_i^T \vec{\beta}_1}}{\sum_{k=0}^{K-1}
64+
* e^{\vec{x}_i^T \vec{\beta}_k}}\\
65+
* P(y_i=K-1|\vec{x}_i, \beta) = \frac{e^{\vec{x}_i^T \vec{\beta}_{K-1}}\,}{\sum_{k=0}^{K-1}
66+
* e^{\vec{x}_i^T \vec{\beta}_k}}
67+
* $$
68+
* </blockquote>
69+
*
70+
* The model coefficients $\beta = (\beta_0, \beta_1, \beta_2, ..., \beta_{K-1})$ become a matrix
71+
* which has dimension of $K \times (N+1)$ if the intercepts are added. If the intercepts are not
72+
* added, the dimension will be $K \times N$.
73+
*
74+
* Note that the coefficients in the model above lack identifiability. That is, any constant scalar
75+
* can be added to all of the coefficients and the probabilities remain the same.
76+
*
77+
* <blockquote>
78+
* $$
79+
* \begin{align}
80+
* \frac{e^{\vec{x}_i^T \left(\vec{\beta}_0 + \vec{c}\right)}}{\sum_{k=0}^{K-1}
81+
* e^{\vec{x}_i^T \left(\vec{\beta}_k + \vec{c}\right)}}
82+
* = \frac{e^{\vec{x}_i^T \vec{\beta}_0}e^{\vec{x}_i^T \vec{c}}\,}{e^{\vec{x}_i^T \vec{c}}
83+
* \sum_{k=0}^{K-1} e^{\vec{x}_i^T \vec{\beta}_k}}
84+
* = \frac{e^{\vec{x}_i^T \vec{\beta}_0}}{\sum_{k=0}^{K-1} e^{\vec{x}_i^T \vec{\beta}_k}}
85+
* \end{align}
86+
* $$
87+
* </blockquote>
88+
*
89+
* However, when regularization is added to the loss function, the coefficients are indeed
90+
* identifiable because there is only one set of coefficients which minimizes the regularization
91+
* term. When no regularization is applied, we choose the coefficients with the minimum L2
92+
* penalty for consistency and reproducibility. For further discussion see:
93+
*
94+
* Friedman, et al. "Regularization Paths for Generalized Linear Models via Coordinate Descent"
95+
*
96+
* The loss of objective function for a single instance of data (we do not include the
97+
* regularization term here for simplicity) can be written as
98+
*
99+
* <blockquote>
100+
* $$
101+
* \begin{align}
102+
* \ell\left(\beta, x_i\right) &= -log{P\left(y_i \middle| \vec{x}_i, \beta\right)} \\
103+
* &= log\left(\sum_{k=0}^{K-1}e^{\vec{x}_i^T \vec{\beta}_k}\right) - \vec{x}_i^T \vec{\beta}_y\\
104+
* &= log\left(\sum_{k=0}^{K-1} e^{margins_k}\right) - margins_y
105+
* \end{align}
106+
* $$
107+
* </blockquote>
108+
*
109+
* where ${margins}_k = \vec{x}_i^T \vec{\beta}_k$.
110+
*
111+
* For optimization, we have to calculate the first derivative of the loss function, and a simple
112+
* calculation shows that
113+
*
114+
* <blockquote>
115+
* $$
116+
* \begin{align}
117+
* \frac{\partial \ell(\beta, \vec{x}_i, w_i)}{\partial \beta_{j, k}}
118+
* &= x_{i,j} \cdot w_i \cdot \left(\frac{e^{\vec{x}_i \cdot \vec{\beta}_k}}{\sum_{k'=0}^{K-1}
119+
* e^{\vec{x}_i \cdot \vec{\beta}_{k'}}\,} - I_{y=k}\right) \\
120+
* &= x_{i, j} \cdot w_i \cdot multiplier_k
121+
* \end{align}
122+
* $$
123+
* </blockquote>
124+
*
125+
* where $w_i$ is the sample weight, $I_{y=k}$ is an indicator function
126+
*
127+
* <blockquote>
128+
* $$
129+
* I_{y=k} = \begin{cases}
130+
* 1 & y = k \\
131+
* 0 & else
132+
* \end{cases}
133+
* $$
134+
* </blockquote>
135+
*
136+
* and
137+
*
138+
* <blockquote>
139+
* $$
140+
* multiplier_k = \left(\frac{e^{\vec{x}_i \cdot \vec{\beta}_k}}{\sum_{k=0}^{K-1}
141+
* e^{\vec{x}_i \cdot \vec{\beta}_k}} - I_{y=k}\right)
142+
* $$
143+
* </blockquote>
144+
*
145+
* If any of margins is larger than 709.78, the numerical computation of multiplier and loss
146+
* function will suffer from arithmetic overflow. This issue occurs when there are outliers in
147+
* data which are far away from the hyperplane, and this will cause the failing of training once
148+
* infinity is introduced. Note that this is only a concern when max(margins) &gt; 0.
149+
*
150+
* Fortunately, when max(margins) = maxMargin &gt; 0, the loss function and the multiplier can
151+
* easily be rewritten into the following equivalent numerically stable formula.
152+
*
153+
* <blockquote>
154+
* $$
155+
* \ell\left(\beta, x\right) = log\left(\sum_{k=0}^{K-1} e^{margins_k - maxMargin}\right) -
156+
* margins_{y} + maxMargin
157+
* $$
158+
* </blockquote>
159+
*
160+
* Note that each term, $(margins_k - maxMargin)$ in the exponential is no greater than zero; as a
161+
* result, overflow will not happen with this formula.
162+
*
163+
* For $multiplier$, a similar trick can be applied as the following,
164+
*
165+
* <blockquote>
166+
* $$
167+
* multiplier_k = \left(\frac{e^{\vec{x}_i \cdot \vec{\beta}_k - maxMargin}}{\sum_{k'=0}^{K-1}
168+
* e^{\vec{x}_i \cdot \vec{\beta}_{k'} - maxMargin}} - I_{y=k}\right)
169+
* $$
170+
* </blockquote>
171+
*
172+
*
173+
* @param bcCoefficients The broadcast coefficients corresponding to the features.
174+
* @param bcFeaturesStd The broadcast standard deviation values of the features.
175+
* @param numClasses the number of possible outcomes for k classes classification problem in
176+
* Multinomial Logistic Regression.
177+
* @param fitIntercept Whether to fit an intercept term.
178+
* @param multinomial Whether to use multinomial (softmax) or binary loss
179+
* @note In order to avoid unnecessary computation during calculation of the gradient updates
180+
* we lay out the coefficients in column major order during training. This allows us to
181+
* perform feature standardization once, while still retaining sequential memory access
182+
* for speed. We convert back to row major order when we create the model,
183+
* since this form is optimal for the matrix operations used for prediction.
184+
*/
185+
private[ml] class LogisticAggregator(
186+
bcFeaturesStd: Broadcast[Array[Double]],
187+
numClasses: Int,
188+
fitIntercept: Boolean,
189+
multinomial: Boolean)(bcCoefficients: Broadcast[Vector])
190+
extends DifferentiableLossAggregator[Instance, LogisticAggregator] with Logging {
191+
192+
private val numFeatures = bcFeaturesStd.value.length
193+
private val numFeaturesPlusIntercept = if (fitIntercept) numFeatures + 1 else numFeatures
194+
private val coefficientSize = bcCoefficients.value.size
195+
protected override val dim: Int = coefficientSize
196+
if (multinomial) {
197+
require(numClasses == coefficientSize / numFeaturesPlusIntercept, s"The number of " +
198+
s"coefficients should be ${numClasses * numFeaturesPlusIntercept} but was $coefficientSize")
199+
} else {
200+
require(coefficientSize == numFeaturesPlusIntercept, s"Expected $numFeaturesPlusIntercept " +
201+
s"coefficients but got $coefficientSize")
202+
require(numClasses == 1 || numClasses == 2, s"Binary logistic aggregator requires numClasses " +
203+
s"in {1, 2} but found $numClasses.")
204+
}
205+
206+
@transient private lazy val coefficientsArray: Array[Double] = bcCoefficients.value match {
207+
case DenseVector(values) => values
208+
case _ => throw new IllegalArgumentException(s"coefficients only supports dense vector but " +
209+
s"got type ${bcCoefficients.value.getClass}.)")
210+
}
211+
212+
if (multinomial && numClasses <= 2) {
213+
logInfo(s"Multinomial logistic regression for binary classification yields separate " +
214+
s"coefficients for positive and negative classes. When no regularization is applied, the" +
215+
s"result will be effectively the same as binary logistic regression. When regularization" +
216+
s"is applied, multinomial loss will produce a result different from binary loss.")
217+
}
218+
219+
/** Update gradient and loss using binary loss function. */
220+
private def binaryUpdateInPlace(features: Vector, weight: Double, label: Double): Unit = {
221+
222+
val localFeaturesStd = bcFeaturesStd.value
223+
val localCoefficients = coefficientsArray
224+
val localGradientArray = gradientSumArray
225+
val margin = - {
226+
var sum = 0.0
227+
features.foreachActive { (index, value) =>
228+
if (localFeaturesStd(index) != 0.0 && value != 0.0) {
229+
sum += localCoefficients(index) * value / localFeaturesStd(index)
230+
}
231+
}
232+
if (fitIntercept) sum += localCoefficients(numFeaturesPlusIntercept - 1)
233+
sum
234+
}
235+
236+
val multiplier = weight * (1.0 / (1.0 + math.exp(margin)) - label)
237+
238+
features.foreachActive { (index, value) =>
239+
if (localFeaturesStd(index) != 0.0 && value != 0.0) {
240+
localGradientArray(index) += multiplier * value / localFeaturesStd(index)
241+
}
242+
}
243+
244+
if (fitIntercept) {
245+
localGradientArray(numFeaturesPlusIntercept - 1) += multiplier
246+
}
247+
248+
if (label > 0) {
249+
// The following is equivalent to log(1 + exp(margin)) but more numerically stable.
250+
lossSum += weight * MLUtils.log1pExp(margin)
251+
} else {
252+
lossSum += weight * (MLUtils.log1pExp(margin) - margin)
253+
}
254+
}
255+
256+
/** Update gradient and loss using multinomial (softmax) loss function. */
257+
private def multinomialUpdateInPlace(features: Vector, weight: Double, label: Double): Unit = {
258+
// TODO: use level 2 BLAS operations
259+
/*
260+
Note: this can still be used when numClasses = 2 for binary
261+
logistic regression without pivoting.
262+
*/
263+
val localFeaturesStd = bcFeaturesStd.value
264+
val localCoefficients = coefficientsArray
265+
val localGradientArray = gradientSumArray
266+
267+
// marginOfLabel is margins(label) in the formula
268+
var marginOfLabel = 0.0
269+
var maxMargin = Double.NegativeInfinity
270+
271+
val margins = new Array[Double](numClasses)
272+
features.foreachActive { (index, value) =>
273+
val stdValue = value / localFeaturesStd(index)
274+
var j = 0
275+
while (j < numClasses) {
276+
margins(j) += localCoefficients(index * numClasses + j) * stdValue
277+
j += 1
278+
}
279+
}
280+
var i = 0
281+
while (i < numClasses) {
282+
if (fitIntercept) {
283+
margins(i) += localCoefficients(numClasses * numFeatures + i)
284+
}
285+
if (i == label.toInt) marginOfLabel = margins(i)
286+
if (margins(i) > maxMargin) {
287+
maxMargin = margins(i)
288+
}
289+
i += 1
290+
}
291+
292+
/**
293+
* When maxMargin is greater than 0, the original formula could cause overflow.
294+
* We address this by subtracting maxMargin from all the margins, so it's guaranteed
295+
* that all of the new margins will be smaller than zero to prevent arithmetic overflow.
296+
*/
297+
val multipliers = new Array[Double](numClasses)
298+
val sum = {
299+
var temp = 0.0
300+
var i = 0
301+
while (i < numClasses) {
302+
if (maxMargin > 0) margins(i) -= maxMargin
303+
val exp = math.exp(margins(i))
304+
temp += exp
305+
multipliers(i) = exp
306+
i += 1
307+
}
308+
temp
309+
}
310+
311+
margins.indices.foreach { i =>
312+
multipliers(i) = multipliers(i) / sum - (if (label == i) 1.0 else 0.0)
313+
}
314+
features.foreachActive { (index, value) =>
315+
if (localFeaturesStd(index) != 0.0 && value != 0.0) {
316+
val stdValue = value / localFeaturesStd(index)
317+
var j = 0
318+
while (j < numClasses) {
319+
localGradientArray(index * numClasses + j) += weight * multipliers(j) * stdValue
320+
j += 1
321+
}
322+
}
323+
}
324+
if (fitIntercept) {
325+
var i = 0
326+
while (i < numClasses) {
327+
localGradientArray(numFeatures * numClasses + i) += weight * multipliers(i)
328+
i += 1
329+
}
330+
}
331+
332+
val loss = if (maxMargin > 0) {
333+
math.log(sum) - marginOfLabel + maxMargin
334+
} else {
335+
math.log(sum) - marginOfLabel
336+
}
337+
lossSum += weight * loss
338+
}
339+
340+
/**
341+
* Add a new training instance to this LogisticAggregator, and update the loss and gradient
342+
* of the objective function.
343+
*
344+
* @param instance The instance of data point to be added.
345+
* @return This LogisticAggregator object.
346+
*/
347+
def add(instance: Instance): this.type = {
348+
instance match { case Instance(label, weight, features) =>
349+
require(numFeatures == features.size, s"Dimensions mismatch when adding new instance." +
350+
s" Expecting $numFeatures but got ${features.size}.")
351+
require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0")
352+
353+
if (weight == 0.0) return this
354+
355+
if (multinomial) {
356+
multinomialUpdateInPlace(features, weight, label)
357+
} else {
358+
binaryUpdateInPlace(features, weight, label)
359+
}
360+
weightSum += weight
361+
this
362+
}
363+
}
364+
}

0 commit comments

Comments
 (0)