Skip to content

Commit 36b327d

Browse files
Andrew-Crosbysrowen
authored andcommitted
[SPARK-28062][ML] Avoid unnecessary copy of coefficients vector in HuberAggregator
## What changes were proposed in this pull request? Modifies the HuberAggregator class so that a copy of the coefficients vector isn't created every time that an instance is added. Follows the approach of LeastSquaresAggregator and uses transient lazy class variable to store the reused quantities. (See apache#14109 for explanation of the use of transient lazy variables) On the test case in the linked JIRA, this change gives an order of magnitude performance improvement reducing the time taken to fit the model from 540 to 47 seconds. ## How was this patch tested? Existing unit tests. See https://issues.apache.org/jira/browse/SPARK-28062 for results from running a benchmark script. Closes apache#24880 from Andrew-Crosby/spark-28062. Authored-by: Andrew-Crosby <[email protected]> Signed-off-by: Sean Owen <[email protected]>
1 parent 9ec0496 commit 36b327d

File tree

1 file changed

+3
-1
lines changed

1 file changed

+3
-1
lines changed

mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -81,6 +81,8 @@ private[ml] class HuberAggregator(
8181
} else {
8282
0.0
8383
}
84+
// make transient so we do not serialize between aggregation stages
85+
@transient private lazy val coefficients = bcParameters.value.toArray.slice(0, numFeatures)
8486

8587
/**
8688
* Add a new training instance to this HuberAggregator, and update the loss and gradient
@@ -97,7 +99,7 @@ private[ml] class HuberAggregator(
9799

98100
if (weight == 0.0) return this
99101
val localFeaturesStd = bcFeaturesStd.value
100-
val localCoefficients = bcParameters.value.toArray.slice(0, numFeatures)
102+
val localCoefficients = coefficients
101103
val localGradientSumArray = gradientSumArray
102104

103105
val margin = {

0 commit comments

Comments
 (0)