Skip to content

Conversation

@Andrew-Crosby
Copy link
Contributor

@Andrew-Crosby Andrew-Crosby commented Jun 15, 2019

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 #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.

…s added

Follows approach used in LeastSquaresAggregator
@Andrew-Crosby
Copy link
Contributor Author

@yanboliang @sethah does this look reasonable to you?

0.0
}
// make transient so we do not serialize between aggregation stages
@transient private lazy val featuresStd = bcFeaturesStd.value
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think this change makes sense. It is just getting the broadcast, not a big overhead...

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, this one isn't necessary. coefficients looks good. But it doesn't need to be lazy.

Copy link
Contributor Author

@Andrew-Crosby Andrew-Crosby Jun 18, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the feedback. I've removed the unnecessary change to featuresStd.

@srowen I tried removing the lazy modifier, but that causes both the unit tests and my test case to fail with the following NPE. I don't understand why.

org.apache.spark.SparkException: Job aborted due to stage failure: Task 2 in stage 3.0 failed 1 times, most recent failure: Lost task 2.0 in stage 3.0 (TID 11, localhost, executor driver): java.lang.NullPointerException
        at org.apache.spark.ml.optim.aggregator.HuberAggregator.$anonfun$add$3(HuberAggregator.scala:109)
        at org.apache.spark.ml.linalg.SparseVector.foreachActive(Vectors.scala:613)
        at org.apache.spark.ml.optim.aggregator.HuberAggregator.add(HuberAggregator.scala:107)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh I get it. You wouldn't want to eagerly evaluate the broadcast as it might eval on the driver. OK I think this is a reasonable solution.

@mgaido91
Copy link
Contributor

LGTM, @srowen may you please start the CI? Thanks.

@SparkQA
Copy link

SparkQA commented Jun 18, 2019

Test build #4800 has finished for PR 24880 at commit cdb49c0.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@srowen
Copy link
Member

srowen commented Jun 19, 2019

Merged to master

@srowen srowen closed this in 36b327d Jun 19, 2019
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants