Skip to content

Conversation

@viirya
Copy link
Member

@viirya viirya commented Feb 2, 2017

What changes were proposed in this pull request?

For a Pipeline including long stages, the iterative fit and transform cause extremely grown query plans and RDD lineages, it takes longer time to finish the fit and transform.

This patch introduces PeriodicDatasetCheckpointer to do periodic checkout for dataset used in fit and transform.

This introduces new param checkpointInterval to Pipeline and PipelineModel. Once it is set, we will do periodic checkout by PeriodicDatasetCheckpointer.

As there is existing trait HasCheckpointInterval which already defines checkpointInterval param. This patch lets Pipeline and PipelineModel extend HasCheckpointInterval.

Benchmark

Run the following codes locally.

import org.apache.spark.ml.{Pipeline, PipelineStage}
import org.apache.spark.ml.feature.{OneHotEncoder, StringIndexer, VectorAssembler}

spark.sparkContext.setCheckpointDir("/tmp/checkpoint")

val df = (1 to 40).foldLeft(Seq((1, "foo"), (2, "bar"), (3, "baz")).toDF("id", "x0"))((df, i) => df.withColumn(s"x$i", $"x0"))

val indexers = df.columns.tail.map(c => new StringIndexer()
  .setInputCol(c)
  .setOutputCol(s"${c}_indexed")
  .setHandleInvalid("skip"))

val encoders = indexers.map(indexer => new OneHotEncoder()
  .setInputCol(indexer.getOutputCol)
  .setOutputCol(s"${indexer.getOutputCol}_encoded")
  .setDropLast(true))

val stages: Array[PipelineStage] = indexers ++ encoders
val pipeline = new Pipeline().setStages(stages)
pipeline.setCheckpointInterval(5)  // only run this line after applying this patch

val startTime = System.nanoTime
pipeline.fit(df).transform(df).show
val runningTime = System.nanoTime - startTime

Before this patch: 1786001 ms
After this patch: 69013 ms

This issue is originally reported at http://apache-spark-developers-list.1001551.n3.nabble.com/SQL-ML-Pipeline-performance-regression-between-1-6-and-2-x-tc20803.html

How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

@viirya
Copy link
Member Author

viirya commented Feb 2, 2017

cc @mengxr @jkbradley @liancheng

@viirya
Copy link
Member Author

viirya commented Feb 2, 2017

also cc @MLnick

@viirya viirya changed the title [WIP][ML] Periodic checkout datasets for long ml pipeline [SPARK-19433][ML] Periodic checkout datasets for long ml pipeline Feb 2, 2017
@SparkQA
Copy link

SparkQA commented Feb 2, 2017

Test build #72274 has finished for PR 16775 at commit 5ed5c2a.

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

@SparkQA
Copy link

SparkQA commented Feb 2, 2017

Test build #72275 has started for PR 16775 at commit 7a1b300.

@SparkQA
Copy link

SparkQA commented Feb 2, 2017

Test build #72276 has started for PR 16775 at commit 32c90dd.

@viirya
Copy link
Member Author

viirya commented Feb 2, 2017

retest this please.

@SparkQA
Copy link

SparkQA commented Feb 2, 2017

Test build #72277 has finished for PR 16775 at commit 32c90dd.

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

@DavidArenburg
Copy link

DavidArenburg commented Feb 2, 2017

Wouldn't it better to Vectorize StringIndexer and OneHotEncoder? Like for instance .na.fill or .na.replace operate over the whole data set at once instead of running it in a loop (or at least I think it works this way)? Or similar to how model.matrix works in R. I feel like even with this patch this isn't scalable on lets say 1MM covariates (unless I'm missing something)- and yes, fitting a model on 1MM covariates while using LASSO for feature selection is quite common. If I'm not missing something, as it stands, looping StringIndexer and OneHotEncoder isn't executed at all nodes/cores at the same time- while it should be.

@viirya
Copy link
Member Author

viirya commented Feb 2, 2017

StringIndexer and OneHotEncoder are just used as example here. The concept is to have a pipeline with enough long stages.

@viirya
Copy link
Member Author

viirya commented Feb 3, 2017

For the issue reported on mailing list, I found the root cause makes significant difference between 1.6 and current branch. The fix is at #16785.

However, I think this patch is still useful. So I keep it open for a while for reviewers.

@viirya
Copy link
Member Author

viirya commented Feb 9, 2017

ping @mengxr @jkbradley @liancheng @MLnick May you take a look at this? Thanks.

@mallman
Copy link
Contributor

mallman commented Feb 9, 2017

@viirya I believe this PR meshes with the refactoring and application to pregel GraphX algorithms in #15125. Basically, it moves the periodic checkpointing code from mllib into core and uses it in GraphX to checkpoint long lineages. This is essential to scale GraphX to huge graphs, as described in my comment in the PR, and solves a very real problem for us. Can you take a look at that PR?

@viirya
Copy link
Member Author

viirya commented Feb 23, 2017

I think we can solve this issue by tackling the codes in SQL. So close it for now.

@viirya viirya closed this Feb 23, 2017
@viirya viirya deleted the periodic-checkout-for-long-ml-pipeline branch December 27, 2023 18:34
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants