-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-11846] Add save/load for AFTSurvivalRegression and IsotonicRegression #9836
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
Closed
Closed
Changes from 4 commits
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
4f38770
add AFTSurvivialRegression save/load
yinxusen 9a2c72c
add Isotonic regression save/load
yinxusen fa1d84b
add params
yinxusen c784fab
fix misspell
yinxusen c53ddb2
fix style issue
yinxusen a9dcb8a
minor fix
yinxusen File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,18 +17,24 @@ | |
|
|
||
| package org.apache.spark.ml.regression | ||
|
|
||
| import scala.collection.mutable | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
|
|
||
| import org.apache.spark.Logging | ||
| import org.apache.spark.annotation.{Experimental, Since} | ||
| import org.apache.spark.ml.{Estimator, Model} | ||
| import org.apache.spark.ml.param._ | ||
| import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasLabelCol, HasPredictionCol, HasWeightCol} | ||
| import org.apache.spark.ml.util.{Identifiable, SchemaUtils} | ||
| import org.apache.spark.ml.param.shared._ | ||
| import org.apache.spark.ml.regression.IsotonicRegressionModel.IsotonicRegressionModelWriter | ||
| import org.apache.spark.ml.util._ | ||
| import org.apache.spark.ml.{Estimator, Model} | ||
| import org.apache.spark.mllib.linalg.{Vector, VectorUDT, Vectors} | ||
| import org.apache.spark.mllib.regression.{IsotonicRegression => MLlibIsotonicRegression, IsotonicRegressionModel => MLlibIsotonicRegressionModel} | ||
| import org.apache.spark.mllib.regression.{IsotonicRegression => MLlibIsotonicRegression} | ||
| import org.apache.spark.mllib.regression.{IsotonicRegressionModel => MLlibIsotonicRegressionModel} | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.{DataFrame, Row} | ||
| import org.apache.spark.sql.functions.{col, lit, udf} | ||
| import org.apache.spark.sql.types.{DoubleType, StructType} | ||
| import org.apache.spark.sql.{DataFrame, Row} | ||
| import org.apache.spark.storage.StorageLevel | ||
|
|
||
| /** | ||
|
|
@@ -127,7 +133,8 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures | |
| @Since("1.5.0") | ||
| @Experimental | ||
| class IsotonicRegression @Since("1.5.0") (@Since("1.5.0") override val uid: String) | ||
| extends Estimator[IsotonicRegressionModel] with IsotonicRegressionBase { | ||
| extends Estimator[IsotonicRegressionModel] | ||
| with IsotonicRegressionBase with DefaultParamsWritable { | ||
|
|
||
| @Since("1.5.0") | ||
| def this() = this(Identifiable.randomUID("isoReg")) | ||
|
|
@@ -179,6 +186,13 @@ class IsotonicRegression @Since("1.5.0") (@Since("1.5.0") override val uid: Stri | |
| } | ||
| } | ||
|
|
||
| @Since("1.6.0") | ||
| object IsotonicRegression extends DefaultParamsReadable[IsotonicRegression] { | ||
|
|
||
| @Since("1.6.0") | ||
| override def load(path: String): IsotonicRegression = super.load(path) | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Model fitted by IsotonicRegression. | ||
|
|
@@ -194,7 +208,7 @@ class IsotonicRegression @Since("1.5.0") (@Since("1.5.0") override val uid: Stri | |
| class IsotonicRegressionModel private[ml] ( | ||
| override val uid: String, | ||
| private val oldModel: MLlibIsotonicRegressionModel) | ||
| extends Model[IsotonicRegressionModel] with IsotonicRegressionBase { | ||
| extends Model[IsotonicRegressionModel] with IsotonicRegressionBase with MLWritable { | ||
|
|
||
| /** @group setParam */ | ||
| @Since("1.5.0") | ||
|
|
@@ -240,4 +254,61 @@ class IsotonicRegressionModel private[ml] ( | |
| override def transformSchema(schema: StructType): StructType = { | ||
| validateAndTransformSchema(schema, fitting = false) | ||
| } | ||
|
|
||
| @Since("1.6.0") | ||
| override def write: MLWriter = | ||
| new IsotonicRegressionModelWriter(this) | ||
| } | ||
|
|
||
| @Since("1.6.0") | ||
| object IsotonicRegressionModel extends MLReadable[IsotonicRegressionModel] { | ||
|
|
||
| @Since("1.6.0") | ||
| override def read: MLReader[IsotonicRegressionModel] = new IsotonicRegressionModelReader | ||
|
|
||
| @Since("1.6.0") | ||
| override def load(path: String): IsotonicRegressionModel = super.load(path) | ||
|
|
||
| /** [[MLWriter]] instance for [[IsotonicRegressionModel]] */ | ||
| private[IsotonicRegressionModel] class IsotonicRegressionModelWriter ( | ||
| instance: IsotonicRegressionModel | ||
| ) extends MLWriter with Logging { | ||
|
|
||
| private case class Data( | ||
| boundaries: Array[Double], | ||
| predictions: Array[Double], | ||
| isotonic: Boolean) | ||
|
|
||
| override protected def saveImpl(path: String): Unit = { | ||
| // Save metadata and Params | ||
| DefaultParamsWriter.saveMetadata(instance, path, sc) | ||
| // Save model data: boundaries, predictions, isotonic | ||
| val data = Data( | ||
| instance.oldModel.boundaries, instance.oldModel.predictions, instance.oldModel.isotonic) | ||
| val dataPath = new Path(path, "data").toString | ||
| sqlContext.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) | ||
| } | ||
| } | ||
|
|
||
| private class IsotonicRegressionModelReader extends MLReader[IsotonicRegressionModel] { | ||
|
|
||
| /** Checked against metadata when loading model */ | ||
| private val className = classOf[IsotonicRegressionModel].getName | ||
|
|
||
| override def load(path: String): IsotonicRegressionModel = { | ||
| val metadata = DefaultParamsReader.loadMetadata(path, sc, className) | ||
|
|
||
| val dataPath = new Path(path, "data").toString | ||
| val data = sqlContext.read.format("parquet").load(dataPath) | ||
| .select("boundaries", "predictions", "isotonic").head() | ||
| val boundaries = data.getAs[mutable.WrappedArray[Double]](0).toArray | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| val predictions = data.getAs[mutable.WrappedArray[Double]](1).toArray | ||
| val isotonic = data.getBoolean(2) | ||
| val model = new IsotonicRegressionModel( | ||
| metadata.uid, new MLlibIsotonicRegressionModel(boundaries, predictions, isotonic)) | ||
|
|
||
| DefaultParamsReader.getAndSetParams(model, metadata) | ||
| model | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
minor:
sqlContext.read.parquet(dataPath)