-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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-11237][ML] Add pmml export for k-means in Spark ML #20907
Closed
holdenk
wants to merge
5
commits into
apache:master
from
holdenk:SPARK-11237-Add-PMML-Export-for-KMeans
Closed
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
4431407
Switch KMeans to GeneralMLWriter and add the internal writer.
holdenk 25d6f77
Add PMML export for KMeans model
holdenk e2dbe15
Call testPMMLWrite to actually verify PMML export
holdenk 3d529f8
Merge branch 'master' into SPARK-11237-Add-PMML-Export-for-KMeans
holdenk e0f9b09
Check the actual model
holdenk 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
4 changes: 3 additions & 1 deletion
4
mllib/src/main/resources/META-INF/services/org.apache.spark.ml.util.MLFormatRegister
This file contains 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 |
---|---|---|
@@ -1,2 +1,4 @@ | ||
org.apache.spark.ml.regression.InternalLinearRegressionModelWriter | ||
org.apache.spark.ml.regression.PMMLLinearRegressionModelWriter | ||
org.apache.spark.ml.regression.PMMLLinearRegressionModelWriter | ||
org.apache.spark.ml.clustering.InternalKMeansModelWriter | ||
org.apache.spark.ml.clustering.PMMLKMeansModelWriter |
This file contains 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 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 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 |
---|---|---|
|
@@ -19,17 +19,22 @@ package org.apache.spark.ml.clustering | |
|
||
import scala.util.Random | ||
|
||
import org.dmg.pmml.{ClusteringModel, PMML} | ||
|
||
import org.apache.spark.{SparkException, SparkFunSuite} | ||
import org.apache.spark.ml.linalg.{Vector, Vectors} | ||
import org.apache.spark.ml.param.ParamMap | ||
import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} | ||
import org.apache.spark.mllib.clustering.{DistanceMeasure, KMeans => MLlibKMeans} | ||
import org.apache.spark.ml.util._ | ||
import org.apache.spark.mllib.clustering.{DistanceMeasure, KMeans => MLlibKMeans, | ||
KMeansModel => MLlibKMeansModel} | ||
import org.apache.spark.mllib.linalg.{Vectors => MLlibVectors} | ||
import org.apache.spark.mllib.util.MLlibTestSparkContext | ||
import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} | ||
|
||
private[clustering] case class TestRow(features: Vector) | ||
|
||
class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { | ||
class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest | ||
with PMMLReadWriteTest { | ||
|
||
final val k = 5 | ||
@transient var dataset: Dataset[_] = _ | ||
|
@@ -202,6 +207,27 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultR | |
testEstimatorAndModelReadWrite(kmeans, dataset, KMeansSuite.allParamSettings, | ||
KMeansSuite.allParamSettings, checkModelData) | ||
} | ||
|
||
test("pmml export") { | ||
val clusterCenters = Array( | ||
MLlibVectors.dense(1.0, 2.0, 6.0), | ||
MLlibVectors.dense(1.0, 3.0, 0.0), | ||
MLlibVectors.dense(1.0, 4.0, 6.0)) | ||
val oldKmeansModel = new MLlibKMeansModel(clusterCenters) | ||
val kmeansModel = new KMeansModel("", oldKmeansModel) | ||
def checkModel(pmml: PMML): Unit = { | ||
// Check the header descripiton is what we expect | ||
assert(pmml.getHeader.getDescription === "k-means clustering") | ||
// check that the number of fields match the single vector size | ||
assert(pmml.getDataDictionary.getNumberOfFields === clusterCenters(0).size) | ||
// This verify that there is a model attached to the pmml object and the model is a clustering | ||
// one. It also verifies that the pmml model has the same number of clusters of the spark | ||
// model. | ||
val pmmlClusteringModel = pmml.getModels.get(0).asInstanceOf[ClusteringModel] | ||
assert(pmmlClusteringModel.getNumberOfClusters === clusterCenters.length) | ||
} | ||
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. Isn't this missing a call to 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. Oh yeah :( Thanks for catching that. |
||
testPMMLWrite(sc, kmeansModel, checkModel) | ||
} | ||
} | ||
|
||
object KMeansSuite { | ||
|
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.
doesn't this type change
Data
->ClusterData
change the schema of the output parquet file?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'm not 100% sure. I'll manually test we can load the old format first.
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.
Wait no this shouldn't change anything, were saving this with a DataFrame and the schema is the same.
See the schema from 1:
res3: org.apache.spark.sql.types.StructType = StructType(StructField(clusterIdx,IntegerType,false), StructField(clusterCenter,org.apache.spark.ml.linalg.VectorUDT@3bfc3ba7,true))
and the new oneorg.apache.spark.sql.types.StructType = StructType(StructField(clusterIdx,IntegerType,false), StructField(clusterCenter,org.apache.spark.ml.linalg.VectorUDT@3bfc3ba7,true))
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.
👍