-
Notifications
You must be signed in to change notification settings - Fork 28.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-5596] [mllib] ML model import/export for GLMs, NaiveBayes #4233
Conversation
Test build #26190 has started for PR 4233 at commit
|
Test build #26190 has finished for PR 4233 at commit
|
Test PASSed. |
14711b7
to
365314f
Compare
Test build #26427 has started for PR 4233 at commit
|
Test build #26427 has finished for PR 4233 at commit
|
Test PASSed. |
|
||
override def save(sc: SparkContext, path: String): Unit = { | ||
val sqlContext = new SQLContext(sc) | ||
import sqlContext._ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
import sqlContext._
is no longer needed due to recent API change. implicit val sqlContext = new SQLContext(sc)
should work.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I tried that, and it fails to compile. (I tried removed sqlContext from save(), as well as having it be an implicit val without the import. Neither worked.) Is there another import I need?
[error] /Users/josephkb/spark/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala:89: type mismatch;
[error] found : org.apache.spark.rdd.RDD[org.apache.spark.mllib.classification.LogisticRegressionModel.Metadata]
[error] required: org.apache.spark.sql.DataFrame
[error] Error occurred in an application involving default arguments.
[error] val metadataRDD: DataFrame = sc.parallelize(Seq(metadata))
[error] ^
[error] /Users/josephkb/spark/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala:94: type mismatch;
[error] found : org.apache.spark.rdd.RDD[org.apache.spark.mllib.classification.LogisticRegressionModel.Data]
[error] required: org.apache.spark.sql.DataFrame
[error] Error occurred in an application involving default arguments.
[error] val dataRDD: DataFrame = sc.parallelize(Seq(data))
[error] ^
[error] two errors found
[error] (mllib/compile:compile) Compilation failed
It may not be an issue though, with the other change you suggested below. I'll see.
Test build #26459 has started for PR 4233 at commit
|
Test build #26459 has finished for PR 4233 at commit
|
Test FAILed. |
Test build #26579 has started for PR 4233 at commit
|
Test build #26579 has finished for PR 4233 at commit
|
Test FAILed. |
Test build #26580 has started for PR 4233 at commit
|
Test build #26580 has finished for PR 4233 at commit
|
Test FAILed. |
|
||
override def load(sc: SparkContext, path: String): LogisticRegressionModel = { | ||
val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) | ||
val classNameV1_0 = "org.apache.spark.mllib.classification.LogisticRegressionModel" |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe we should put a comment here about why using literal string name.
Btw, I'm also changing paths to use org.apache.hadoop.fs.Path to create URIs (instead of my hard-coded path separators). |
…numClasses in model metadata. Improvements to unit tests
Test build #26784 has started for PR 4233 at commit
|
Hopefully that took care of everything! |
Test build #26786 has started for PR 4233 at commit
|
Test build #26786 has finished for PR 4233 at commit
|
Test FAILed. |
Test build #26784 has finished for PR 4233 at commit
|
Test PASSed. |
The first failure was from Kafka tests |
This is a PR for Parquet-based model import/export. Please see the design doc on [the JIRA](https://issues.apache.org/jira/browse/SPARK-4587). Note: This includes only a subset of regression and classification models: * NaiveBayes, SVM, LogisticRegression * LinearRegression, RidgeRegression, Lasso Follow-up PRs will cover other models. Sketch of current contents: * New traits: Saveable, Loader * Implementations for some algorithms * Also: Added LogisticRegressionModel.getThreshold method (so that unit test could check the threshold) CC: mengxr selvinsource Author: Joseph K. Bradley <joseph@databricks.com> Closes #4233 from jkbradley/ml-import-export and squashes the following commits: 87c4eb8 [Joseph K. Bradley] small cleanups 12d9059 [Joseph K. Bradley] Many cleanups after code review. Major changes: Storing numFeatures, numClasses in model metadata. Improvements to unit tests b4ee064 [Joseph K. Bradley] Reorganized save/load for regression and classification. Renamed concepts to Saveable, Loader a34aef5 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into ml-import-export ee99228 [Joseph K. Bradley] scala style fix 79675d5 [Joseph K. Bradley] cleanups in LogisticRegression after rebasing after multinomial PR d1e5882 [Joseph K. Bradley] organized imports 2935963 [Joseph K. Bradley] Added save/load and tests for most classification and regression models c495dba [Joseph K. Bradley] made version for model import/export local to each model 1496852 [Joseph K. Bradley] Added save/load for NaiveBayes 8d46386 [Joseph K. Bradley] Added save/load to NaiveBayes 1577d70 [Joseph K. Bradley] fixed issues after rebasing on master (DataFrame patch) 64914a3 [Joseph K. Bradley] added getThreshold to SVMModel b1fc5ec [Joseph K. Bradley] small cleanups 418ba1b [Joseph K. Bradley] Added save, load to mllib.classification.LogisticRegressionModel, plus test suite (cherry picked from commit 975bcef) Signed-off-by: Xiangrui Meng <meng@databricks.com>
LGTM. Merged into master and branch-1.3. Thanks! |
following #4233. jkbradley Author: Xiangrui Meng <meng@databricks.com> Closes #4422 from mengxr/SPARK-5598 and squashes the following commits: a059394 [Xiangrui Meng] SaveLoad not extending Loader 14b7ea6 [Xiangrui Meng] address comments f487cb2 [Xiangrui Meng] add unit tests 62fc43c [Xiangrui Meng] implement save/load for MFM (cherry picked from commit 5c299c5) Signed-off-by: Xiangrui Meng <meng@databricks.com>
following #4233. jkbradley Author: Xiangrui Meng <meng@databricks.com> Closes #4422 from mengxr/SPARK-5598 and squashes the following commits: a059394 [Xiangrui Meng] SaveLoad not extending Loader 14b7ea6 [Xiangrui Meng] address comments f487cb2 [Xiangrui Meng] add unit tests 62fc43c [Xiangrui Meng] implement save/load for MFM
This is a PR for Parquet-based model import/export. Please see the design doc on the JIRA.
Note: This includes only a subset of regression and classification models:
Follow-up PRs will cover other models.
Sketch of current contents:
CC: @mengxr @selvinsource