From a0e36797cd771da1130d0ddcdedaec44b2e6357e Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sun, 12 Oct 2014 13:30:49 +0100 Subject: [PATCH 01/41] export and pmml export traits kmeans test implementation --- mllib/pom.xml | 5 +++ .../spark/mllib/export/ModelExport.scala | 26 +++++++++++++ .../mllib/export/ModelExportFactory.scala | 32 ++++++++++++++++ .../spark/mllib/export/ModelExportType.scala | 25 +++++++++++++ .../export/pmml/KMeansPMMLModelExport.scala | 30 +++++++++++++++ .../mllib/export/pmml/PMMLModelExport.scala | 37 +++++++++++++++++++ 6 files changed, 155 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala diff --git a/mllib/pom.xml b/mllib/pom.xml index a5eeef88e9d62..cfeabe4025de6 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -98,6 +98,11 @@ test-jar test + + org.jpmml + pmml-model + 1.1.7 + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala new file mode 100644 index 0000000000000..274366208bd36 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.export + +import java.io.OutputStream + +trait ModelExport { + + def save(outputStream: OutputStream): Unit + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala new file mode 100644 index 0000000000000..afce4e305aaac --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.export + +import org.apache.spark.mllib.clustering.KMeansModel +import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport + +object ModelExportFactory { + + //TODO: introduce model export typed + + def createModelExport(model: Any): ModelExport = model match { + case kmeans: KMeansModel => new KMeansPMMLModelExport + case _ => throw new IllegalArgumentException("Export not supported for model " + model.getClass) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala new file mode 100644 index 0000000000000..5abb7d6bb4e71 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.export + +object ModelExportType extends Enumeration{ + + type ModelExportType = Value + val PMML = Value + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala new file mode 100644 index 0000000000000..f53443e3e646d --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.export.pmml + +class KMeansPMMLModelExport extends PMMLModelExport{ + + populateKMeansPMML(); + + def populateKMeansPMML(): Unit = { + //TODO: set here header description + pmml.setVersion("testing... kmeans..."); + //TODO: generate the model... + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala new file mode 100644 index 0000000000000..42203e6b9291a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.export.pmml + +import org.apache.spark.mllib.export.ModelExport +import java.io.OutputStream +import org.jpmml.model.JAXBUtil +import org.dmg.pmml.PMML +import javax.xml.transform.stream.StreamResult +import scala.beans.BeanProperty + +trait PMMLModelExport extends ModelExport{ + + @BeanProperty + var pmml: PMML = new PMML(); + //TODO: set here header app copyright and timestamp + + def save(outputStream: OutputStream): Unit = { + JAXBUtil.marshalPMML(pmml, new StreamResult(outputStream)); + } + +} From 226e1849ef6717fccd07122b13954471ff43cd28 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sat, 18 Oct 2014 15:03:45 +0100 Subject: [PATCH 02/41] added javadoc and export model type in case there is a need to support other types of export (not just PMML) --- .../spark/mllib/export/ModelExport.scala | 3 +++ .../mllib/export/ModelExportFactory.scala | 18 +++++++++++++----- .../spark/mllib/export/ModelExportType.scala | 4 ++++ .../export/pmml/KMeansPMMLModelExport.scala | 14 +++++++++++--- .../mllib/export/pmml/PMMLModelExport.scala | 7 +++++++ 5 files changed, 38 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala index 274366208bd36..f986c0cb95348 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala @@ -21,6 +21,9 @@ import java.io.OutputStream trait ModelExport { + /** + * Write the exported model to the output stream specified + */ def save(outputStream: OutputStream): Unit } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala index afce4e305aaac..7e2e76f53988c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala @@ -19,14 +19,22 @@ package org.apache.spark.mllib.export import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport +import org.apache.spark.mllib.export.ModelExportType._ object ModelExportFactory { - //TODO: introduce model export typed - - def createModelExport(model: Any): ModelExport = model match { - case kmeans: KMeansModel => new KMeansPMMLModelExport - case _ => throw new IllegalArgumentException("Export not supported for model " + model.getClass) + /** + * Factory object to help creating the necessary ModelExport implementation + * taking as input the ModelExportType (for example PMML) and the machine learning model (for example KMeansModel). + */ + def createModelExport(model: Any, exportType: ModelExportType): ModelExport = { + return exportType match{ + case PMML => model match{ + case kmeans: KMeansModel => new KMeansPMMLModelExport(kmeans) + case _ => throw new IllegalArgumentException("Export not supported for model: " + model.getClass) + } + case _ => throw new IllegalArgumentException("Export type not supported:" + exportType) + } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala index 5abb7d6bb4e71..1e940a6aa5e50 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala @@ -17,6 +17,10 @@ package org.apache.spark.mllib.export +/** + * Defines export types. + * - PMML exports the machine learning models in an XML-based file format called Predictive Model Markup Language developed by the Data Mining Group (www.dmg.org). + */ object ModelExportType extends Enumeration{ type ModelExportType = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala index f53443e3e646d..99ab256adfd0b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala @@ -17,11 +17,19 @@ package org.apache.spark.mllib.export.pmml -class KMeansPMMLModelExport extends PMMLModelExport{ +import org.apache.spark.mllib.clustering.KMeansModel - populateKMeansPMML(); +/** + * PMML Model Export for KMeansModel class + */ +class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ + + /** + * Export the input KMeansModel model to PMML format + */ + populateKMeansPMML(model); - def populateKMeansPMML(): Unit = { + private def populateKMeansPMML(model : KMeansModel): Unit = { //TODO: set here header description pmml.setVersion("testing... kmeans..."); //TODO: generate the model... diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala index 42203e6b9291a..6d8e8ff0797f6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala @@ -26,10 +26,17 @@ import scala.beans.BeanProperty trait PMMLModelExport extends ModelExport{ + /** + * Holder of the exported model in PMML format + */ @BeanProperty var pmml: PMML = new PMML(); //TODO: set here header app copyright and timestamp + /** + * Write the exported model (in PMML XML) to the output stream specified + */ + @Override def save(outputStream: OutputStream): Unit = { JAXBUtil.marshalPMML(pmml, new StreamResult(outputStream)); } From 9bc494fa30508d7cd6f2146b22c0dc77750c09d1 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sun, 26 Oct 2014 17:50:44 +0000 Subject: [PATCH 03/41] added scala suite tests added saveLocalFile to ModelExport trait --- .../spark/mllib/export/ModelExport.scala | 9 ++++ .../export/ModelExportFactorySuite.scala | 53 +++++++++++++++++++ .../pmml/KMeansPMMLModelExportSuite.scala | 51 ++++++++++++++++++ 3 files changed, 113 insertions(+) create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala index f986c0cb95348..c0fea6ad95d9b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala @@ -18,6 +18,8 @@ package org.apache.spark.mllib.export import java.io.OutputStream +import java.io.FileOutputStream +import java.io.File trait ModelExport { @@ -25,5 +27,12 @@ trait ModelExport { * Write the exported model to the output stream specified */ def save(outputStream: OutputStream): Unit + + /** + * Write the exported model to the local file specified + */ + def saveLocalFile(path: String): Unit = { + save(new FileOutputStream(new File(path))); + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala new file mode 100644 index 0000000000000..fc627fcb75584 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.export + +import org.apache.spark.mllib.clustering.KMeansModel +import org.apache.spark.mllib.linalg.Vectors +import org.scalatest.FunSuite +import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport + +class ModelExportFactorySuite extends FunSuite{ + + test("ModelExportFactory create KMeansPMMLModelExport when passing a KMeansModel") { + + val clusterCenters = Array( + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0) + ) + + val kmeansModel = new KMeansModel(clusterCenters); + + val modelExport = ModelExportFactory.createModelExport(kmeansModel, ModelExportType.PMML) + + assert(modelExport.isInstanceOf[KMeansPMMLModelExport]) + + } + + test("ModelExportFactory generate IllegalArgumentException when passing an unsupported model") { + + val invalidModel = new Object; + + intercept[IllegalArgumentException] { + ModelExportFactory.createModelExport(invalidModel, ModelExportType.PMML) + } + + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala new file mode 100644 index 0000000000000..02339b0e20e28 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.export.pmml + +import org.scalatest.FunSuite +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.export.ModelExportFactory +import org.apache.spark.mllib.clustering.KMeansModel +import org.apache.spark.mllib.export.ModelExportType + +class KMeansPMMLModelExportSuite extends FunSuite{ + + test("KMeansPMMLModelExport generate PMML format") { + + val clusterCenters = Array( + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0) + ) + + val kmeansModel = new KMeansModel(clusterCenters); + + val modelExport = ModelExportFactory.createModelExport(kmeansModel, ModelExportType.PMML) + + assert(modelExport.isInstanceOf[PMMLModelExport]) + + //TODO: asserts + //compare pmml fields to strings + modelExport.asInstanceOf[PMMLModelExport].getPmml() + //use document builder to load the xml generated and validated the notes by looking for them + modelExport.asInstanceOf[PMMLModelExport].save(System.out) + //saveLocalFile too??? search how to unit test file creating in java + + } + +} From 8e71b8d8cc3922e834fe0f21e4f05e773241dad7 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Mon, 27 Oct 2014 21:58:54 +0000 Subject: [PATCH 04/41] kmeans pmml export implementation --- .../export/pmml/KMeansPMMLModelExport.scala | 63 ++++++++++++++++++- .../mllib/export/pmml/PMMLModelExport.scala | 19 +++++- .../export/ModelExportFactorySuite.scala | 2 +- 3 files changed, 79 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala index 99ab256adfd0b..2f0af9a18f470 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala @@ -18,6 +18,24 @@ package org.apache.spark.mllib.export.pmml import org.apache.spark.mllib.clustering.KMeansModel +import org.dmg.pmml.DataDictionary +import org.dmg.pmml.FieldName +import org.dmg.pmml.DataField +import org.dmg.pmml.OpType +import org.dmg.pmml.DataType +import org.dmg.pmml.MiningSchema +import org.dmg.pmml.MiningField +import org.dmg.pmml.FieldUsageType +import org.dmg.pmml.ComparisonMeasure +import org.dmg.pmml.ComparisonMeasure.Kind +import org.dmg.pmml.SquaredEuclidean +import org.dmg.pmml.ClusteringModel +import org.dmg.pmml.MiningFunctionType +import org.dmg.pmml.ClusteringModel.ModelClass +import org.dmg.pmml.ClusteringField +import org.dmg.pmml.CompareFunctionType +import org.dmg.pmml.Cluster +import org.dmg.pmml.Array.Type /** * PMML Model Export for KMeansModel class @@ -30,9 +48,48 @@ class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ populateKMeansPMML(model); private def populateKMeansPMML(model : KMeansModel): Unit = { - //TODO: set here header description - pmml.setVersion("testing... kmeans..."); - //TODO: generate the model... + + pmml.getHeader().setDescription("k-means clustering"); + + if(model.clusterCenters.length > 0){ + + val clusterCenter = model.clusterCenters(0) + + var fields = new Array[FieldName](clusterCenter.size) + + var dataDictionary = new DataDictionary() + + var miningSchema = new MiningSchema() + + for ( i <- 0 to (clusterCenter.size - 1)) { + fields(i) = FieldName.create("field_"+i) + dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema.withMiningFields(new MiningField(fields(i)).withUsageType(FieldUsageType.ACTIVE)) + } + + var comparisonMeasure = new ComparisonMeasure() + .withKind(Kind.DISTANCE) + .withMeasure(new SquaredEuclidean() + ); + + dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()); + + pmml.setDataDictionary(dataDictionary); + + var clusteringModel = new ClusteringModel(miningSchema, comparisonMeasure, MiningFunctionType.CLUSTERING, ModelClass.CENTER_BASED, model.clusterCenters.length) + .withModelName("k-means"); + + for ( i <- 0 to (clusterCenter.size - 1)) { + clusteringModel.withClusteringFields(new ClusteringField(fields(i)).withCompareFunction(CompareFunctionType.ABS_DIFF)) + var cluster = new Cluster().withName("cluster_"+i).withArray(new org.dmg.pmml.Array().withType(Type.REAL).withN(clusterCenter.size).withValue(model.clusterCenters(i).toArray.mkString(" "))) + //cluster.withSize(value) //we don't have the size of the single cluster but only the centroids (withValue) + clusteringModel.withClusters(cluster) + } + + pmml.withModels(clusteringModel); + + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala index 6d8e8ff0797f6..c1e84f62f9223 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala @@ -23,6 +23,11 @@ import org.jpmml.model.JAXBUtil import org.dmg.pmml.PMML import javax.xml.transform.stream.StreamResult import scala.beans.BeanProperty +import org.dmg.pmml.Application +import org.dmg.pmml.Timestamp +import org.dmg.pmml.Header +import java.text.SimpleDateFormat +import java.util.Date trait PMMLModelExport extends ModelExport{ @@ -31,7 +36,19 @@ trait PMMLModelExport extends ModelExport{ */ @BeanProperty var pmml: PMML = new PMML(); - //TODO: set here header app copyright and timestamp + + setHeader(pmml); + + private def setHeader(pmml : PMML): Unit = { + var version = getClass().getPackage().getImplementationVersion() + var app = new Application().withName("Apache Spark MLlib").withVersion(version) + var timestamp = new Timestamp().withContent(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss").format(new Date())) + var header = new Header() + .withCopyright("www.dmg.org") + .withApplication(app) + .withTimestamp(timestamp); + pmml.setHeader(header); + } /** * Write the exported model (in PMML XML) to the output stream specified diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala index fc627fcb75584..9b6b4160d6120 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala @@ -40,7 +40,7 @@ class ModelExportFactorySuite extends FunSuite{ } - test("ModelExportFactory generate IllegalArgumentException when passing an unsupported model") { + test("ModelExportFactory throws IllegalArgumentException when passing an unsupported model") { val invalidModel = new Object; From 1433b1139e510457419f2ffd29195bf29f6e556e Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 29 Oct 2014 10:06:07 +0000 Subject: [PATCH 05/41] complete suite tests --- .../export/ModelExportFactorySuite.scala | 9 +++++-- .../pmml/KMeansPMMLModelExportSuite.scala | 25 +++++++++++++------ 2 files changed, 24 insertions(+), 10 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala index 9b6b4160d6120..bdc0239e94993 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala @@ -26,25 +26,30 @@ class ModelExportFactorySuite extends FunSuite{ test("ModelExportFactory create KMeansPMMLModelExport when passing a KMeansModel") { + //arrange val clusterCenters = Array( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), Vectors.dense(1.0, 4.0, 6.0) ) - val kmeansModel = new KMeansModel(clusterCenters); + //act val modelExport = ModelExportFactory.createModelExport(kmeansModel, ModelExportType.PMML) + //assert assert(modelExport.isInstanceOf[KMeansPMMLModelExport]) } - test("ModelExportFactory throws IllegalArgumentException when passing an unsupported model") { + test("ModelExportFactory throw IllegalArgumentException when passing an unsupported model") { + //arrange val invalidModel = new Object; + //assert intercept[IllegalArgumentException] { + //act ModelExportFactory.createModelExport(invalidModel, ModelExportType.PMML) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala index 02339b0e20e28..4bfd60906a670 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala @@ -22,29 +22,38 @@ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.export.ModelExportFactory import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.export.ModelExportType +import org.dmg.pmml.ClusteringModel +import javax.xml.parsers.DocumentBuilderFactory +import java.io.ByteArrayOutputStream class KMeansPMMLModelExportSuite extends FunSuite{ test("KMeansPMMLModelExport generate PMML format") { + //arrange model to test val clusterCenters = Array( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), Vectors.dense(1.0, 4.0, 6.0) ) - val kmeansModel = new KMeansModel(clusterCenters); + //act by exporting the model to the PMML format val modelExport = ModelExportFactory.createModelExport(kmeansModel, ModelExportType.PMML) - + + //assert that the PMML format is as expected assert(modelExport.isInstanceOf[PMMLModelExport]) + var pmml = modelExport.asInstanceOf[PMMLModelExport].getPmml() + 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 + assert(pmml.getModels().get(0).asInstanceOf[ClusteringModel].getNumberOfClusters() === clusterCenters.size) - //TODO: asserts - //compare pmml fields to strings - modelExport.asInstanceOf[PMMLModelExport].getPmml() - //use document builder to load the xml generated and validated the notes by looking for them - modelExport.asInstanceOf[PMMLModelExport].save(System.out) - //saveLocalFile too??? search how to unit test file creating in java + //manual checking + //modelExport.asInstanceOf[PMMLModelExport].save(System.out) + //modelExport.asInstanceOf[PMMLModelExport].saveLocalFile("/tmp/kmeans.xml") } From 88414391312e4030e462d7d5b37539211ae60316 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 29 Oct 2014 11:25:42 +0000 Subject: [PATCH 06/41] adjust scala style in order to compile --- .../mllib/export/ModelExportFactory.scala | 20 +++++----- .../spark/mllib/export/ModelExportType.scala | 3 +- .../export/pmml/KMeansPMMLModelExport.scala | 40 ++++++++++++------- .../mllib/export/pmml/PMMLModelExport.scala | 11 ++--- 4 files changed, 45 insertions(+), 29 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala index 7e2e76f53988c..d7b1efc19dedb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala @@ -25,16 +25,18 @@ object ModelExportFactory { /** * Factory object to help creating the necessary ModelExport implementation - * taking as input the ModelExportType (for example PMML) and the machine learning model (for example KMeansModel). + * taking as input the ModelExportType (for example PMML) + * and the machine learning model (for example KMeansModel). */ def createModelExport(model: Any, exportType: ModelExportType): ModelExport = { - return exportType match{ - case PMML => model match{ - case kmeans: KMeansModel => new KMeansPMMLModelExport(kmeans) - case _ => throw new IllegalArgumentException("Export not supported for model: " + model.getClass) - } - case _ => throw new IllegalArgumentException("Export type not supported:" + exportType) - } + return exportType match{ + case PMML => model match{ + case kmeans: KMeansModel => new KMeansPMMLModelExport(kmeans) + case _ => + throw new IllegalArgumentException("Export not supported for model: " + model.getClass) + } + case _ => throw new IllegalArgumentException("Export type not supported:" + exportType) + } } - + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala index 1e940a6aa5e50..60bc3eabb9144 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala @@ -19,7 +19,8 @@ package org.apache.spark.mllib.export /** * Defines export types. - * - PMML exports the machine learning models in an XML-based file format called Predictive Model Markup Language developed by the Data Mining Group (www.dmg.org). + * - PMML exports the machine learning models in an XML-based file format + * called Predictive Model Markup Language developed by the Data Mining Group (www.dmg.org). */ object ModelExportType extends Enumeration{ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala index 2f0af9a18f470..37d7b6bf71734 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala @@ -62,30 +62,42 @@ class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ var miningSchema = new MiningSchema() for ( i <- 0 to (clusterCenter.size - 1)) { - fields(i) = FieldName.create("field_"+i) - dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) - miningSchema.withMiningFields(new MiningField(fields(i)).withUsageType(FieldUsageType.ACTIVE)) - } + fields(i) = FieldName.create("field_" + i) + dataDictionary + .withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema + .withMiningFields(new MiningField(fields(i)) + .withUsageType(FieldUsageType.ACTIVE)) + } var comparisonMeasure = new ComparisonMeasure() - .withKind(Kind.DISTANCE) - .withMeasure(new SquaredEuclidean() + .withKind(Kind.DISTANCE) + .withMeasure(new SquaredEuclidean() ); dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()); - + pmml.setDataDictionary(dataDictionary); - var clusteringModel = new ClusteringModel(miningSchema, comparisonMeasure, MiningFunctionType.CLUSTERING, ModelClass.CENTER_BASED, model.clusterCenters.length) - .withModelName("k-means"); + var clusteringModel = new ClusteringModel(miningSchema, comparisonMeasure, + MiningFunctionType.CLUSTERING, ModelClass.CENTER_BASED, model.clusterCenters.length) + .withModelName("k-means"); for ( i <- 0 to (clusterCenter.size - 1)) { - clusteringModel.withClusteringFields(new ClusteringField(fields(i)).withCompareFunction(CompareFunctionType.ABS_DIFF)) - var cluster = new Cluster().withName("cluster_"+i).withArray(new org.dmg.pmml.Array().withType(Type.REAL).withN(clusterCenter.size).withValue(model.clusterCenters(i).toArray.mkString(" "))) - //cluster.withSize(value) //we don't have the size of the single cluster but only the centroids (withValue) - clusteringModel.withClusters(cluster) + clusteringModel.withClusteringFields( + new ClusteringField(fields(i)).withCompareFunction(CompareFunctionType.ABS_DIFF) + ) + var cluster = new Cluster() + .withName("cluster_" + i) + .withArray(new org.dmg.pmml.Array() + .withType(Type.REAL) + .withN(clusterCenter.size) + .withValue(model.clusterCenters(i).toArray.mkString(" "))) + //we don't have the size of the single cluster but only the centroids (withValue) + //.withSize(value) + clusteringModel.withClusters(cluster) } - + pmml.withModels(clusteringModel); } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala index c1e84f62f9223..f18f9cee8ea05 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala @@ -41,13 +41,14 @@ trait PMMLModelExport extends ModelExport{ private def setHeader(pmml : PMML): Unit = { var version = getClass().getPackage().getImplementationVersion() - var app = new Application().withName("Apache Spark MLlib").withVersion(version) - var timestamp = new Timestamp().withContent(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss").format(new Date())) - var header = new Header() - .withCopyright("www.dmg.org") + var app = new Application().withName("Apache Spark MLlib").withVersion(version) + var timestamp = new Timestamp() + .withContent(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss").format(new Date())) + var header = new Header() + .withCopyright("www.dmg.org") .withApplication(app) .withTimestamp(timestamp); - pmml.setHeader(header); + pmml.setHeader(header); } /** From 07a29bf0a08c146e827302215ded3d20122cf20c Mon Sep 17 00:00:00 2001 From: selvinsource Date: Wed, 29 Oct 2014 11:47:20 +0000 Subject: [PATCH 07/41] Update LICENSE added https://github.com/jpmml/jpmml-model library under BSD-style licenses the library is used to export the mining model to the PMML format --- LICENSE | 1 + 1 file changed, 1 insertion(+) diff --git a/LICENSE b/LICENSE index a7eee041129cb..0517dfb0ab53d 100644 --- a/LICENSE +++ b/LICENSE @@ -790,6 +790,7 @@ BSD-style licenses The following components are provided under a BSD-style license. See project link for details. (BSD 3 Clause) core (com.github.fommil.netlib:core:1.1.2 - https://github.com/fommil/netlib-java/core) + (BSD 3 Clause) JPMML-Model (org.jpmml:pmml-model:1.1.7 - https://github.com/jpmml/jpmml-model) (BSD 3-clause style license) jblas (org.jblas:jblas:1.2.3 - http://jblas.org/) (BSD License) AntLR Parser Generator (antlr:antlr:2.7.7 - http://www.antlr.org/) (BSD License) Javolution (javolution:javolution:5.5.1 - http://javolution.org) From cd6c07c654141dcd0eb468e09bf70680a318bdb5 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 29 Oct 2014 09:57:12 -0700 Subject: [PATCH 08/41] fixed scala style to run tests --- .../spark/mllib/export/pmml/KMeansPMMLModelExport.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala index 37d7b6bf71734..4a2b2bae05751 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala @@ -93,8 +93,9 @@ class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ .withType(Type.REAL) .withN(clusterCenter.size) .withValue(model.clusterCenters(i).toArray.mkString(" "))) - //we don't have the size of the single cluster but only the centroids (withValue) - //.withSize(value) + // we don't have the size of the single cluster but only the centroids (withValue) + // .withSize(value) + clusteringModel.withClusters(cluster) } From aba5ee1d75b012a17a1db077bdd88c0970310cae Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Thu, 30 Oct 2014 22:18:54 +0000 Subject: [PATCH 09/41] fixed cluster export --- .../export/pmml/KMeansPMMLModelExport.scala | 28 +++++++++---------- 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala index 4a2b2bae05751..f99bff54f5e74 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala @@ -61,32 +61,30 @@ class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ var miningSchema = new MiningSchema() - for ( i <- 0 to (clusterCenter.size - 1)) { - fields(i) = FieldName.create("field_" + i) - dataDictionary - .withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) - miningSchema - .withMiningFields(new MiningField(fields(i)) - .withUsageType(FieldUsageType.ACTIVE)) - } - var comparisonMeasure = new ComparisonMeasure() .withKind(Kind.DISTANCE) .withMeasure(new SquaredEuclidean() ); - dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()); - - pmml.setDataDictionary(dataDictionary); - var clusteringModel = new ClusteringModel(miningSchema, comparisonMeasure, MiningFunctionType.CLUSTERING, ModelClass.CENTER_BASED, model.clusterCenters.length) .withModelName("k-means"); for ( i <- 0 to (clusterCenter.size - 1)) { + fields(i) = FieldName.create("field_" + i) + dataDictionary + .withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema + .withMiningFields(new MiningField(fields(i)) + .withUsageType(FieldUsageType.ACTIVE)) clusteringModel.withClusteringFields( new ClusteringField(fields(i)).withCompareFunction(CompareFunctionType.ABS_DIFF) - ) + ) + } + + dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()); + + for ( i <- 0 to (model.clusterCenters.size - 1)) { var cluster = new Cluster() .withName("cluster_" + i) .withArray(new org.dmg.pmml.Array() @@ -95,10 +93,10 @@ class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ .withValue(model.clusterCenters(i).toArray.mkString(" "))) // we don't have the size of the single cluster but only the centroids (withValue) // .withSize(value) - clusteringModel.withClusters(cluster) } + pmml.setDataDictionary(dataDictionary); pmml.withModels(clusteringModel); } From e1eb25177ddc611c867c0f4b1a99c885fc34de11 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 5 Nov 2014 10:29:58 +0000 Subject: [PATCH 10/41] removed serialization part, this will be part of the ModelExporter helper object --- .../spark/mllib/export/ModelExport.scala | 18 +----------------- 1 file changed, 1 insertion(+), 17 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala index c0fea6ad95d9b..a6daa6686cf5a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala @@ -17,22 +17,6 @@ package org.apache.spark.mllib.export -import java.io.OutputStream -import java.io.FileOutputStream -import java.io.File - -trait ModelExport { - - /** - * Write the exported model to the output stream specified - */ - def save(outputStream: OutputStream): Unit - - /** - * Write the exported model to the local file specified - */ - def saveLocalFile(path: String): Unit = { - save(new FileOutputStream(new File(path))); - } +private[mllib] trait ModelExport { } From 6357b9890e6f4245a2e8a7d0f90dd6d80495e60a Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 5 Nov 2014 10:30:30 +0000 Subject: [PATCH 11/41] set it to private --- .../org/apache/spark/mllib/export/ModelExportFactory.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala index d7b1efc19dedb..26bc2a499778e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala @@ -21,7 +21,7 @@ import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport import org.apache.spark.mllib.export.ModelExportType._ -object ModelExportFactory { +private[mllib] object ModelExportFactory { /** * Factory object to help creating the necessary ModelExport implementation From c3ef9b80addd476368c58449dfb62ab03f6311fb Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 5 Nov 2014 10:31:09 +0000 Subject: [PATCH 12/41] set it to private --- .../scala/org/apache/spark/mllib/export/ModelExportType.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala index 60bc3eabb9144..f59040f3a3440 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala @@ -22,7 +22,7 @@ package org.apache.spark.mllib.export * - PMML exports the machine learning models in an XML-based file format * called Predictive Model Markup Language developed by the Data Mining Group (www.dmg.org). */ -object ModelExportType extends Enumeration{ +private[mllib] object ModelExportType extends Enumeration{ type ModelExportType = Value val PMML = Value From 349a76bbc86096b06fdc56f8c9cc159095766bec Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 5 Nov 2014 10:31:45 +0000 Subject: [PATCH 13/41] new helper object to serialize the models to pmml format --- .../spark/mllib/export/ModelExporter.scala | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/export/ModelExporter.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExporter.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExporter.scala new file mode 100644 index 0000000000000..af35fee5eec06 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExporter.scala @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.export + +import java.io.File +import javax.xml.transform.stream.StreamResult + +import org.jpmml.model.JAXBUtil + +import org.apache.spark.mllib.export.pmml.PMMLModelExport + +object ModelExporter { + + /** + * Export the input model to the stream result in PMML format + */ + def toPMML(inputModel: Any, streamResult: StreamResult): Unit = { + val modelExport = ModelExportFactory.createModelExport(inputModel, ModelExportType.PMML) + val pmml = modelExport.asInstanceOf[PMMLModelExport].getPmml() + JAXBUtil.marshalPMML(pmml, streamResult) + } + + /** + * Export the input model to a local path in PMML format + */ + def toPMML(inputModel: Any, localPath: String): Unit = { + toPMML(inputModel, new StreamResult(new File(localPath))) + } + +} From 834ca44c0be714f9b8248872e8f9f3681ba2f414 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 5 Nov 2014 10:32:20 +0000 Subject: [PATCH 14/41] reordered the import accordingly to the guidelines --- .../export/pmml/KMeansPMMLModelExport.scala | 45 ++++++++++--------- 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala index f99bff54f5e74..e166ae67f7edd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala @@ -17,30 +17,31 @@ package org.apache.spark.mllib.export.pmml -import org.apache.spark.mllib.clustering.KMeansModel +import org.dmg.pmml.Array.Type +import org.dmg.pmml.Cluster +import org.dmg.pmml.ClusteringField +import org.dmg.pmml.ClusteringModel +import org.dmg.pmml.ClusteringModel.ModelClass +import org.dmg.pmml.CompareFunctionType +import org.dmg.pmml.ComparisonMeasure +import org.dmg.pmml.ComparisonMeasure.Kind import org.dmg.pmml.DataDictionary -import org.dmg.pmml.FieldName import org.dmg.pmml.DataField -import org.dmg.pmml.OpType import org.dmg.pmml.DataType -import org.dmg.pmml.MiningSchema -import org.dmg.pmml.MiningField +import org.dmg.pmml.FieldName import org.dmg.pmml.FieldUsageType -import org.dmg.pmml.ComparisonMeasure -import org.dmg.pmml.ComparisonMeasure.Kind -import org.dmg.pmml.SquaredEuclidean -import org.dmg.pmml.ClusteringModel +import org.dmg.pmml.MiningField import org.dmg.pmml.MiningFunctionType -import org.dmg.pmml.ClusteringModel.ModelClass -import org.dmg.pmml.ClusteringField -import org.dmg.pmml.CompareFunctionType -import org.dmg.pmml.Cluster -import org.dmg.pmml.Array.Type +import org.dmg.pmml.MiningSchema +import org.dmg.pmml.OpType +import org.dmg.pmml.SquaredEuclidean + +import org.apache.spark.mllib.clustering.KMeansModel /** * PMML Model Export for KMeansModel class */ -class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ +private[mllib] class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ /** * Export the input KMeansModel model to PMML format @@ -55,18 +56,18 @@ class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ val clusterCenter = model.clusterCenters(0) - var fields = new Array[FieldName](clusterCenter.size) + val fields = new Array[FieldName](clusterCenter.size) - var dataDictionary = new DataDictionary() + val dataDictionary = new DataDictionary() - var miningSchema = new MiningSchema() + val miningSchema = new MiningSchema() - var comparisonMeasure = new ComparisonMeasure() + val comparisonMeasure = new ComparisonMeasure() .withKind(Kind.DISTANCE) .withMeasure(new SquaredEuclidean() ); - var clusteringModel = new ClusteringModel(miningSchema, comparisonMeasure, + val clusteringModel = new ClusteringModel(miningSchema, comparisonMeasure, MiningFunctionType.CLUSTERING, ModelClass.CENTER_BASED, model.clusterCenters.length) .withModelName("k-means"); @@ -84,8 +85,8 @@ class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()); - for ( i <- 0 to (model.clusterCenters.size - 1)) { - var cluster = new Cluster() + for ( i <- 0 until model.clusterCenters.size ) { + val cluster = new Cluster() .withName("cluster_" + i) .withArray(new org.dmg.pmml.Array() .withType(Type.REAL) From a1b4dc3316e49bc58e728ac9ff694e5edf56944b Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 5 Nov 2014 10:33:34 +0000 Subject: [PATCH 15/41] updated imports --- .../org/apache/spark/mllib/export/ModelExportFactorySuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala index bdc0239e94993..eb0b7f18b1046 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.export +import scala.annotation.varargs import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.linalg.Vectors import org.scalatest.FunSuite From df8a89e16080b53b9b59f3ff8c6eba1cb1fa750a Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 5 Nov 2014 10:34:15 +0000 Subject: [PATCH 16/41] added pmml version to pmml model changed the copyright to spark --- .../mllib/export/pmml/PMMLModelExport.scala | 39 ++++++++----------- 1 file changed, 16 insertions(+), 23 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala index f18f9cee8ea05..665eff71644ad 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala @@ -17,46 +17,39 @@ package org.apache.spark.mllib.export.pmml -import org.apache.spark.mllib.export.ModelExport -import java.io.OutputStream -import org.jpmml.model.JAXBUtil -import org.dmg.pmml.PMML -import javax.xml.transform.stream.StreamResult +import java.text.SimpleDateFormat +import java.util.Date + import scala.beans.BeanProperty + import org.dmg.pmml.Application -import org.dmg.pmml.Timestamp import org.dmg.pmml.Header -import java.text.SimpleDateFormat -import java.util.Date +import org.dmg.pmml.PMML +import org.dmg.pmml.Timestamp + +import org.apache.spark.mllib.export.ModelExport -trait PMMLModelExport extends ModelExport{ +private[mllib] trait PMMLModelExport extends ModelExport{ /** * Holder of the exported model in PMML format */ @BeanProperty - var pmml: PMML = new PMML(); + val pmml: PMML = new PMML(); setHeader(pmml); private def setHeader(pmml : PMML): Unit = { - var version = getClass().getPackage().getImplementationVersion() - var app = new Application().withName("Apache Spark MLlib").withVersion(version) - var timestamp = new Timestamp() + val version = getClass().getPackage().getImplementationVersion() + val app = new Application().withName("Apache Spark MLlib").withVersion(version) + val timestamp = new Timestamp() .withContent(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss").format(new Date())) - var header = new Header() - .withCopyright("www.dmg.org") + val header = new Header() + .withCopyright("Apache Spark MLlib") .withApplication(app) .withTimestamp(timestamp); pmml.setHeader(header); + pmml.setVersion("4.2") } - /** - * Write the exported model (in PMML XML) to the output stream specified - */ - @Override - def save(outputStream: OutputStream): Unit = { - JAXBUtil.marshalPMML(pmml, new StreamResult(outputStream)); - } - } From ae8b993735e77d8c8c17ddf838379c5667c1d6ab Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 5 Nov 2014 10:35:12 +0000 Subject: [PATCH 17/41] updated some commented tests to use the new ModelExporter object reordered the imports --- .../export/pmml/KMeansPMMLModelExportSuite.scala | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala index 4bfd60906a670..30fafe146f367 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala @@ -17,14 +17,13 @@ package org.apache.spark.mllib.export.pmml -import org.scalatest.FunSuite -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.export.ModelExportFactory +import scala.annotation.varargs import org.apache.spark.mllib.clustering.KMeansModel +import org.apache.spark.mllib.export.ModelExportFactory import org.apache.spark.mllib.export.ModelExportType +import org.apache.spark.mllib.linalg.Vectors +import org.scalatest.FunSuite import org.dmg.pmml.ClusteringModel -import javax.xml.parsers.DocumentBuilderFactory -import java.io.ByteArrayOutputStream class KMeansPMMLModelExportSuite extends FunSuite{ @@ -43,7 +42,7 @@ class KMeansPMMLModelExportSuite extends FunSuite{ //assert that the PMML format is as expected assert(modelExport.isInstanceOf[PMMLModelExport]) - var pmml = modelExport.asInstanceOf[PMMLModelExport].getPmml() + val pmml = modelExport.asInstanceOf[PMMLModelExport].getPmml() 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) @@ -52,8 +51,8 @@ class KMeansPMMLModelExportSuite extends FunSuite{ assert(pmml.getModels().get(0).asInstanceOf[ClusteringModel].getNumberOfClusters() === clusterCenters.size) //manual checking - //modelExport.asInstanceOf[PMMLModelExport].save(System.out) - //modelExport.asInstanceOf[PMMLModelExport].saveLocalFile("/tmp/kmeans.xml") + //ModelExporter.toPMML(kmeansModel,new StreamResult(System.out)) + //ModelExporter.toPMML(kmeansModel,"/tmp/kmeans.xml") } From e29dfb99ffecb3cd78e5f2a1357b2244bc9c9737 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 5 Nov 2014 12:38:07 +0000 Subject: [PATCH 18/41] removed version, by default is set to 4.2 (latest from jpmml) removed copyright --- .../mllib/export/pmml/KMeansPMMLModelExport.scala | 14 +++++++------- .../spark/mllib/export/pmml/PMMLModelExport.scala | 6 ++---- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala index e166ae67f7edd..909f172fc1f72 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala @@ -46,11 +46,11 @@ private[mllib] class KMeansPMMLModelExport(model : KMeansModel) extends PMMLMode /** * Export the input KMeansModel model to PMML format */ - populateKMeansPMML(model); + populateKMeansPMML(model) private def populateKMeansPMML(model : KMeansModel): Unit = { - pmml.getHeader().setDescription("k-means clustering"); + pmml.getHeader().setDescription("k-means clustering") if(model.clusterCenters.length > 0){ @@ -65,11 +65,11 @@ private[mllib] class KMeansPMMLModelExport(model : KMeansModel) extends PMMLMode val comparisonMeasure = new ComparisonMeasure() .withKind(Kind.DISTANCE) .withMeasure(new SquaredEuclidean() - ); + ) val clusteringModel = new ClusteringModel(miningSchema, comparisonMeasure, MiningFunctionType.CLUSTERING, ModelClass.CENTER_BASED, model.clusterCenters.length) - .withModelName("k-means"); + .withModelName("k-means") for ( i <- 0 to (clusterCenter.size - 1)) { fields(i) = FieldName.create("field_" + i) @@ -83,7 +83,7 @@ private[mllib] class KMeansPMMLModelExport(model : KMeansModel) extends PMMLMode ) } - dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()); + dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()) for ( i <- 0 until model.clusterCenters.size ) { val cluster = new Cluster() @@ -97,8 +97,8 @@ private[mllib] class KMeansPMMLModelExport(model : KMeansModel) extends PMMLMode clusteringModel.withClusters(cluster) } - pmml.setDataDictionary(dataDictionary); - pmml.withModels(clusteringModel); + pmml.setDataDictionary(dataDictionary) + pmml.withModels(clusteringModel) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala index 665eff71644ad..cf9f993e88e59 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala @@ -45,11 +45,9 @@ private[mllib] trait PMMLModelExport extends ModelExport{ val timestamp = new Timestamp() .withContent(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss").format(new Date())) val header = new Header() - .withCopyright("Apache Spark MLlib") .withApplication(app) - .withTimestamp(timestamp); - pmml.setHeader(header); - pmml.setVersion("4.2") + .withTimestamp(timestamp) + pmml.setHeader(header) } } From 78515ec9d8403d555b996163ca32409d91cb30f7 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Fri, 28 Nov 2014 07:39:57 +0000 Subject: [PATCH 19/41] [SPARK-1406] added pmml export for LinearRegressionModel, RidgeRegressionModel and LassoModel --- .../mllib/export/ModelExportFactory.scala | 13 ++- .../GeneralizedLinearPMMLModelExport.scala | 94 +++++++++++++++++++ .../export/pmml/KMeansPMMLModelExport.scala | 2 +- .../export/ModelExportFactorySuite.scala | 32 +++++++ ...eneralizedLinearPMMLModelExportSuite.scala | 87 +++++++++++++++++ 5 files changed, 226 insertions(+), 2 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala index 26bc2a499778e..079a5efc5c219 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala @@ -20,6 +20,10 @@ package org.apache.spark.mllib.export import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport import org.apache.spark.mllib.export.ModelExportType._ +import org.apache.spark.mllib.regression.LinearRegressionModel +import org.apache.spark.mllib.export.pmml.GeneralizedLinearPMMLModelExport +import org.apache.spark.mllib.regression.RidgeRegressionModel +import org.apache.spark.mllib.regression.LassoModel private[mllib] object ModelExportFactory { @@ -31,7 +35,14 @@ private[mllib] object ModelExportFactory { def createModelExport(model: Any, exportType: ModelExportType): ModelExport = { return exportType match{ case PMML => model match{ - case kmeans: KMeansModel => new KMeansPMMLModelExport(kmeans) + case kmeans: KMeansModel => + new KMeansPMMLModelExport(kmeans) + case linearRegression: LinearRegressionModel => + new GeneralizedLinearPMMLModelExport(linearRegression, "linear regression") + case ridgeRegression: RidgeRegressionModel => + new GeneralizedLinearPMMLModelExport(ridgeRegression, "ridge regression") + case lassoRegression: LassoModel => + new GeneralizedLinearPMMLModelExport(lassoRegression, "lasso regression") case _ => throw new IllegalArgumentException("Export not supported for model: " + model.getClass) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala new file mode 100644 index 0000000000000..edfacafa258ed --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.export.pmml + +import org.dmg.pmml.Array.Type +import org.dmg.pmml.Cluster +import org.dmg.pmml.ClusteringField +import org.dmg.pmml.ClusteringModel +import org.dmg.pmml.ClusteringModel.ModelClass +import org.dmg.pmml.CompareFunctionType +import org.dmg.pmml.ComparisonMeasure +import org.dmg.pmml.ComparisonMeasure.Kind +import org.dmg.pmml.DataDictionary +import org.dmg.pmml.DataField +import org.dmg.pmml.DataType +import org.dmg.pmml.FieldName +import org.dmg.pmml.FieldUsageType +import org.dmg.pmml.MiningField +import org.dmg.pmml.MiningFunctionType +import org.dmg.pmml.MiningSchema +import org.dmg.pmml.OpType +import org.dmg.pmml.SquaredEuclidean +import org.apache.spark.mllib.clustering.KMeansModel +import org.apache.spark.mllib.regression.LinearRegressionModel +import org.apache.spark.mllib.regression.GeneralizedLinearModel +import org.dmg.pmml.RegressionModel +import org.dmg.pmml.RegressionTable +import org.dmg.pmml.NumericPredictor + +/** + * PMML Model Export for GeneralizedLinear abstract class + */ +private[mllib] class GeneralizedLinearPMMLModelExport( + model : GeneralizedLinearModel, + description : String) + extends PMMLModelExport{ + + /** + * Export the input GeneralizedLinearModel model to PMML format + */ + populateGeneralizedLinearPMML(model) + + private def populateGeneralizedLinearPMML(model : GeneralizedLinearModel): Unit = { + + pmml.getHeader().setDescription(description) + + if(model.weights.size > 0){ + + val fields = new Array[FieldName](model.weights.size) + + val dataDictionary = new DataDictionary() + + val miningSchema = new MiningSchema() + + val regressionTable = new RegressionTable(model.intercept) + + val regressionModel = new RegressionModel(miningSchema,MiningFunctionType.REGRESSION) + .withModelName(description).withRegressionTables(regressionTable) + + for ( i <- 0 until model.weights.size) { + fields(i) = FieldName.create("field_" + i) + dataDictionary + .withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema + .withMiningFields(new MiningField(fields(i)) + .withUsageType(FieldUsageType.ACTIVE)) + regressionTable.withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) + } + + dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()) + + pmml.setDataDictionary(dataDictionary) + pmml.withModels(regressionModel) + + } + + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala index 909f172fc1f72..c10d48fb8eb4c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala @@ -71,7 +71,7 @@ private[mllib] class KMeansPMMLModelExport(model : KMeansModel) extends PMMLMode MiningFunctionType.CLUSTERING, ModelClass.CENTER_BASED, model.clusterCenters.length) .withModelName("k-means") - for ( i <- 0 to (clusterCenter.size - 1)) { + for ( i <- 0 until clusterCenter.size) { fields(i) = FieldName.create("field_" + i) dataDictionary .withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala index eb0b7f18b1046..ec4b300d71e56 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala @@ -22,6 +22,11 @@ import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.linalg.Vectors import org.scalatest.FunSuite import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport +import org.apache.spark.mllib.util.LinearDataGenerator +import org.apache.spark.mllib.regression.LinearRegressionModel +import org.apache.spark.mllib.export.pmml.GeneralizedLinearPMMLModelExport +import org.apache.spark.mllib.regression.LassoModel +import org.apache.spark.mllib.regression.RidgeRegressionModel class ModelExportFactorySuite extends FunSuite{ @@ -43,6 +48,33 @@ class ModelExportFactorySuite extends FunSuite{ } + test("ModelExportFactory create GeneralizedLinearPMMLModelExport when passing a" + +"LinearRegressionModel, RidgeRegressionModel or LassoModel") { + + //arrange + val linearInput = LinearDataGenerator.generateLinearInput( + 3.0, Array(10.0, 10.0), 1, 17) + val linearRegressionModel = new LinearRegressionModel(linearInput(0).features, linearInput(0).label); + val ridgeRegressionModel = new RidgeRegressionModel(linearInput(0).features, linearInput(0).label); + val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label); + + //act + val linearModelExport = ModelExportFactory.createModelExport(linearRegressionModel, ModelExportType.PMML) + //assert + assert(linearModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + + //act + val ridgeModelExport = ModelExportFactory.createModelExport(ridgeRegressionModel, ModelExportType.PMML) + //assert + assert(ridgeModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + + //act + val lassoModelExport = ModelExportFactory.createModelExport(lassoModel, ModelExportType.PMML) + //assert + assert(lassoModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + + } + test("ModelExportFactory throw IllegalArgumentException when passing an unsupported model") { //arrange diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala new file mode 100644 index 0000000000000..20245c6917c5b --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.export.pmml + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.export.ModelExportFactory +import org.apache.spark.mllib.export.ModelExportType +import org.apache.spark.mllib.regression.LassoModel +import org.apache.spark.mllib.regression.LinearRegressionModel +import org.apache.spark.mllib.regression.RidgeRegressionModel +import org.apache.spark.mllib.util.LinearDataGenerator +import org.scalatest.FunSuite +import org.dmg.pmml.RegressionModel + +class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ + + test("GeneralizedLinearPMMLModelExport generate PMML format") { + + //arrange models to test + val linearInput = LinearDataGenerator.generateLinearInput( + 3.0, Array(10.0, 10.0), 1, 17) + val linearRegressionModel = new LinearRegressionModel(linearInput(0).features, linearInput(0).label); + val ridgeRegressionModel = new RidgeRegressionModel(linearInput(0).features, linearInput(0).label); + val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label); + + //act by exporting the model to the PMML format + val linearModelExport = ModelExportFactory.createModelExport(linearRegressionModel, ModelExportType.PMML) + //assert that the PMML format is as expected + assert(linearModelExport.isInstanceOf[PMMLModelExport]) + var pmml = linearModelExport.asInstanceOf[PMMLModelExport].getPmml() + assert(pmml.getHeader().getDescription() === "linear regression") + //check that the number of fields match the weights size + assert(pmml.getDataDictionary().getNumberOfFields() === linearRegressionModel.weights.size) + //this verify that there is a model attached to the pmml object and the model is a regression one + //it also verifies that the pmml model has a regression table with the same number of predictors of the model weights + assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] + .getRegressionTables().get(0).getNumericPredictors().size() === linearRegressionModel.weights.size) + + //act + val ridgeModelExport = ModelExportFactory.createModelExport(ridgeRegressionModel, ModelExportType.PMML) + //assert that the PMML format is as expected + assert(ridgeModelExport.isInstanceOf[PMMLModelExport]) + pmml = ridgeModelExport.asInstanceOf[PMMLModelExport].getPmml() + assert(pmml.getHeader().getDescription() === "ridge regression") + //check that the number of fields match the weights size + assert(pmml.getDataDictionary().getNumberOfFields() === ridgeRegressionModel.weights.size) + //this verify that there is a model attached to the pmml object and the model is a regression one + //it also verifies that the pmml model has a regression table with the same number of predictors of the model weights + assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] + .getRegressionTables().get(0).getNumericPredictors().size() === ridgeRegressionModel.weights.size) + + //act + val lassoModelExport = ModelExportFactory.createModelExport(lassoModel, ModelExportType.PMML) + //assert that the PMML format is as expected + assert(lassoModelExport.isInstanceOf[PMMLModelExport]) + pmml = lassoModelExport.asInstanceOf[PMMLModelExport].getPmml() + assert(pmml.getHeader().getDescription() === "lasso regression") + //check that the number of fields match the weights size + assert(pmml.getDataDictionary().getNumberOfFields() === lassoModel.weights.size) + //this verify that there is a model attached to the pmml object and the model is a regression one + //it also verifies that the pmml model has a regression table with the same number of predictors of the model weights + assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] + .getRegressionTables().get(0).getNumericPredictors().size() === lassoModel.weights.size) + + //manual checking + //ModelExporter.toPMML(linearRegressionModel,"/tmp/linearregression.xml") + //ModelExporter.toPMML(ridgeRegressionModel,"/tmp/ridgeregression.xml") + //ModelExporter.toPMML(lassoModel,"/tmp/lassoregression.xml") + + } + +} From 3ae8ae5189a257ad9d9242b7ef6957831ef0a2c8 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sat, 29 Nov 2014 08:46:39 +0000 Subject: [PATCH 20/41] [SPARK-1406] Adjusted imported order according to the guidelines --- .../spark/mllib/export/ModelExportFactory.scala | 7 ++++--- .../pmml/GeneralizedLinearPMMLModelExport.scala | 16 +++------------- .../mllib/export/ModelExportFactorySuite.scala | 12 ++++++------ .../GeneralizedLinearPMMLModelExportSuite.scala | 6 +++--- .../export/pmml/KMeansPMMLModelExportSuite.scala | 6 +++--- 5 files changed, 19 insertions(+), 28 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala index 079a5efc5c219..f52d22093739d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala @@ -18,12 +18,13 @@ package org.apache.spark.mllib.export import org.apache.spark.mllib.clustering.KMeansModel +import org.apache.spark.mllib.export.ModelExportType.ModelExportType +import org.apache.spark.mllib.export.ModelExportType.PMML +import org.apache.spark.mllib.export.pmml.GeneralizedLinearPMMLModelExport import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport -import org.apache.spark.mllib.export.ModelExportType._ +import org.apache.spark.mllib.regression.LassoModel import org.apache.spark.mllib.regression.LinearRegressionModel -import org.apache.spark.mllib.export.pmml.GeneralizedLinearPMMLModelExport import org.apache.spark.mllib.regression.RidgeRegressionModel -import org.apache.spark.mllib.regression.LassoModel private[mllib] object ModelExportFactory { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala index edfacafa258ed..aed7ef5b26bc2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala @@ -17,14 +17,6 @@ package org.apache.spark.mllib.export.pmml -import org.dmg.pmml.Array.Type -import org.dmg.pmml.Cluster -import org.dmg.pmml.ClusteringField -import org.dmg.pmml.ClusteringModel -import org.dmg.pmml.ClusteringModel.ModelClass -import org.dmg.pmml.CompareFunctionType -import org.dmg.pmml.ComparisonMeasure -import org.dmg.pmml.ComparisonMeasure.Kind import org.dmg.pmml.DataDictionary import org.dmg.pmml.DataField import org.dmg.pmml.DataType @@ -33,14 +25,12 @@ import org.dmg.pmml.FieldUsageType import org.dmg.pmml.MiningField import org.dmg.pmml.MiningFunctionType import org.dmg.pmml.MiningSchema +import org.dmg.pmml.NumericPredictor import org.dmg.pmml.OpType -import org.dmg.pmml.SquaredEuclidean -import org.apache.spark.mllib.clustering.KMeansModel -import org.apache.spark.mllib.regression.LinearRegressionModel -import org.apache.spark.mllib.regression.GeneralizedLinearModel import org.dmg.pmml.RegressionModel import org.dmg.pmml.RegressionTable -import org.dmg.pmml.NumericPredictor + +import org.apache.spark.mllib.regression.GeneralizedLinearModel /** * PMML Model Export for GeneralizedLinear abstract class diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala index ec4b300d71e56..b2208dd6d0c7d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala @@ -17,16 +17,16 @@ package org.apache.spark.mllib.export -import scala.annotation.varargs +import org.scalatest.FunSuite + import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.linalg.Vectors -import org.scalatest.FunSuite -import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport -import org.apache.spark.mllib.util.LinearDataGenerator -import org.apache.spark.mllib.regression.LinearRegressionModel -import org.apache.spark.mllib.export.pmml.GeneralizedLinearPMMLModelExport import org.apache.spark.mllib.regression.LassoModel +import org.apache.spark.mllib.regression.LinearRegressionModel import org.apache.spark.mllib.regression.RidgeRegressionModel +import org.apache.spark.mllib.util.LinearDataGenerator +import org.apache.spark.mllib.export.pmml.GeneralizedLinearPMMLModelExport +import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport class ModelExportFactorySuite extends FunSuite{ diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala index 20245c6917c5b..c3ecbd0b4b2cb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala @@ -17,15 +17,15 @@ package org.apache.spark.mllib.export.pmml -import org.apache.spark.annotation.DeveloperApi +import org.dmg.pmml.RegressionModel +import org.scalatest.FunSuite + import org.apache.spark.mllib.export.ModelExportFactory import org.apache.spark.mllib.export.ModelExportType import org.apache.spark.mllib.regression.LassoModel import org.apache.spark.mllib.regression.LinearRegressionModel import org.apache.spark.mllib.regression.RidgeRegressionModel import org.apache.spark.mllib.util.LinearDataGenerator -import org.scalatest.FunSuite -import org.dmg.pmml.RegressionModel class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala index 30fafe146f367..471e311b53653 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala @@ -17,13 +17,13 @@ package org.apache.spark.mllib.export.pmml -import scala.annotation.varargs +import org.dmg.pmml.ClusteringModel +import org.scalatest.FunSuite + import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.export.ModelExportFactory import org.apache.spark.mllib.export.ModelExportType import org.apache.spark.mllib.linalg.Vectors -import org.scalatest.FunSuite -import org.dmg.pmml.ClusteringModel class KMeansPMMLModelExportSuite extends FunSuite{ From 1faf985b82afec0f5bf0a8dc9cd4f387390aa487 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sat, 29 Nov 2014 08:58:42 +0000 Subject: [PATCH 21/41] [SPARK-1406] Added target field to the regression model for completeness Adjusted unit test to deal with this change --- .../export/pmml/GeneralizedLinearPMMLModelExport.scala | 10 ++++++++++ .../pmml/GeneralizedLinearPMMLModelExportSuite.scala | 6 +++--- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala index aed7ef5b26bc2..6c8599e81ab8a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala @@ -72,6 +72,16 @@ private[mllib] class GeneralizedLinearPMMLModelExport( regressionTable.withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) } + //for completeness add target field + val targetField = FieldName.create("target"); + dataDictionary + .withDataFields( + new DataField(targetField, OpType.CONTINUOUS, DataType.DOUBLE) + ) + miningSchema + .withMiningFields(new MiningField(targetField) + .withUsageType(FieldUsageType.TARGET)) + dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()) pmml.setDataDictionary(dataDictionary) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala index c3ecbd0b4b2cb..628c74e418a28 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala @@ -45,7 +45,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ var pmml = linearModelExport.asInstanceOf[PMMLModelExport].getPmml() assert(pmml.getHeader().getDescription() === "linear regression") //check that the number of fields match the weights size - assert(pmml.getDataDictionary().getNumberOfFields() === linearRegressionModel.weights.size) + assert(pmml.getDataDictionary().getNumberOfFields() === linearRegressionModel.weights.size + 1) //this verify that there is a model attached to the pmml object and the model is a regression one //it also verifies that the pmml model has a regression table with the same number of predictors of the model weights assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] @@ -58,7 +58,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ pmml = ridgeModelExport.asInstanceOf[PMMLModelExport].getPmml() assert(pmml.getHeader().getDescription() === "ridge regression") //check that the number of fields match the weights size - assert(pmml.getDataDictionary().getNumberOfFields() === ridgeRegressionModel.weights.size) + assert(pmml.getDataDictionary().getNumberOfFields() === ridgeRegressionModel.weights.size + 1) //this verify that there is a model attached to the pmml object and the model is a regression one //it also verifies that the pmml model has a regression table with the same number of predictors of the model weights assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] @@ -71,7 +71,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ pmml = lassoModelExport.asInstanceOf[PMMLModelExport].getPmml() assert(pmml.getHeader().getDescription() === "lasso regression") //check that the number of fields match the weights size - assert(pmml.getDataDictionary().getNumberOfFields() === lassoModel.weights.size) + assert(pmml.getDataDictionary().getNumberOfFields() === lassoModel.weights.size + 1) //this verify that there is a model attached to the pmml object and the model is a regression one //it also verifies that the pmml model has a regression table with the same number of predictors of the model weights assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] From 19adf29d0bf2c5ffbdd7ba257296475463dd5eae Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sat, 29 Nov 2014 10:33:16 +0000 Subject: [PATCH 22/41] [SPARK-1406] Fixed scala style --- .../mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala index 6c8599e81ab8a..8b3a20602895f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala @@ -72,7 +72,7 @@ private[mllib] class GeneralizedLinearPMMLModelExport( regressionTable.withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) } - //for completeness add target field + // for completeness add target field val targetField = FieldName.create("target"); dataDictionary .withDataFields( From da2ec11fdf79567a0c6ee175f09b7ef7e6c35398 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Mon, 8 Dec 2014 22:10:49 +0000 Subject: [PATCH 23/41] [SPARK-1406] added linear SVM PMML export --- .../spark/mllib/export/ModelExportFactory.scala | 3 +++ .../mllib/export/ModelExportFactorySuite.scala | 11 +++++++++-- .../GeneralizedLinearPMMLModelExportSuite.scala | 16 ++++++++++++++++ 3 files changed, 28 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala index f52d22093739d..282a32ebc5ced 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.export +import org.apache.spark.mllib.classification.SVMModel import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.export.ModelExportType.ModelExportType import org.apache.spark.mllib.export.ModelExportType.PMML @@ -44,6 +45,8 @@ private[mllib] object ModelExportFactory { new GeneralizedLinearPMMLModelExport(ridgeRegression, "ridge regression") case lassoRegression: LassoModel => new GeneralizedLinearPMMLModelExport(lassoRegression, "lasso regression") + case svm: SVMModel => + new GeneralizedLinearPMMLModelExport(svm, "linear SVM") case _ => throw new IllegalArgumentException("Export not supported for model: " + model.getClass) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala index b2208dd6d0c7d..6792e2d674bb4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.export import org.scalatest.FunSuite +import org.apache.spark.mllib.classification.SVMModel import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LassoModel @@ -48,8 +49,8 @@ class ModelExportFactorySuite extends FunSuite{ } - test("ModelExportFactory create GeneralizedLinearPMMLModelExport when passing a" - +"LinearRegressionModel, RidgeRegressionModel or LassoModel") { + test("ModelExportFactory create GeneralizedLinearPMMLModelExport when passing a " + +"LinearRegressionModel, RidgeRegressionModel, LassoModel or SVMModel") { //arrange val linearInput = LinearDataGenerator.generateLinearInput( @@ -57,6 +58,7 @@ class ModelExportFactorySuite extends FunSuite{ val linearRegressionModel = new LinearRegressionModel(linearInput(0).features, linearInput(0).label); val ridgeRegressionModel = new RidgeRegressionModel(linearInput(0).features, linearInput(0).label); val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label); + val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label); //act val linearModelExport = ModelExportFactory.createModelExport(linearRegressionModel, ModelExportType.PMML) @@ -73,6 +75,11 @@ class ModelExportFactorySuite extends FunSuite{ //assert assert(lassoModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + //act + val svmModelExport = ModelExportFactory.createModelExport(svmModel, ModelExportType.PMML) + //assert + assert(svmModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + } test("ModelExportFactory throw IllegalArgumentException when passing an unsupported model") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala index 628c74e418a28..402a84c2c8a47 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.export.pmml import org.dmg.pmml.RegressionModel import org.scalatest.FunSuite +import org.apache.spark.mllib.classification.SVMModel import org.apache.spark.mllib.export.ModelExportFactory import org.apache.spark.mllib.export.ModelExportType import org.apache.spark.mllib.regression.LassoModel @@ -37,6 +38,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ val linearRegressionModel = new LinearRegressionModel(linearInput(0).features, linearInput(0).label); val ridgeRegressionModel = new RidgeRegressionModel(linearInput(0).features, linearInput(0).label); val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label); + val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label); //act by exporting the model to the PMML format val linearModelExport = ModelExportFactory.createModelExport(linearRegressionModel, ModelExportType.PMML) @@ -76,11 +78,25 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ //it also verifies that the pmml model has a regression table with the same number of predictors of the model weights assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] .getRegressionTables().get(0).getNumericPredictors().size() === lassoModel.weights.size) + + //act + val svmModelExport = ModelExportFactory.createModelExport(svmModel, ModelExportType.PMML) + //assert that the PMML format is as expected + assert(svmModelExport.isInstanceOf[PMMLModelExport]) + pmml = svmModelExport.asInstanceOf[PMMLModelExport].getPmml() + assert(pmml.getHeader().getDescription() === "linear SVM") + //check that the number of fields match the weights size + assert(pmml.getDataDictionary().getNumberOfFields() === svmModel.weights.size + 1) + //this verify that there is a model attached to the pmml object and the model is a regression one + //it also verifies that the pmml model has a regression table with the same number of predictors of the model weights + assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] + .getRegressionTables().get(0).getNumericPredictors().size() === svmModel.weights.size) //manual checking //ModelExporter.toPMML(linearRegressionModel,"/tmp/linearregression.xml") //ModelExporter.toPMML(ridgeRegressionModel,"/tmp/ridgeregression.xml") //ModelExporter.toPMML(lassoModel,"/tmp/lassoregression.xml") + //ModelExporter.toPMML(svmModel,"/tmp/svm.xml") } From 03bc3a51b39440ea5f050f2c0470b5e66da85cf1 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Tue, 9 Dec 2014 08:06:12 +0000 Subject: [PATCH 24/41] added logistic regression --- .../mllib/export/ModelExportFactory.scala | 11 +- .../GeneralizedLinearPMMLModelExport.scala | 2 +- .../LogisticRegressionPMMLModelExport.scala | 101 ++++++++++++++++++ .../export/ModelExportFactorySuite.scala | 24 ++++- ...eneralizedLinearPMMLModelExportSuite.scala | 4 +- ...gisticRegressionPMMLModelExportSuite.scala | 62 +++++++++++ 6 files changed, 196 insertions(+), 8 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExport.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExportSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala index 282a32ebc5ced..4889be8e3c7ec 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala @@ -17,12 +17,14 @@ package org.apache.spark.mllib.export +import org.apache.spark.mllib.classification.LogisticRegressionModel import org.apache.spark.mllib.classification.SVMModel import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.export.ModelExportType.ModelExportType import org.apache.spark.mllib.export.ModelExportType.PMML import org.apache.spark.mllib.export.pmml.GeneralizedLinearPMMLModelExport import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport +import org.apache.spark.mllib.export.pmml.LogisticRegressionPMMLModelExport import org.apache.spark.mllib.regression.LassoModel import org.apache.spark.mllib.regression.LinearRegressionModel import org.apache.spark.mllib.regression.RidgeRegressionModel @@ -46,7 +48,14 @@ private[mllib] object ModelExportFactory { case lassoRegression: LassoModel => new GeneralizedLinearPMMLModelExport(lassoRegression, "lasso regression") case svm: SVMModel => - new GeneralizedLinearPMMLModelExport(svm, "linear SVM") + new GeneralizedLinearPMMLModelExport( + svm, + "linear SVM: if predicted value > 0, the outcome is positive, or negative otherwise") + case logisticRegression: LogisticRegressionModel => + new LogisticRegressionPMMLModelExport( + logisticRegression, + "logistic regression: if predicted value > 0.5, " + + "the outcome is positive, or negative otherwise") case _ => throw new IllegalArgumentException("Export not supported for model: " + model.getClass) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala index 8b3a20602895f..8b3d1ce9e3e0f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala @@ -33,7 +33,7 @@ import org.dmg.pmml.RegressionTable import org.apache.spark.mllib.regression.GeneralizedLinearModel /** - * PMML Model Export for GeneralizedLinear abstract class + * PMML Model Export for GeneralizedLinearModel abstract class */ private[mllib] class GeneralizedLinearPMMLModelExport( model : GeneralizedLinearModel, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExport.scala new file mode 100644 index 0000000000000..f0c6708af58c4 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExport.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.export.pmml + +import org.dmg.pmml.DataDictionary +import org.dmg.pmml.DataField +import org.dmg.pmml.DataType +import org.dmg.pmml.FieldName +import org.dmg.pmml.FieldUsageType +import org.dmg.pmml.MiningField +import org.dmg.pmml.MiningFunctionType +import org.dmg.pmml.MiningSchema +import org.dmg.pmml.NumericPredictor +import org.dmg.pmml.OpType +import org.dmg.pmml.RegressionModel +import org.dmg.pmml.RegressionTable +import org.apache.spark.mllib.classification.LogisticRegressionModel +import org.dmg.pmml.RegressionNormalizationMethodType + +/** + * PMML Model Export for LogisticRegressionModel class + */ +private[mllib] class LogisticRegressionPMMLModelExport( + model : LogisticRegressionModel, + description : String) + extends PMMLModelExport{ + + /** + * Export the input LogisticRegressionModel model to PMML format + */ + populateLogisticRegressionPMML(model) + + private def populateLogisticRegressionPMML(model : LogisticRegressionModel): Unit = { + + pmml.getHeader().setDescription(description) + + if(model.weights.size > 0){ + + val fields = new Array[FieldName](model.weights.size) + + val dataDictionary = new DataDictionary() + + val miningSchema = new MiningSchema() + + val regressionTableYES = new RegressionTable(model.intercept) + .withTargetCategory("YES") + + val regressionTableNO = new RegressionTable(0.0) + .withTargetCategory("NO") + + val regressionModel = new RegressionModel(miningSchema,MiningFunctionType.CLASSIFICATION) + .withModelName(description) + .withNormalizationMethod(RegressionNormalizationMethodType.LOGIT) + .withRegressionTables(regressionTableYES, regressionTableNO) + + for ( i <- 0 until model.weights.size) { + fields(i) = FieldName.create("field_" + i) + dataDictionary + .withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema + .withMiningFields(new MiningField(fields(i)) + .withUsageType(FieldUsageType.ACTIVE)) + regressionTableYES + .withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) + } + + // add target field + val targetField = FieldName.create("target"); + dataDictionary + .withDataFields( + new DataField(targetField, OpType.CONTINUOUS, DataType.DOUBLE) + ) + miningSchema + .withMiningFields(new MiningField(targetField) + .withUsageType(FieldUsageType.TARGET)) + + dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()) + + pmml.setDataDictionary(dataDictionary) + pmml.withModels(regressionModel) + + } + + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala index 6792e2d674bb4..d63a544ebdf97 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.export import org.scalatest.FunSuite +import org.apache.spark.mllib.classification.LogisticRegressionModel import org.apache.spark.mllib.classification.SVMModel import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.linalg.Vectors @@ -28,6 +29,7 @@ import org.apache.spark.mllib.regression.RidgeRegressionModel import org.apache.spark.mllib.util.LinearDataGenerator import org.apache.spark.mllib.export.pmml.GeneralizedLinearPMMLModelExport import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport +import org.apache.spark.mllib.export.pmml.LogisticRegressionPMMLModelExport class ModelExportFactorySuite extends FunSuite{ @@ -55,10 +57,10 @@ class ModelExportFactorySuite extends FunSuite{ //arrange val linearInput = LinearDataGenerator.generateLinearInput( 3.0, Array(10.0, 10.0), 1, 17) - val linearRegressionModel = new LinearRegressionModel(linearInput(0).features, linearInput(0).label); - val ridgeRegressionModel = new RidgeRegressionModel(linearInput(0).features, linearInput(0).label); - val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label); - val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label); + val linearRegressionModel = new LinearRegressionModel(linearInput(0).features, linearInput(0).label) + val ridgeRegressionModel = new RidgeRegressionModel(linearInput(0).features, linearInput(0).label) + val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label) + val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) //act val linearModelExport = ModelExportFactory.createModelExport(linearRegressionModel, ModelExportType.PMML) @@ -82,6 +84,20 @@ class ModelExportFactorySuite extends FunSuite{ } + test("ModelExportFactory create LogisticRegressionPMMLModelExport when passing a LogisticRegressionModel") { + + //arrange + val linearInput = LinearDataGenerator.generateLinearInput( + 3.0, Array(10.0, 10.0), 1, 17) + val logisticRegressionModel = new LogisticRegressionModel(linearInput(0).features, linearInput(0).label); + + //act + val logisticRegressionModelExport = ModelExportFactory.createModelExport(logisticRegressionModel, ModelExportType.PMML) + //assert + assert(logisticRegressionModelExport.isInstanceOf[LogisticRegressionPMMLModelExport]) + + } + test("ModelExportFactory throw IllegalArgumentException when passing an unsupported model") { //arrange diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala index 402a84c2c8a47..e27c193a7f7f8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala @@ -84,7 +84,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ //assert that the PMML format is as expected assert(svmModelExport.isInstanceOf[PMMLModelExport]) pmml = svmModelExport.asInstanceOf[PMMLModelExport].getPmml() - assert(pmml.getHeader().getDescription() === "linear SVM") + assert(pmml.getHeader().getDescription() === "linear SVM: if predicted value > 0, the outcome is positive, or negative otherwise") //check that the number of fields match the weights size assert(pmml.getDataDictionary().getNumberOfFields() === svmModel.weights.size + 1) //this verify that there is a model attached to the pmml object and the model is a regression one @@ -96,7 +96,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ //ModelExporter.toPMML(linearRegressionModel,"/tmp/linearregression.xml") //ModelExporter.toPMML(ridgeRegressionModel,"/tmp/ridgeregression.xml") //ModelExporter.toPMML(lassoModel,"/tmp/lassoregression.xml") - //ModelExporter.toPMML(svmModel,"/tmp/svm.xml") + //ModelExporter.toPMML(svmModel,"/tmp/linearsvm.xml") } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExportSuite.scala new file mode 100644 index 0000000000000..27093938102ba --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExportSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.export.pmml + +import org.dmg.pmml.RegressionModel +import org.scalatest.FunSuite + +import org.apache.spark.mllib.classification.LogisticRegressionModel +import org.apache.spark.mllib.export.ModelExportFactory +import org.apache.spark.mllib.export.ModelExportType +import org.apache.spark.mllib.util.LinearDataGenerator + +class LogisticRegressionPMMLModelExportSuite extends FunSuite{ + + test("LogisticRegressionPMMLModelExport generate PMML format") { + + //arrange models to test + val linearInput = LinearDataGenerator.generateLinearInput( + 3.0, Array(10.0, 10.0), 1, 17) + val logisticRegressionModel = new LogisticRegressionModel(linearInput(0).features, linearInput(0).label); + + //act by exporting the model to the PMML format + val logisticModelExport = ModelExportFactory.createModelExport(logisticRegressionModel, ModelExportType.PMML) + //assert that the PMML format is as expected + assert(logisticModelExport.isInstanceOf[PMMLModelExport]) + var pmml = logisticModelExport.asInstanceOf[PMMLModelExport].getPmml() + assert(pmml.getHeader().getDescription() === "logistic regression: if predicted value > 0.5, the outcome is positive, or negative otherwise") + //check that the number of fields match the weights size + assert(pmml.getDataDictionary().getNumberOfFields() === logisticRegressionModel.weights.size + 1) + //this verify that there is a model attached to the pmml object and the model is a regression one + //it also verifies that the pmml model has a regression table (for target category YES) with the same number of predictors of the model weights + assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] + .getRegressionTables().get(0).getTargetCategory() === "YES") + assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] + .getRegressionTables().get(0).getNumericPredictors().size() === logisticRegressionModel.weights.size) + //verify if there is a second table with target category NO and no predictors + assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] + .getRegressionTables().get(1).getTargetCategory() === "NO") + assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] + .getRegressionTables().get(1).getNumericPredictors().size() === 0) + + //manual checking + //ModelExporter.toPMML(logisticRegressionModel,"/tmp/logisticregression.xml") + + } + +} From 8fe12bb58358c0d08ede99ed9520e386aedf5110 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sat, 13 Dec 2014 16:28:35 +0000 Subject: [PATCH 25/41] [SPARK-1406] Adjusted logistic regression export description and target categories --- .../apache/spark/mllib/export/ModelExportFactory.scala | 5 +---- .../pmml/LogisticRegressionPMMLModelExport.scala | 6 +++--- .../pmml/LogisticRegressionPMMLModelExportSuite.scala | 10 +++++----- 3 files changed, 9 insertions(+), 12 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala index 4889be8e3c7ec..618fe79a7b14a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala @@ -52,10 +52,7 @@ private[mllib] object ModelExportFactory { svm, "linear SVM: if predicted value > 0, the outcome is positive, or negative otherwise") case logisticRegression: LogisticRegressionModel => - new LogisticRegressionPMMLModelExport( - logisticRegression, - "logistic regression: if predicted value > 0.5, " - + "the outcome is positive, or negative otherwise") + new LogisticRegressionPMMLModelExport(logisticRegression, "logistic regression") case _ => throw new IllegalArgumentException("Export not supported for model: " + model.getClass) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExport.scala index f0c6708af58c4..0d65bc9ddc627 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExport.scala @@ -58,10 +58,10 @@ private[mllib] class LogisticRegressionPMMLModelExport( val miningSchema = new MiningSchema() val regressionTableYES = new RegressionTable(model.intercept) - .withTargetCategory("YES") + .withTargetCategory("1") val regressionTableNO = new RegressionTable(0.0) - .withTargetCategory("NO") + .withTargetCategory("0") val regressionModel = new RegressionModel(miningSchema,MiningFunctionType.CLASSIFICATION) .withModelName(description) @@ -83,7 +83,7 @@ private[mllib] class LogisticRegressionPMMLModelExport( val targetField = FieldName.create("target"); dataDictionary .withDataFields( - new DataField(targetField, OpType.CONTINUOUS, DataType.DOUBLE) + new DataField(targetField, OpType.CATEGORICAL, DataType.STRING) ) miningSchema .withMiningFields(new MiningField(targetField) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExportSuite.scala index 27093938102ba..0bb6c9a60a485 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExportSuite.scala @@ -39,18 +39,18 @@ class LogisticRegressionPMMLModelExportSuite extends FunSuite{ //assert that the PMML format is as expected assert(logisticModelExport.isInstanceOf[PMMLModelExport]) var pmml = logisticModelExport.asInstanceOf[PMMLModelExport].getPmml() - assert(pmml.getHeader().getDescription() === "logistic regression: if predicted value > 0.5, the outcome is positive, or negative otherwise") + assert(pmml.getHeader().getDescription() === "logistic regression") //check that the number of fields match the weights size assert(pmml.getDataDictionary().getNumberOfFields() === logisticRegressionModel.weights.size + 1) //this verify that there is a model attached to the pmml object and the model is a regression one - //it also verifies that the pmml model has a regression table (for target category YES) with the same number of predictors of the model weights + //it also verifies that the pmml model has a regression table (for target category 1) with the same number of predictors of the model weights assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(0).getTargetCategory() === "YES") + .getRegressionTables().get(0).getTargetCategory() === "1") assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] .getRegressionTables().get(0).getNumericPredictors().size() === logisticRegressionModel.weights.size) - //verify if there is a second table with target category NO and no predictors + //verify if there is a second table with target category 0 and no predictors assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(1).getTargetCategory() === "NO") + .getRegressionTables().get(1).getTargetCategory() === "0") assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] .getRegressionTables().get(1).getNumericPredictors().size() === 0) From 7b33b4e7a7cb6fde18cb369a20e236daa367cb12 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sun, 8 Feb 2015 17:18:58 +0000 Subject: [PATCH 26/41] [SPARK-1406] Added a PMMLExportable interface Restructured code in a new package mllib.pmml Supported models implements the new PMMLExportable interface: LogisticRegression, SVM, KMeansModel, LinearRegression, RidgeRegression, Lasso --- .../spark/mllib/export/ModelExport.scala | 22 ------- .../spark/mllib/export/ModelExportType.scala | 30 --------- .../spark/mllib/export/ModelExporter.scala | 45 ------------- .../spark/mllib/pmml/PMMLExportable.scala | 66 +++++++++++++++++++ .../GeneralizedLinearPMMLModelExport.scala | 3 +- .../export}/KMeansPMMLModelExport.scala | 3 +- .../LogisticRegressionPMMLModelExport.scala | 4 +- .../export}/PMMLModelExport.scala | 8 +-- .../export/PMMLModelExportFactory.scala} | 25 +++---- ...eneralizedLinearPMMLModelExportSuite.scala | 21 +++--- .../export}/KMeansPMMLModelExportSuite.scala | 12 ++-- ...gisticRegressionPMMLModelExportSuite.scala | 9 +-- .../export/PMMLModelExportFactorySuite.scala} | 30 ++++----- 13 files changed, 110 insertions(+), 168 deletions(-) delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/export/ModelExporter.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala rename mllib/src/main/scala/org/apache/spark/mllib/{export/pmml => pmml/export}/GeneralizedLinearPMMLModelExport.scala (98%) rename mllib/src/main/scala/org/apache/spark/mllib/{export/pmml => pmml/export}/KMeansPMMLModelExport.scala (98%) rename mllib/src/main/scala/org/apache/spark/mllib/{export/pmml => pmml/export}/LogisticRegressionPMMLModelExport.scala (98%) rename mllib/src/main/scala/org/apache/spark/mllib/{export/pmml => pmml/export}/PMMLModelExport.scala (91%) rename mllib/src/main/scala/org/apache/spark/mllib/{export/ModelExportFactory.scala => pmml/export/PMMLModelExportFactory.scala} (67%) rename mllib/src/test/scala/org/apache/spark/mllib/{export/pmml => pmml/export}/GeneralizedLinearPMMLModelExportSuite.scala (85%) rename mllib/src/test/scala/org/apache/spark/mllib/{export/pmml => pmml/export}/KMeansPMMLModelExportSuite.scala (84%) rename mllib/src/test/scala/org/apache/spark/mllib/{export/pmml => pmml/export}/LogisticRegressionPMMLModelExportSuite.scala (88%) rename mllib/src/test/scala/org/apache/spark/mllib/{export/ModelExportFactorySuite.scala => pmml/export/PMMLModelExportFactorySuite.scala} (68%) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala deleted file mode 100644 index a6daa6686cf5a..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExport.scala +++ /dev/null @@ -1,22 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.export - -private[mllib] trait ModelExport { - -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala deleted file mode 100644 index f59040f3a3440..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportType.scala +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.export - -/** - * Defines export types. - * - PMML exports the machine learning models in an XML-based file format - * called Predictive Model Markup Language developed by the Data Mining Group (www.dmg.org). - */ -private[mllib] object ModelExportType extends Enumeration{ - - type ModelExportType = Value - val PMML = Value - -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExporter.scala b/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExporter.scala deleted file mode 100644 index af35fee5eec06..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExporter.scala +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.export - -import java.io.File -import javax.xml.transform.stream.StreamResult - -import org.jpmml.model.JAXBUtil - -import org.apache.spark.mllib.export.pmml.PMMLModelExport - -object ModelExporter { - - /** - * Export the input model to the stream result in PMML format - */ - def toPMML(inputModel: Any, streamResult: StreamResult): Unit = { - val modelExport = ModelExportFactory.createModelExport(inputModel, ModelExportType.PMML) - val pmml = modelExport.asInstanceOf[PMMLModelExport].getPmml() - JAXBUtil.marshalPMML(pmml, streamResult) - } - - /** - * Export the input model to a local path in PMML format - */ - def toPMML(inputModel: Any, localPath: String): Unit = { - toPMML(inputModel, new StreamResult(new File(localPath))) - } - -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala new file mode 100644 index 0000000000000..b0ebc85b3719b --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.pmml + +import java.io.File +import java.io.OutputStream +import java.io.StringWriter +import javax.xml.transform.stream.StreamResult +import org.jpmml.model.JAXBUtil +import org.apache.spark.mllib.pmml.export.PMMLModelExport +import org.apache.spark.mllib.pmml.export.PMMLModelExportFactory + +/** + * Export model to the PMML format + * Predictive Model Markup Language (PMML) in an XML-based file format + * developed by the Data Mining Group (www.dmg.org). + */ +trait PMMLExportable { + + /** + * Export the model to the stream result in PMML format + */ + private def toPMML(streamResult: StreamResult): Unit = { + val pmmlModelExport = PMMLModelExportFactory.createPMMLModelExport(this) + JAXBUtil.marshalPMML(pmmlModelExport.getPmml(), streamResult) + } + + /** + * Export the model to a local File in PMML format + */ + def toPMML(localPath: String): Unit = { + toPMML(new StreamResult(new File(localPath))) + } + + /** + * Export the model to the Outputtream in PMML format + */ + def toPMML(outputStream: OutputStream): Unit = { + toPMML(new StreamResult(outputStream)) + } + + /** + * Export the model to a String in PMML format + */ + def toPMML(): String = { + var writer = new StringWriter(); + toPMML(new StreamResult(writer)) + return writer.toString(); + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala similarity index 98% rename from mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala rename to mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala index 8b3d1ce9e3e0f..94bbd705a9b69 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.export.pmml +package org.apache.spark.mllib.pmml.export import org.dmg.pmml.DataDictionary import org.dmg.pmml.DataField @@ -29,7 +29,6 @@ import org.dmg.pmml.NumericPredictor import org.dmg.pmml.OpType import org.dmg.pmml.RegressionModel import org.dmg.pmml.RegressionTable - import org.apache.spark.mllib.regression.GeneralizedLinearModel /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala similarity index 98% rename from mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala rename to mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala index c10d48fb8eb4c..901fbb6858a20 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.export.pmml +package org.apache.spark.mllib.pmml.export import org.dmg.pmml.Array.Type import org.dmg.pmml.Cluster @@ -35,7 +35,6 @@ import org.dmg.pmml.MiningFunctionType import org.dmg.pmml.MiningSchema import org.dmg.pmml.OpType import org.dmg.pmml.SquaredEuclidean - import org.apache.spark.mllib.clustering.KMeansModel /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala similarity index 98% rename from mllib/src/main/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExport.scala rename to mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala index 0d65bc9ddc627..0b1d1d465b939 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.export.pmml +package org.apache.spark.mllib.pmml.export import org.dmg.pmml.DataDictionary import org.dmg.pmml.DataField @@ -29,8 +29,8 @@ import org.dmg.pmml.NumericPredictor import org.dmg.pmml.OpType import org.dmg.pmml.RegressionModel import org.dmg.pmml.RegressionTable -import org.apache.spark.mllib.classification.LogisticRegressionModel import org.dmg.pmml.RegressionNormalizationMethodType +import org.apache.spark.mllib.classification.LogisticRegressionModel /** * PMML Model Export for LogisticRegressionModel class diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala similarity index 91% rename from mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala rename to mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala index cf9f993e88e59..14ab5e0d2c7b6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/pmml/PMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala @@ -15,21 +15,17 @@ * limitations under the License. */ -package org.apache.spark.mllib.export.pmml +package org.apache.spark.mllib.pmml.export import java.text.SimpleDateFormat import java.util.Date - import scala.beans.BeanProperty - import org.dmg.pmml.Application import org.dmg.pmml.Header import org.dmg.pmml.PMML import org.dmg.pmml.Timestamp -import org.apache.spark.mllib.export.ModelExport - -private[mllib] trait PMMLModelExport extends ModelExport{ +private[mllib] trait PMMLModelExport { /** * Holder of the exported model in PMML format diff --git a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala similarity index 67% rename from mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala rename to mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala index 618fe79a7b14a..f97b1ace61ef9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/export/ModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala @@ -15,30 +15,23 @@ * limitations under the License. */ -package org.apache.spark.mllib.export +package org.apache.spark.mllib.pmml.export import org.apache.spark.mllib.classification.LogisticRegressionModel import org.apache.spark.mllib.classification.SVMModel import org.apache.spark.mllib.clustering.KMeansModel -import org.apache.spark.mllib.export.ModelExportType.ModelExportType -import org.apache.spark.mllib.export.ModelExportType.PMML -import org.apache.spark.mllib.export.pmml.GeneralizedLinearPMMLModelExport -import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport -import org.apache.spark.mllib.export.pmml.LogisticRegressionPMMLModelExport import org.apache.spark.mllib.regression.LassoModel import org.apache.spark.mllib.regression.LinearRegressionModel import org.apache.spark.mllib.regression.RidgeRegressionModel -private[mllib] object ModelExportFactory { +private[mllib] object PMMLModelExportFactory { /** - * Factory object to help creating the necessary ModelExport implementation - * taking as input the ModelExportType (for example PMML) - * and the machine learning model (for example KMeansModel). + * Factory object to help creating the necessary PMMLModelExport implementation + * taking as input the machine learning model (for example KMeansModel). */ - def createModelExport(model: Any, exportType: ModelExportType): ModelExport = { - return exportType match{ - case PMML => model match{ + def createPMMLModelExport(model: Any): PMMLModelExport = { + return model match{ case kmeans: KMeansModel => new KMeansPMMLModelExport(kmeans) case linearRegression: LinearRegressionModel => @@ -54,10 +47,8 @@ private[mllib] object ModelExportFactory { case logisticRegression: LogisticRegressionModel => new LogisticRegressionPMMLModelExport(logisticRegression, "logistic regression") case _ => - throw new IllegalArgumentException("Export not supported for model: " + model.getClass) - } - case _ => throw new IllegalArgumentException("Export type not supported:" + exportType) - } + throw new IllegalArgumentException("PMML Export not supported for model: " + model.getClass) + } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala similarity index 85% rename from mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala index e27c193a7f7f8..9b0c81b5b099f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/GeneralizedLinearPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala @@ -15,14 +15,11 @@ * limitations under the License. */ -package org.apache.spark.mllib.export.pmml +package org.apache.spark.mllib.pmml.export import org.dmg.pmml.RegressionModel import org.scalatest.FunSuite - import org.apache.spark.mllib.classification.SVMModel -import org.apache.spark.mllib.export.ModelExportFactory -import org.apache.spark.mllib.export.ModelExportType import org.apache.spark.mllib.regression.LassoModel import org.apache.spark.mllib.regression.LinearRegressionModel import org.apache.spark.mllib.regression.RidgeRegressionModel @@ -41,7 +38,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label); //act by exporting the model to the PMML format - val linearModelExport = ModelExportFactory.createModelExport(linearRegressionModel, ModelExportType.PMML) + val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) //assert that the PMML format is as expected assert(linearModelExport.isInstanceOf[PMMLModelExport]) var pmml = linearModelExport.asInstanceOf[PMMLModelExport].getPmml() @@ -54,7 +51,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ .getRegressionTables().get(0).getNumericPredictors().size() === linearRegressionModel.weights.size) //act - val ridgeModelExport = ModelExportFactory.createModelExport(ridgeRegressionModel, ModelExportType.PMML) + val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) //assert that the PMML format is as expected assert(ridgeModelExport.isInstanceOf[PMMLModelExport]) pmml = ridgeModelExport.asInstanceOf[PMMLModelExport].getPmml() @@ -67,7 +64,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ .getRegressionTables().get(0).getNumericPredictors().size() === ridgeRegressionModel.weights.size) //act - val lassoModelExport = ModelExportFactory.createModelExport(lassoModel, ModelExportType.PMML) + val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) //assert that the PMML format is as expected assert(lassoModelExport.isInstanceOf[PMMLModelExport]) pmml = lassoModelExport.asInstanceOf[PMMLModelExport].getPmml() @@ -80,7 +77,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ .getRegressionTables().get(0).getNumericPredictors().size() === lassoModel.weights.size) //act - val svmModelExport = ModelExportFactory.createModelExport(svmModel, ModelExportType.PMML) + val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) //assert that the PMML format is as expected assert(svmModelExport.isInstanceOf[PMMLModelExport]) pmml = svmModelExport.asInstanceOf[PMMLModelExport].getPmml() @@ -93,10 +90,10 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ .getRegressionTables().get(0).getNumericPredictors().size() === svmModel.weights.size) //manual checking - //ModelExporter.toPMML(linearRegressionModel,"/tmp/linearregression.xml") - //ModelExporter.toPMML(ridgeRegressionModel,"/tmp/ridgeregression.xml") - //ModelExporter.toPMML(lassoModel,"/tmp/lassoregression.xml") - //ModelExporter.toPMML(svmModel,"/tmp/linearsvm.xml") + //linearRegressionModel.toPMML("/tmp/linearregression.xml") + //ridgeRegressionModel.toPMML("/tmp/ridgeregression.xml") + //lassoModel.toPMML("/tmp/lassoregression.xml") + //svmModel.toPMML("/tmp/linearsvm.xml") } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala similarity index 84% rename from mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala index 471e311b53653..00682b0f78190 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/KMeansPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala @@ -15,14 +15,11 @@ * limitations under the License. */ -package org.apache.spark.mllib.export.pmml +package org.apache.spark.mllib.pmml.export import org.dmg.pmml.ClusteringModel import org.scalatest.FunSuite - import org.apache.spark.mllib.clustering.KMeansModel -import org.apache.spark.mllib.export.ModelExportFactory -import org.apache.spark.mllib.export.ModelExportType import org.apache.spark.mllib.linalg.Vectors class KMeansPMMLModelExportSuite extends FunSuite{ @@ -38,7 +35,7 @@ class KMeansPMMLModelExportSuite extends FunSuite{ val kmeansModel = new KMeansModel(clusterCenters); //act by exporting the model to the PMML format - val modelExport = ModelExportFactory.createModelExport(kmeansModel, ModelExportType.PMML) + val modelExport = PMMLModelExportFactory.createPMMLModelExport(kmeansModel) //assert that the PMML format is as expected assert(modelExport.isInstanceOf[PMMLModelExport]) @@ -51,8 +48,9 @@ class KMeansPMMLModelExportSuite extends FunSuite{ assert(pmml.getModels().get(0).asInstanceOf[ClusteringModel].getNumberOfClusters() === clusterCenters.size) //manual checking - //ModelExporter.toPMML(kmeansModel,new StreamResult(System.out)) - //ModelExporter.toPMML(kmeansModel,"/tmp/kmeans.xml") + //kmeansModel.toPMML("/tmp/kmeans.xml") + //kmeansModel.toPMML(System.out) + //System.out.println(kmeansModel.toPMML()) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala similarity index 88% rename from mllib/src/test/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExportSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala index 0bb6c9a60a485..b96194d47b882 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/pmml/LogisticRegressionPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala @@ -15,14 +15,11 @@ * limitations under the License. */ -package org.apache.spark.mllib.export.pmml +package org.apache.spark.mllib.pmml.export import org.dmg.pmml.RegressionModel import org.scalatest.FunSuite - import org.apache.spark.mllib.classification.LogisticRegressionModel -import org.apache.spark.mllib.export.ModelExportFactory -import org.apache.spark.mllib.export.ModelExportType import org.apache.spark.mllib.util.LinearDataGenerator class LogisticRegressionPMMLModelExportSuite extends FunSuite{ @@ -35,7 +32,7 @@ class LogisticRegressionPMMLModelExportSuite extends FunSuite{ val logisticRegressionModel = new LogisticRegressionModel(linearInput(0).features, linearInput(0).label); //act by exporting the model to the PMML format - val logisticModelExport = ModelExportFactory.createModelExport(logisticRegressionModel, ModelExportType.PMML) + val logisticModelExport = PMMLModelExportFactory.createPMMLModelExport(logisticRegressionModel) //assert that the PMML format is as expected assert(logisticModelExport.isInstanceOf[PMMLModelExport]) var pmml = logisticModelExport.asInstanceOf[PMMLModelExport].getPmml() @@ -55,7 +52,7 @@ class LogisticRegressionPMMLModelExportSuite extends FunSuite{ .getRegressionTables().get(1).getNumericPredictors().size() === 0) //manual checking - //ModelExporter.toPMML(logisticRegressionModel,"/tmp/logisticregression.xml") + //logisticRegressionModel.toPMML("/tmp/logisticregression.xml") } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala similarity index 68% rename from mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala index d63a544ebdf97..5b34e5a8329fb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/export/ModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.mllib.export +package org.apache.spark.mllib.pmml.export import org.scalatest.FunSuite - import org.apache.spark.mllib.classification.LogisticRegressionModel import org.apache.spark.mllib.classification.SVMModel import org.apache.spark.mllib.clustering.KMeansModel @@ -27,13 +26,10 @@ import org.apache.spark.mllib.regression.LassoModel import org.apache.spark.mllib.regression.LinearRegressionModel import org.apache.spark.mllib.regression.RidgeRegressionModel import org.apache.spark.mllib.util.LinearDataGenerator -import org.apache.spark.mllib.export.pmml.GeneralizedLinearPMMLModelExport -import org.apache.spark.mllib.export.pmml.KMeansPMMLModelExport -import org.apache.spark.mllib.export.pmml.LogisticRegressionPMMLModelExport -class ModelExportFactorySuite extends FunSuite{ +class PMMLModelExportFactorySuite extends FunSuite{ - test("ModelExportFactory create KMeansPMMLModelExport when passing a KMeansModel") { + test("PMMLModelExportFactory create KMeansPMMLModelExport when passing a KMeansModel") { //arrange val clusterCenters = Array( @@ -44,14 +40,14 @@ class ModelExportFactorySuite extends FunSuite{ val kmeansModel = new KMeansModel(clusterCenters); //act - val modelExport = ModelExportFactory.createModelExport(kmeansModel, ModelExportType.PMML) + val modelExport = PMMLModelExportFactory.createPMMLModelExport(kmeansModel) //assert assert(modelExport.isInstanceOf[KMeansPMMLModelExport]) } - test("ModelExportFactory create GeneralizedLinearPMMLModelExport when passing a " + test("PMMLModelExportFactory create GeneralizedLinearPMMLModelExport when passing a " +"LinearRegressionModel, RidgeRegressionModel, LassoModel or SVMModel") { //arrange @@ -63,28 +59,28 @@ class ModelExportFactorySuite extends FunSuite{ val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) //act - val linearModelExport = ModelExportFactory.createModelExport(linearRegressionModel, ModelExportType.PMML) + val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) //assert assert(linearModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) //act - val ridgeModelExport = ModelExportFactory.createModelExport(ridgeRegressionModel, ModelExportType.PMML) + val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) //assert assert(ridgeModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) //act - val lassoModelExport = ModelExportFactory.createModelExport(lassoModel, ModelExportType.PMML) + val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) //assert assert(lassoModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) //act - val svmModelExport = ModelExportFactory.createModelExport(svmModel, ModelExportType.PMML) + val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) //assert assert(svmModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) } - test("ModelExportFactory create LogisticRegressionPMMLModelExport when passing a LogisticRegressionModel") { + test("PMMLModelExportFactory create LogisticRegressionPMMLModelExport when passing a LogisticRegressionModel") { //arrange val linearInput = LinearDataGenerator.generateLinearInput( @@ -92,13 +88,13 @@ class ModelExportFactorySuite extends FunSuite{ val logisticRegressionModel = new LogisticRegressionModel(linearInput(0).features, linearInput(0).label); //act - val logisticRegressionModelExport = ModelExportFactory.createModelExport(logisticRegressionModel, ModelExportType.PMML) + val logisticRegressionModelExport = PMMLModelExportFactory.createPMMLModelExport(logisticRegressionModel) //assert assert(logisticRegressionModelExport.isInstanceOf[LogisticRegressionPMMLModelExport]) } - test("ModelExportFactory throw IllegalArgumentException when passing an unsupported model") { + test("PMMLModelExportFactory throw IllegalArgumentException when passing an unsupported model") { //arrange val invalidModel = new Object; @@ -106,7 +102,7 @@ class ModelExportFactorySuite extends FunSuite{ //assert intercept[IllegalArgumentException] { //act - ModelExportFactory.createModelExport(invalidModel, ModelExportType.PMML) + PMMLModelExportFactory.createPMMLModelExport(invalidModel) } } From f46c75cdbea8170c332026ffd05d5ca1667d50c6 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sun, 8 Feb 2015 17:54:19 +0000 Subject: [PATCH 27/41] [SPARK-1406] Added PMMLExportable to supported models --- .../apache/spark/mllib/classification/LogisticRegression.scala | 3 ++- .../main/scala/org/apache/spark/mllib/classification/SVM.scala | 3 ++- .../scala/org/apache/spark/mllib/clustering/KMeansModel.scala | 3 ++- .../main/scala/org/apache/spark/mllib/regression/Lasso.scala | 3 ++- .../org/apache/spark/mllib/regression/LinearRegression.scala | 3 ++- .../org/apache/spark/mllib/regression/RidgeRegression.scala | 3 ++- 6 files changed, 12 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 9a391bfff76a3..43dc93c353e44 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -23,6 +23,7 @@ import org.apache.spark.mllib.classification.impl.GLMClassificationModel import org.apache.spark.mllib.linalg.BLAS.dot import org.apache.spark.mllib.linalg.{DenseVector, Vector} import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.{DataValidators, Saveable, Loader} import org.apache.spark.rdd.RDD @@ -46,7 +47,7 @@ class LogisticRegressionModel ( val numFeatures: Int, val numClasses: Int) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable - with Saveable { + with Saveable with PMMLExportable { if (numClasses == 2) { require(weights.size == numFeatures, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 24d31e62ba500..5cc83ae4ab5ab 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -22,6 +22,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.classification.impl.GLMClassificationModel import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.{DataValidators, Saveable, Loader} import org.apache.spark.rdd.RDD @@ -37,7 +38,7 @@ class SVMModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable - with Saveable { + with Saveable with PMMLExportable { private var threshold: Option[Double] = Some(0.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 3b95a9e6936e8..644db66f9c5d0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -21,11 +21,12 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.pmml.PMMLExportable /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ -class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable { +class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable with PMMLExportable { /** Total number of clusters. */ def k: Int = clusterCenters.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 1159e59fff5f6..8a9f60eea3c01 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression.impl.GLMRegressionModel import org.apache.spark.mllib.util.{Saveable, Loader} import org.apache.spark.rdd.RDD @@ -34,7 +35,7 @@ class LassoModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) - with RegressionModel with Serializable with Saveable { + with RegressionModel with Serializable with Saveable with PMMLExportable { override protected def predictPoint( dataMatrix: Vector, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 0136dcfdceaef..b212bb7d55fe2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression.impl.GLMRegressionModel import org.apache.spark.mllib.util.{Saveable, Loader} import org.apache.spark.rdd.RDD @@ -34,7 +35,7 @@ class LinearRegressionModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable - with Saveable { + with Saveable with PMMLExportable { override protected def predictPoint( dataMatrix: Vector, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index f2a5f1db1ece6..8700846f8c679 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression.impl.GLMRegressionModel import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD @@ -35,7 +36,7 @@ class RidgeRegressionModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) - with RegressionModel with Serializable with Saveable { + with RegressionModel with Serializable with Saveable with PMMLExportable { override protected def predictPoint( dataMatrix: Vector, From 7a949d0b3f5b397f4df4fcd70f3658cc66e8f04e Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sun, 8 Feb 2015 18:14:34 +0000 Subject: [PATCH 28/41] [SPARK-1406] Fixed scala style --- .../spark/mllib/pmml/export/PMMLModelExportFactory.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala index f97b1ace61ef9..a33ac14bbc446 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala @@ -47,7 +47,8 @@ private[mllib] object PMMLModelExportFactory { case logisticRegression: LogisticRegressionModel => new LogisticRegressionPMMLModelExport(logisticRegression, "logistic regression") case _ => - throw new IllegalArgumentException("PMML Export not supported for model: " + model.getClass) + throw new IllegalArgumentException("PMML Export not supported for model: " + + model.getClass) } } From b25bbf7ae7eb92541f59f5cfe7495280d60fca43 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sun, 1 Mar 2015 16:25:15 +0000 Subject: [PATCH 29/41] [SPARK-1406] Added export of pmml to distributed file system using the spark context --- .../org/apache/spark/mllib/pmml/PMMLExportable.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala index b0ebc85b3719b..988271fae292f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala @@ -22,12 +22,13 @@ import java.io.OutputStream import java.io.StringWriter import javax.xml.transform.stream.StreamResult import org.jpmml.model.JAXBUtil +import org.apache.spark.SparkContext import org.apache.spark.mllib.pmml.export.PMMLModelExport import org.apache.spark.mllib.pmml.export.PMMLModelExportFactory /** * Export model to the PMML format - * Predictive Model Markup Language (PMML) in an XML-based file format + * Predictive Model Markup Language (PMML) is an XML-based file format * developed by the Data Mining Group (www.dmg.org). */ trait PMMLExportable { @@ -47,6 +48,14 @@ trait PMMLExportable { toPMML(new StreamResult(new File(localPath))) } + /** + * Export the model to a distributed file in PMML format + */ + def toPMML(sc: SparkContext, path: String): Unit = { + val pmml = toPMML() + sc.parallelize(Array(pmml),1).saveAsTextFile(path) + } + /** * Export the model to the Outputtream in PMML format */ From e2ffae8987d5cfaaf8a94da1ab95f6e3ca3ba1b8 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sun, 19 Apr 2015 17:06:39 +0100 Subject: [PATCH 30/41] fixed scala style --- .../spark/mllib/clustering/KMeansModel.scala | 3 +- ...eneralizedLinearPMMLModelExportSuite.scala | 80 +++++++++++-------- .../export/KMeansPMMLModelExportSuite.scala | 24 +++--- ...gisticRegressionPMMLModelExportSuite.scala | 32 +++++--- .../export/PMMLModelExportFactorySuite.scala | 62 +++++++------- 5 files changed, 115 insertions(+), 86 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 1dcc04f17444b..ba228b11fcec3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -35,7 +35,8 @@ import org.apache.spark.sql.Row /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ -class KMeansModel (val clusterCenters: Array[Vector]) extends Saveable with Serializable with PMMLExportable { +class KMeansModel ( + val clusterCenters: Array[Vector]) extends Saveable with Serializable with PMMLExportable { /** A Java-friendly constructor that takes an Iterable of Vectors. */ def this(centers: java.lang.Iterable[Vector]) = this(centers.asScala.toArray) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala index 9b0c81b5b099f..b160b6881dc43 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala @@ -29,71 +29,85 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ test("GeneralizedLinearPMMLModelExport generate PMML format") { - //arrange models to test + // arrange models to test val linearInput = LinearDataGenerator.generateLinearInput( 3.0, Array(10.0, 10.0), 1, 17) - val linearRegressionModel = new LinearRegressionModel(linearInput(0).features, linearInput(0).label); - val ridgeRegressionModel = new RidgeRegressionModel(linearInput(0).features, linearInput(0).label); + val linearRegressionModel = new LinearRegressionModel( + linearInput(0).features, linearInput(0).label); + val ridgeRegressionModel = new RidgeRegressionModel( + linearInput(0).features, linearInput(0).label); val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label); val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label); - //act by exporting the model to the PMML format - val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) - //assert that the PMML format is as expected + // act by exporting the model to the PMML format + val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) + // assert that the PMML format is as expected assert(linearModelExport.isInstanceOf[PMMLModelExport]) var pmml = linearModelExport.asInstanceOf[PMMLModelExport].getPmml() assert(pmml.getHeader().getDescription() === "linear regression") - //check that the number of fields match the weights size - assert(pmml.getDataDictionary().getNumberOfFields() === linearRegressionModel.weights.size + 1) - //this verify that there is a model attached to the pmml object and the model is a regression one - //it also verifies that the pmml model has a regression table with the same number of predictors of the model weights + // check that the number of fields match the weights size + assert(pmml.getDataDictionary().getNumberOfFields() + === linearRegressionModel.weights.size + 1) + // this verify that there is a model attached to the pmml object + // and the model is a regression one + // it also verifies that the pmml model has a regression table + // with the same number of predictors of the model weights assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(0).getNumericPredictors().size() === linearRegressionModel.weights.size) + .getRegressionTables().get(0).getNumericPredictors().size() + === linearRegressionModel.weights.size) - //act + // act val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) - //assert that the PMML format is as expected + // assert that the PMML format is as expected assert(ridgeModelExport.isInstanceOf[PMMLModelExport]) pmml = ridgeModelExport.asInstanceOf[PMMLModelExport].getPmml() assert(pmml.getHeader().getDescription() === "ridge regression") - //check that the number of fields match the weights size + // check that the number of fields match the weights size assert(pmml.getDataDictionary().getNumberOfFields() === ridgeRegressionModel.weights.size + 1) - //this verify that there is a model attached to the pmml object and the model is a regression one - //it also verifies that the pmml model has a regression table with the same number of predictors of the model weights + // this verify that there is a model attached to the pmml object + // and the model is a regression one + // it also verifies that the pmml model has a regression table + // with the same number of predictors of the model weights assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(0).getNumericPredictors().size() === ridgeRegressionModel.weights.size) + .getRegressionTables().get(0).getNumericPredictors().size() + === ridgeRegressionModel.weights.size) - //act + // act val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) - //assert that the PMML format is as expected + // assert that the PMML format is as expected assert(lassoModelExport.isInstanceOf[PMMLModelExport]) pmml = lassoModelExport.asInstanceOf[PMMLModelExport].getPmml() assert(pmml.getHeader().getDescription() === "lasso regression") - //check that the number of fields match the weights size + // check that the number of fields match the weights size assert(pmml.getDataDictionary().getNumberOfFields() === lassoModel.weights.size + 1) - //this verify that there is a model attached to the pmml object and the model is a regression one - //it also verifies that the pmml model has a regression table with the same number of predictors of the model weights + // this verify that there is a model attached to the pmml object + // and the model is a regression one + // it also verifies that the pmml model has a regression table + // with the same number of predictors of the model weights assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] .getRegressionTables().get(0).getNumericPredictors().size() === lassoModel.weights.size) - //act + // act val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) - //assert that the PMML format is as expected + // assert that the PMML format is as expected assert(svmModelExport.isInstanceOf[PMMLModelExport]) pmml = svmModelExport.asInstanceOf[PMMLModelExport].getPmml() - assert(pmml.getHeader().getDescription() === "linear SVM: if predicted value > 0, the outcome is positive, or negative otherwise") - //check that the number of fields match the weights size + assert(pmml.getHeader().getDescription() + === "linear SVM: if predicted value > 0, the outcome is positive, or negative otherwise") + // check that the number of fields match the weights size assert(pmml.getDataDictionary().getNumberOfFields() === svmModel.weights.size + 1) - //this verify that there is a model attached to the pmml object and the model is a regression one - //it also verifies that the pmml model has a regression table with the same number of predictors of the model weights + // this verify that there is a model attached to the pmml object + // and the model is a regression one + // it also verifies that the pmml model has a regression table + // with the same number of predictors of the model weights assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] .getRegressionTables().get(0).getNumericPredictors().size() === svmModel.weights.size) - //manual checking - //linearRegressionModel.toPMML("/tmp/linearregression.xml") - //ridgeRegressionModel.toPMML("/tmp/ridgeregression.xml") - //lassoModel.toPMML("/tmp/lassoregression.xml") - //svmModel.toPMML("/tmp/linearsvm.xml") + // manual checking + // linearRegressionModel.toPMML("/tmp/linearregression.xml") + // ridgeRegressionModel.toPMML("/tmp/ridgeregression.xml") + // lassoModel.toPMML("/tmp/lassoregression.xml") + // svmModel.toPMML("/tmp/linearsvm.xml") } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala index 00682b0f78190..83def5ace0cbc 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala @@ -26,7 +26,7 @@ class KMeansPMMLModelExportSuite extends FunSuite{ test("KMeansPMMLModelExport generate PMML format") { - //arrange model to test + // arrange model to test val clusterCenters = Array( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), @@ -34,23 +34,25 @@ class KMeansPMMLModelExportSuite extends FunSuite{ ) val kmeansModel = new KMeansModel(clusterCenters); - //act by exporting the model to the PMML format + // act by exporting the model to the PMML format val modelExport = PMMLModelExportFactory.createPMMLModelExport(kmeansModel) - //assert that the PMML format is as expected + // assert that the PMML format is as expected assert(modelExport.isInstanceOf[PMMLModelExport]) val pmml = modelExport.asInstanceOf[PMMLModelExport].getPmml() assert(pmml.getHeader().getDescription() === "k-means clustering") - //check that the number of fields match the single vector size + // 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 - assert(pmml.getModels().get(0).asInstanceOf[ClusteringModel].getNumberOfClusters() === clusterCenters.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 + assert(pmml.getModels().get(0).asInstanceOf[ClusteringModel].getNumberOfClusters() + === clusterCenters.size) - //manual checking - //kmeansModel.toPMML("/tmp/kmeans.xml") - //kmeansModel.toPMML(System.out) - //System.out.println(kmeansModel.toPMML()) + // manual checking + // kmeansModel.toPMML("/tmp/kmeans.xml") + // kmeansModel.toPMML(System.out) + // System.out.println(kmeansModel.toPMML()) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala index b96194d47b882..ca5d8ca8b2f5b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala @@ -26,33 +26,39 @@ class LogisticRegressionPMMLModelExportSuite extends FunSuite{ test("LogisticRegressionPMMLModelExport generate PMML format") { - //arrange models to test + // arrange models to test val linearInput = LinearDataGenerator.generateLinearInput( 3.0, Array(10.0, 10.0), 1, 17) - val logisticRegressionModel = new LogisticRegressionModel(linearInput(0).features, linearInput(0).label); + val logisticRegressionModel = new LogisticRegressionModel( + linearInput(0).features, linearInput(0).label); - //act by exporting the model to the PMML format - val logisticModelExport = PMMLModelExportFactory.createPMMLModelExport(logisticRegressionModel) - //assert that the PMML format is as expected + // act by exporting the model to the PMML format + val logisticModelExport = PMMLModelExportFactory + .createPMMLModelExport(logisticRegressionModel) + // assert that the PMML format is as expected assert(logisticModelExport.isInstanceOf[PMMLModelExport]) var pmml = logisticModelExport.asInstanceOf[PMMLModelExport].getPmml() assert(pmml.getHeader().getDescription() === "logistic regression") - //check that the number of fields match the weights size - assert(pmml.getDataDictionary().getNumberOfFields() === logisticRegressionModel.weights.size + 1) - //this verify that there is a model attached to the pmml object and the model is a regression one - //it also verifies that the pmml model has a regression table (for target category 1) with the same number of predictors of the model weights + // check that the number of fields match the weights size + assert( + pmml.getDataDictionary().getNumberOfFields() === logisticRegressionModel.weights.size + 1) + // this verify that there is a model attached to the pmml object + // and the model is a regression one + // it also verifies that the pmml model has a regression table (for target category 1) + // with the same number of predictors of the model weights assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] .getRegressionTables().get(0).getTargetCategory() === "1") assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(0).getNumericPredictors().size() === logisticRegressionModel.weights.size) - //verify if there is a second table with target category 0 and no predictors + .getRegressionTables().get(0).getNumericPredictors().size() + === logisticRegressionModel.weights.size) + // verify if there is a second table with target category 0 and no predictors assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] .getRegressionTables().get(1).getTargetCategory() === "0") assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] .getRegressionTables().get(1).getNumericPredictors().size() === 0) - //manual checking - //logisticRegressionModel.toPMML("/tmp/logisticregression.xml") + // manual checking + // logisticRegressionModel.toPMML("/tmp/logisticregression.xml") } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala index 5b34e5a8329fb..a54cd247a120f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala @@ -31,7 +31,7 @@ class PMMLModelExportFactorySuite extends FunSuite{ test("PMMLModelExportFactory create KMeansPMMLModelExport when passing a KMeansModel") { - //arrange + // arrange val clusterCenters = Array( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), @@ -39,70 +39,76 @@ class PMMLModelExportFactorySuite extends FunSuite{ ) val kmeansModel = new KMeansModel(clusterCenters); - //act + // act val modelExport = PMMLModelExportFactory.createPMMLModelExport(kmeansModel) - //assert + // assert assert(modelExport.isInstanceOf[KMeansPMMLModelExport]) } test("PMMLModelExportFactory create GeneralizedLinearPMMLModelExport when passing a " - +"LinearRegressionModel, RidgeRegressionModel, LassoModel or SVMModel") { + + "LinearRegressionModel, RidgeRegressionModel, LassoModel or SVMModel") { - //arrange + // arrange val linearInput = LinearDataGenerator.generateLinearInput( 3.0, Array(10.0, 10.0), 1, 17) - val linearRegressionModel = new LinearRegressionModel(linearInput(0).features, linearInput(0).label) - val ridgeRegressionModel = new RidgeRegressionModel(linearInput(0).features, linearInput(0).label) + val linearRegressionModel = new LinearRegressionModel( + linearInput(0).features, linearInput(0).label) + val ridgeRegressionModel = new RidgeRegressionModel( + linearInput(0).features, linearInput(0).label) val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label) val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) - //act - val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) - //assert + // act + val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) + // assert assert(linearModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) - //act - val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) - //assert + // act + val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) + // assert assert(ridgeModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) - //act + // act val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) - //assert + // assert assert(lassoModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) - //act + // act val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) - //assert + // assert assert(svmModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) } - test("PMMLModelExportFactory create LogisticRegressionPMMLModelExport when passing a LogisticRegressionModel") { + test("PMMLModelExportFactory create LogisticRegressionPMMLModelExport " + + "when passing a LogisticRegressionModel") { - //arrange + // arrange val linearInput = LinearDataGenerator.generateLinearInput( 3.0, Array(10.0, 10.0), 1, 17) - val logisticRegressionModel = new LogisticRegressionModel(linearInput(0).features, linearInput(0).label); + val logisticRegressionModel = new LogisticRegressionModel( + linearInput(0).features, linearInput(0).label); - //act - val logisticRegressionModelExport = PMMLModelExportFactory.createPMMLModelExport(logisticRegressionModel) - //assert + // act + val logisticRegressionModelExport = PMMLModelExportFactory + .createPMMLModelExport(logisticRegressionModel) + // assert assert(logisticRegressionModelExport.isInstanceOf[LogisticRegressionPMMLModelExport]) } - test("PMMLModelExportFactory throw IllegalArgumentException when passing an unsupported model") { + test("PMMLModelExportFactory throw IllegalArgumentException " + + "when passing an unsupported model") { - //arrange + // arrange val invalidModel = new Object; - //assert + // assert intercept[IllegalArgumentException] { - //act - PMMLModelExportFactory.createPMMLModelExport(invalidModel) + // act + PMMLModelExportFactory.createPMMLModelExport(invalidModel) } } From 1676e15ff5bec346466a095a593efdd324f6b515 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sun, 19 Apr 2015 17:17:15 +0100 Subject: [PATCH 31/41] fixed scala issue --- .../pmml/export/GeneralizedLinearPMMLModelExportSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala index b160b6881dc43..dd50112eba7cb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala @@ -57,7 +57,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ === linearRegressionModel.weights.size) // act - val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) + val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) // assert that the PMML format is as expected assert(ridgeModelExport.isInstanceOf[PMMLModelExport]) pmml = ridgeModelExport.asInstanceOf[PMMLModelExport].getPmml() From 472d75777c98439522a6e3226cfb83f7cc1dd00d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 20 Apr 2015 13:20:58 -0700 Subject: [PATCH 32/41] fix code style --- .../spark/mllib/pmml/PMMLExportable.scala | 31 +++-- .../GeneralizedLinearPMMLModelExport.scala | 70 ++++------ .../pmml/export/KMeansPMMLModelExport.scala | 122 +++++++--------- .../LogisticRegressionPMMLModelExport.scala | 79 ++++------- .../mllib/pmml/export/PMMLModelExport.scala | 24 ++-- .../pmml/export/PMMLModelExportFactory.scala | 37 +++-- ...eneralizedLinearPMMLModelExportSuite.scala | 131 ++++++++---------- .../export/KMeansPMMLModelExportSuite.scala | 36 ++--- ...gisticRegressionPMMLModelExportSuite.scala | 51 +++---- .../export/PMMLModelExportFactorySuite.scala | 97 +++++-------- 10 files changed, 273 insertions(+), 405 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala index 988271fae292f..938a7998cdf5f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala @@ -21,9 +21,10 @@ import java.io.File import java.io.OutputStream import java.io.StringWriter import javax.xml.transform.stream.StreamResult + import org.jpmml.model.JAXBUtil + import org.apache.spark.SparkContext -import org.apache.spark.mllib.pmml.export.PMMLModelExport import org.apache.spark.mllib.pmml.export.PMMLModelExportFactory /** @@ -34,42 +35,42 @@ import org.apache.spark.mllib.pmml.export.PMMLModelExportFactory trait PMMLExportable { /** - * Export the model to the stream result in PMML format - */ + * Export the model to the stream result in PMML format + */ private def toPMML(streamResult: StreamResult): Unit = { val pmmlModelExport = PMMLModelExportFactory.createPMMLModelExport(this) - JAXBUtil.marshalPMML(pmmlModelExport.getPmml(), streamResult) + JAXBUtil.marshalPMML(pmmlModelExport.getPmml, streamResult) } /** - * Export the model to a local File in PMML format - */ + * Export the model to a local file in PMML format + */ def toPMML(localPath: String): Unit = { toPMML(new StreamResult(new File(localPath))) } /** - * Export the model to a distributed file in PMML format - */ + * Export the model to a directory on a distributed file system in PMML format + */ def toPMML(sc: SparkContext, path: String): Unit = { val pmml = toPMML() - sc.parallelize(Array(pmml),1).saveAsTextFile(path) + sc.parallelize(Array(pmml), 1).saveAsTextFile(path) } /** - * Export the model to the Outputtream in PMML format - */ + * Export the model to the OutputStream in PMML format + */ def toPMML(outputStream: OutputStream): Unit = { toPMML(new StreamResult(outputStream)) } /** - * Export the model to a String in PMML format - */ + * Export the model to a String in PMML format + */ def toPMML(): String = { - var writer = new StringWriter(); + val writer = new StringWriter toPMML(new StreamResult(writer)) - return writer.toString(); + writer.toString } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala index 94bbd705a9b69..baab1a2dbf963 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala @@ -17,18 +17,10 @@ package org.apache.spark.mllib.pmml.export -import org.dmg.pmml.DataDictionary -import org.dmg.pmml.DataField -import org.dmg.pmml.DataType -import org.dmg.pmml.FieldName -import org.dmg.pmml.FieldUsageType -import org.dmg.pmml.MiningField -import org.dmg.pmml.MiningFunctionType -import org.dmg.pmml.MiningSchema -import org.dmg.pmml.NumericPredictor -import org.dmg.pmml.OpType -import org.dmg.pmml.RegressionModel -import org.dmg.pmml.RegressionTable +import scala.{Array => SArray} + +import org.dmg.pmml._ + import org.apache.spark.mllib.regression.GeneralizedLinearModel /** @@ -39,55 +31,43 @@ private[mllib] class GeneralizedLinearPMMLModelExport( description : String) extends PMMLModelExport{ + populateGeneralizedLinearPMML(model) + /** - * Export the input GeneralizedLinearModel model to PMML format + * Export the input GeneralizedLinearModel model to PMML format. */ - populateGeneralizedLinearPMML(model) - - private def populateGeneralizedLinearPMML(model : GeneralizedLinearModel): Unit = { + private def populateGeneralizedLinearPMML(model: GeneralizedLinearModel): Unit = { + pmml.getHeader.setDescription(description) - pmml.getHeader().setDescription(description) - if(model.weights.size > 0){ - - val fields = new Array[FieldName](model.weights.size) - - val dataDictionary = new DataDictionary() - - val miningSchema = new MiningSchema() - + val fields = new SArray[FieldName](model.weights.size) + val dataDictionary = new DataDictionary + val miningSchema = new MiningSchema val regressionTable = new RegressionTable(model.intercept) - val regressionModel = new RegressionModel(miningSchema,MiningFunctionType.REGRESSION) - .withModelName(description).withRegressionTables(regressionTable) - - for ( i <- 0 until model.weights.size) { + .withModelName(description) + .withRegressionTables(regressionTable) + + for (i <- 0 until model.weights.size) { fields(i) = FieldName.create("field_" + i) - dataDictionary - .withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) miningSchema - .withMiningFields(new MiningField(fields(i)) - .withUsageType(FieldUsageType.ACTIVE)) + .withMiningFields(new MiningField(fields(i)) + .withUsageType(FieldUsageType.ACTIVE)) regressionTable.withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) } // for completeness add target field - val targetField = FieldName.create("target"); - dataDictionary - .withDataFields( - new DataField(targetField, OpType.CONTINUOUS, DataType.DOUBLE) - ) - miningSchema + val targetField = FieldName.create("target") + dataDictionary.withDataFields(new DataField(targetField, OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema .withMiningFields(new MiningField(targetField) .withUsageType(FieldUsageType.TARGET)) - - dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()) - + + dataDictionary.withNumberOfFields(dataDictionary.getDataFields.size) + pmml.setDataDictionary(dataDictionary) pmml.withModels(regressionModel) - } - } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala index 901fbb6858a20..c12b275b2185c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala @@ -17,24 +17,10 @@ package org.apache.spark.mllib.pmml.export -import org.dmg.pmml.Array.Type -import org.dmg.pmml.Cluster -import org.dmg.pmml.ClusteringField -import org.dmg.pmml.ClusteringModel -import org.dmg.pmml.ClusteringModel.ModelClass -import org.dmg.pmml.CompareFunctionType -import org.dmg.pmml.ComparisonMeasure -import org.dmg.pmml.ComparisonMeasure.Kind -import org.dmg.pmml.DataDictionary -import org.dmg.pmml.DataField -import org.dmg.pmml.DataType -import org.dmg.pmml.FieldName -import org.dmg.pmml.FieldUsageType -import org.dmg.pmml.MiningField -import org.dmg.pmml.MiningFunctionType -import org.dmg.pmml.MiningSchema -import org.dmg.pmml.OpType -import org.dmg.pmml.SquaredEuclidean +import scala.{Array => SArray} + +import org.dmg.pmml._ + import org.apache.spark.mllib.clustering.KMeansModel /** @@ -42,65 +28,53 @@ import org.apache.spark.mllib.clustering.KMeansModel */ private[mllib] class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ + populateKMeansPMML(model) + /** - * Export the input KMeansModel model to PMML format + * Export the input KMeansModel model to PMML format. */ - populateKMeansPMML(model) - private def populateKMeansPMML(model : KMeansModel): Unit = { - - pmml.getHeader().setDescription("k-means clustering") - - if(model.clusterCenters.length > 0){ - - val clusterCenter = model.clusterCenters(0) - - val fields = new Array[FieldName](clusterCenter.size) - - val dataDictionary = new DataDictionary() - - val miningSchema = new MiningSchema() - - val comparisonMeasure = new ComparisonMeasure() - .withKind(Kind.DISTANCE) - .withMeasure(new SquaredEuclidean() - ) - - val clusteringModel = new ClusteringModel(miningSchema, comparisonMeasure, - MiningFunctionType.CLUSTERING, ModelClass.CENTER_BASED, model.clusterCenters.length) + pmml.getHeader.setDescription("k-means clustering") + + if (model.clusterCenters.length > 0) { + val clusterCenter = model.clusterCenters(0) + val fields = new SArray[FieldName](clusterCenter.size) + val dataDictionary = new DataDictionary + val miningSchema = new MiningSchema + val comparisonMeasure = new ComparisonMeasure() + .withKind(ComparisonMeasure.Kind.DISTANCE) + .withMeasure(new SquaredEuclidean()) + val clusteringModel = new ClusteringModel(miningSchema, comparisonMeasure, + MiningFunctionType.CLUSTERING, ClusteringModel.ModelClass.CENTER_BASED, + model.clusterCenters.length) .withModelName("k-means") - - for ( i <- 0 until clusterCenter.size) { - fields(i) = FieldName.create("field_" + i) - dataDictionary - .withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) - miningSchema - .withMiningFields(new MiningField(fields(i)) - .withUsageType(FieldUsageType.ACTIVE)) - clusteringModel.withClusteringFields( - new ClusteringField(fields(i)).withCompareFunction(CompareFunctionType.ABS_DIFF) - ) - } - - dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()) - - for ( i <- 0 until model.clusterCenters.size ) { - val cluster = new Cluster() - .withName("cluster_" + i) - .withArray(new org.dmg.pmml.Array() - .withType(Type.REAL) - .withN(clusterCenter.size) - .withValue(model.clusterCenters(i).toArray.mkString(" "))) - // we don't have the size of the single cluster but only the centroids (withValue) - // .withSize(value) - clusteringModel.withClusters(cluster) - } - - pmml.setDataDictionary(dataDictionary) - pmml.withModels(clusteringModel) - - } - + + for (i <- 0 until clusterCenter.size) { + fields(i) = FieldName.create("field_" + i) + dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema + .withMiningFields(new MiningField(fields(i)) + .withUsageType(FieldUsageType.ACTIVE)) + clusteringModel.withClusteringFields( + new ClusteringField(fields(i)).withCompareFunction(CompareFunctionType.ABS_DIFF)) + } + + dataDictionary.withNumberOfFields(dataDictionary.getDataFields.size) + + for (i <- 0 until model.clusterCenters.length) { + val cluster = new Cluster() + .withName("cluster_" + i) + .withArray(new org.dmg.pmml.Array() + .withType(Array.Type.REAL) + .withN(clusterCenter.size) + .withValue(model.clusterCenters(i).toArray.mkString(" "))) + // we don't have the size of the single cluster but only the centroids (withValue) + // .withSize(value) + clusteringModel.withClusters(cluster) + } + + pmml.setDataDictionary(dataDictionary) + pmml.withModels(clusteringModel) + } } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala index 0b1d1d465b939..75c28e1c03514 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala @@ -17,19 +17,10 @@ package org.apache.spark.mllib.pmml.export -import org.dmg.pmml.DataDictionary -import org.dmg.pmml.DataField -import org.dmg.pmml.DataType -import org.dmg.pmml.FieldName -import org.dmg.pmml.FieldUsageType -import org.dmg.pmml.MiningField -import org.dmg.pmml.MiningFunctionType -import org.dmg.pmml.MiningSchema -import org.dmg.pmml.NumericPredictor -import org.dmg.pmml.OpType -import org.dmg.pmml.RegressionModel -import org.dmg.pmml.RegressionTable -import org.dmg.pmml.RegressionNormalizationMethodType +import scala.{Array => SArray} + +import org.dmg.pmml._ + import org.apache.spark.mllib.classification.LogisticRegressionModel /** @@ -40,62 +31,46 @@ private[mllib] class LogisticRegressionPMMLModelExport( description : String) extends PMMLModelExport{ + populateLogisticRegressionPMML(model) + /** * Export the input LogisticRegressionModel model to PMML format */ - populateLogisticRegressionPMML(model) - private def populateLogisticRegressionPMML(model : LogisticRegressionModel): Unit = { + pmml.getHeader.setDescription(description) - pmml.getHeader().setDescription(description) - - if(model.weights.size > 0){ - - val fields = new Array[FieldName](model.weights.size) - - val dataDictionary = new DataDictionary() - - val miningSchema = new MiningSchema() - - val regressionTableYES = new RegressionTable(model.intercept) - .withTargetCategory("1") - - val regressionTableNO = new RegressionTable(0.0) - .withTargetCategory("0") - - val regressionModel = new RegressionModel(miningSchema,MiningFunctionType.CLASSIFICATION) - .withModelName(description) - .withNormalizationMethod(RegressionNormalizationMethodType.LOGIT) - .withRegressionTables(regressionTableYES, regressionTableNO) - - for ( i <- 0 until model.weights.size) { + if (model.weights.size > 0) { + val fields = new SArray[FieldName](model.weights.size) + val dataDictionary = new DataDictionary + val miningSchema = new MiningSchema + val regressionTableYES = new RegressionTable(model.intercept).withTargetCategory("1") + val regressionTableNO = new RegressionTable(0.0).withTargetCategory("0") + val regressionModel = new RegressionModel(miningSchema, MiningFunctionType.CLASSIFICATION) + .withModelName(description) + .withNormalizationMethod(RegressionNormalizationMethodType.LOGIT) + .withRegressionTables(regressionTableYES, regressionTableNO) + + for (i <- 0 until model.weights.size) { fields(i) = FieldName.create("field_" + i) - dataDictionary - .withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) miningSchema - .withMiningFields(new MiningField(fields(i)) - .withUsageType(FieldUsageType.ACTIVE)) - regressionTableYES - .withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) + .withMiningFields(new MiningField(fields(i)) + .withUsageType(FieldUsageType.ACTIVE)) + regressionTableYES.withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) } // add target field - val targetField = FieldName.create("target"); + val targetField = FieldName.create("target") dataDictionary - .withDataFields( - new DataField(targetField, OpType.CATEGORICAL, DataType.STRING) - ) - miningSchema + .withDataFields(new DataField(targetField, OpType.CATEGORICAL, DataType.STRING)) + miningSchema .withMiningFields(new MiningField(targetField) .withUsageType(FieldUsageType.TARGET)) - dataDictionary.withNumberOfFields((dataDictionary.getDataFields()).size()) + dataDictionary.withNumberOfFields(dataDictionary.getDataFields.size) pmml.setDataDictionary(dataDictionary) pmml.withModels(regressionModel) - } - } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala index 14ab5e0d2c7b6..ebdeae50bb32f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala @@ -19,11 +19,10 @@ package org.apache.spark.mllib.pmml.export import java.text.SimpleDateFormat import java.util.Date + import scala.beans.BeanProperty -import org.dmg.pmml.Application -import org.dmg.pmml.Header -import org.dmg.pmml.PMML -import org.dmg.pmml.Timestamp + +import org.dmg.pmml.{Application, Header, PMML, Timestamp} private[mllib] trait PMMLModelExport { @@ -31,19 +30,18 @@ private[mllib] trait PMMLModelExport { * Holder of the exported model in PMML format */ @BeanProperty - val pmml: PMML = new PMML(); + val pmml: PMML = new PMML - setHeader(pmml); + setHeader(pmml) - private def setHeader(pmml : PMML): Unit = { - val version = getClass().getPackage().getImplementationVersion() + private def setHeader(pmml: PMML): Unit = { + val version = getClass.getPackage.getImplementationVersion val app = new Application().withName("Apache Spark MLlib").withVersion(version) val timestamp = new Timestamp() - .withContent(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss").format(new Date())) + .withContent(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss").format(new Date())) val header = new Header() - .withApplication(app) - .withTimestamp(timestamp) + .withApplication(app) + .withTimestamp(timestamp) pmml.setHeader(header) - } - + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala index a33ac14bbc446..0c374a46fb562 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala @@ -31,25 +31,24 @@ private[mllib] object PMMLModelExportFactory { * taking as input the machine learning model (for example KMeansModel). */ def createPMMLModelExport(model: Any): PMMLModelExport = { - return model match{ - case kmeans: KMeansModel => - new KMeansPMMLModelExport(kmeans) - case linearRegression: LinearRegressionModel => - new GeneralizedLinearPMMLModelExport(linearRegression, "linear regression") - case ridgeRegression: RidgeRegressionModel => - new GeneralizedLinearPMMLModelExport(ridgeRegression, "ridge regression") - case lassoRegression: LassoModel => - new GeneralizedLinearPMMLModelExport(lassoRegression, "lasso regression") - case svm: SVMModel => - new GeneralizedLinearPMMLModelExport( - svm, - "linear SVM: if predicted value > 0, the outcome is positive, or negative otherwise") - case logisticRegression: LogisticRegressionModel => - new LogisticRegressionPMMLModelExport(logisticRegression, "logistic regression") - case _ => - throw new IllegalArgumentException("PMML Export not supported for model: " - + model.getClass) - } + model match { + case kmeans: KMeansModel => + new KMeansPMMLModelExport(kmeans) + case linear: LinearRegressionModel => + new GeneralizedLinearPMMLModelExport(linear, "linear regression") + case ridge: RidgeRegressionModel => + new GeneralizedLinearPMMLModelExport(ridge, "ridge regression") + case lasso: LassoModel => + new GeneralizedLinearPMMLModelExport(lasso, "lasso regression") + case svm: SVMModel => + new GeneralizedLinearPMMLModelExport(svm, + "linear SVM: if predicted value > 0, the outcome is positive, or negative otherwise") + case logistic: LogisticRegressionModel => + new LogisticRegressionPMMLModelExport(logistic, "logistic regression") + case _ => + throw new IllegalArgumentException( + "PMML Export not supported for model: " + model.getClass.getName) + } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala index dd50112eba7cb..f48d39f889cd3 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala @@ -19,96 +19,87 @@ package org.apache.spark.mllib.pmml.export import org.dmg.pmml.RegressionModel import org.scalatest.FunSuite + import org.apache.spark.mllib.classification.SVMModel import org.apache.spark.mllib.regression.LassoModel import org.apache.spark.mllib.regression.LinearRegressionModel import org.apache.spark.mllib.regression.RidgeRegressionModel import org.apache.spark.mllib.util.LinearDataGenerator -class GeneralizedLinearPMMLModelExportSuite extends FunSuite{ - - test("GeneralizedLinearPMMLModelExport generate PMML format") { +class GeneralizedLinearPMMLModelExportSuite extends FunSuite { - // arrange models to test - val linearInput = LinearDataGenerator.generateLinearInput( - 3.0, Array(10.0, 10.0), 1, 17) - val linearRegressionModel = new LinearRegressionModel( - linearInput(0).features, linearInput(0).label); - val ridgeRegressionModel = new RidgeRegressionModel( - linearInput(0).features, linearInput(0).label); - val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label); - val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label); - - // act by exporting the model to the PMML format + test("linear regression pmml export") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val linearRegressionModel = + new LinearRegressionModel(linearInput(0).features, linearInput(0).label) val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) // assert that the PMML format is as expected assert(linearModelExport.isInstanceOf[PMMLModelExport]) - var pmml = linearModelExport.asInstanceOf[PMMLModelExport].getPmml() - assert(pmml.getHeader().getDescription() === "linear regression") + val pmml = linearModelExport.getPmml + assert(pmml.getHeader.getDescription === "linear regression") // check that the number of fields match the weights size - assert(pmml.getDataDictionary().getNumberOfFields() - === linearRegressionModel.weights.size + 1) - // this verify that there is a model attached to the pmml object - // and the model is a regression one - // it also verifies that the pmml model has a regression table - // with the same number of predictors of the model weights - assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(0).getNumericPredictors().size() - === linearRegressionModel.weights.size) - - // act + assert(pmml.getDataDictionary.getNumberOfFields === linearRegressionModel.weights.size + 1) + // This verifies that there is a model attached to the pmml object and the model is a regression + // one. It also verifies that the pmml model has a regression table with the same number of + // predictors of the model weights. + val pmmlRegressionModel = pmml.getModels.get(0).asInstanceOf[RegressionModel] + assert(pmmlRegressionModel.getRegressionTables.get(0).getNumericPredictors.size + === linearRegressionModel.weights.size) + } + + test("ridge regression pmml export") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val ridgeRegressionModel = + new RidgeRegressionModel(linearInput(0).features, linearInput(0).label) val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) // assert that the PMML format is as expected assert(ridgeModelExport.isInstanceOf[PMMLModelExport]) - pmml = ridgeModelExport.asInstanceOf[PMMLModelExport].getPmml() - assert(pmml.getHeader().getDescription() === "ridge regression") + val pmml = ridgeModelExport.getPmml + assert(pmml.getHeader.getDescription === "ridge regression") // check that the number of fields match the weights size - assert(pmml.getDataDictionary().getNumberOfFields() === ridgeRegressionModel.weights.size + 1) - // this verify that there is a model attached to the pmml object - // and the model is a regression one - // it also verifies that the pmml model has a regression table - // with the same number of predictors of the model weights - assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(0).getNumericPredictors().size() - === ridgeRegressionModel.weights.size) - - // act - val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) + assert(pmml.getDataDictionary.getNumberOfFields === ridgeRegressionModel.weights.size + 1) + // This verify that there is a model attached to the pmml object and the model is a regression + // one. It also verifies that the pmml model has a regression table with the same number of + // predictors of the model weights. + val pmmlRegressionModel = pmml.getModels.get(0).asInstanceOf[RegressionModel] + assert(pmmlRegressionModel.getRegressionTables.get(0).getNumericPredictors.size + === ridgeRegressionModel.weights.size) + } + + test("lasso pmml export") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label) + val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) // assert that the PMML format is as expected assert(lassoModelExport.isInstanceOf[PMMLModelExport]) - pmml = lassoModelExport.asInstanceOf[PMMLModelExport].getPmml() - assert(pmml.getHeader().getDescription() === "lasso regression") + val pmml = lassoModelExport.getPmml + assert(pmml.getHeader.getDescription === "lasso regression") // check that the number of fields match the weights size - assert(pmml.getDataDictionary().getNumberOfFields() === lassoModel.weights.size + 1) - // this verify that there is a model attached to the pmml object - // and the model is a regression one - // it also verifies that the pmml model has a regression table - // with the same number of predictors of the model weights - assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(0).getNumericPredictors().size() === lassoModel.weights.size) - - // act + assert(pmml.getDataDictionary.getNumberOfFields === lassoModel.weights.size + 1) + // This verify that there is a model attached to the pmml object and the model is a regression + // one. It also verifies that the pmml model has a regression table with the same number of + // predictors of the model weights. + val pmmlRegressionModel = pmml.getModels.get(0).asInstanceOf[RegressionModel] + assert(pmmlRegressionModel.getRegressionTables.get(0).getNumericPredictors.size + === lassoModel.weights.size) + } + + test("svm pmml export") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) // assert that the PMML format is as expected assert(svmModelExport.isInstanceOf[PMMLModelExport]) - pmml = svmModelExport.asInstanceOf[PMMLModelExport].getPmml() - assert(pmml.getHeader().getDescription() - === "linear SVM: if predicted value > 0, the outcome is positive, or negative otherwise") + val pmml = svmModelExport.getPmml + assert(pmml.getHeader.getDescription + === "linear SVM: if predicted value > 0, the outcome is positive, or negative otherwise") // check that the number of fields match the weights size - assert(pmml.getDataDictionary().getNumberOfFields() === svmModel.weights.size + 1) - // this verify that there is a model attached to the pmml object - // and the model is a regression one - // it also verifies that the pmml model has a regression table - // with the same number of predictors of the model weights - assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(0).getNumericPredictors().size() === svmModel.weights.size) - - // manual checking - // linearRegressionModel.toPMML("/tmp/linearregression.xml") - // ridgeRegressionModel.toPMML("/tmp/ridgeregression.xml") - // lassoModel.toPMML("/tmp/lassoregression.xml") - // svmModel.toPMML("/tmp/linearsvm.xml") - - } - + assert(pmml.getDataDictionary.getNumberOfFields === svmModel.weights.size + 1) + // This verify that there is a model attached to the pmml object and the model is a regression + // one. It also verifies that the pmml model has a regression table with the same number of + // predictors of the model weights. + val pmmlRegressionModel = pmml.getModels.get(0).asInstanceOf[RegressionModel] + assert(pmmlRegressionModel.getRegressionTables.get(0).getNumericPredictors.size + === svmModel.weights.size) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala index 83def5ace0cbc..f34e2a210a9fd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala @@ -19,41 +19,31 @@ package org.apache.spark.mllib.pmml.export import org.dmg.pmml.ClusteringModel import org.scalatest.FunSuite + import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.linalg.Vectors -class KMeansPMMLModelExportSuite extends FunSuite{ +class KMeansPMMLModelExportSuite extends FunSuite { - test("KMeansPMMLModelExport generate PMML format") { - + test("KMeansPMMLModelExport generate PMML format") { // arrange model to test val clusterCenters = Array( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), - Vectors.dense(1.0, 4.0, 6.0) - ) - val kmeansModel = new KMeansModel(clusterCenters); + Vectors.dense(1.0, 4.0, 6.0)) + val kmeansModel = new KMeansModel(clusterCenters) - // act by exporting the model to the PMML format val modelExport = PMMLModelExportFactory.createPMMLModelExport(kmeansModel) // assert that the PMML format is as expected assert(modelExport.isInstanceOf[PMMLModelExport]) - val pmml = modelExport.asInstanceOf[PMMLModelExport].getPmml() - assert(pmml.getHeader().getDescription() === "k-means clustering") + val pmml = modelExport.asInstanceOf[PMMLModelExport].getPmml + 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 - assert(pmml.getModels().get(0).asInstanceOf[ClusteringModel].getNumberOfClusters() - === clusterCenters.size) - - // manual checking - // kmeansModel.toPMML("/tmp/kmeans.xml") - // kmeansModel.toPMML(System.out) - // System.out.println(kmeansModel.toPMML()) - - } - + 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) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala index ca5d8ca8b2f5b..af642702ed942 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala @@ -19,47 +19,34 @@ package org.apache.spark.mllib.pmml.export import org.dmg.pmml.RegressionModel import org.scalatest.FunSuite + import org.apache.spark.mllib.classification.LogisticRegressionModel import org.apache.spark.mllib.util.LinearDataGenerator class LogisticRegressionPMMLModelExportSuite extends FunSuite{ - test("LogisticRegressionPMMLModelExport generate PMML format") { - - // arrange models to test - val linearInput = LinearDataGenerator.generateLinearInput( - 3.0, Array(10.0, 10.0), 1, 17) - val logisticRegressionModel = new LogisticRegressionModel( - linearInput(0).features, linearInput(0).label); + test("LogisticRegressionPMMLModelExport generate PMML format") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val logisticRegressionModel = + new LogisticRegressionModel(linearInput(0).features, linearInput(0).label) - // act by exporting the model to the PMML format - val logisticModelExport = PMMLModelExportFactory - .createPMMLModelExport(logisticRegressionModel) + val logisticModelExport = PMMLModelExportFactory.createPMMLModelExport(logisticRegressionModel) + // assert that the PMML format is as expected assert(logisticModelExport.isInstanceOf[PMMLModelExport]) - var pmml = logisticModelExport.asInstanceOf[PMMLModelExport].getPmml() - assert(pmml.getHeader().getDescription() === "logistic regression") + val pmml = logisticModelExport.asInstanceOf[PMMLModelExport].getPmml + assert(pmml.getHeader.getDescription === "logistic regression") // check that the number of fields match the weights size - assert( - pmml.getDataDictionary().getNumberOfFields() === logisticRegressionModel.weights.size + 1) - // this verify that there is a model attached to the pmml object - // and the model is a regression one - // it also verifies that the pmml model has a regression table (for target category 1) - // with the same number of predictors of the model weights - assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(0).getTargetCategory() === "1") - assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(0).getNumericPredictors().size() - === logisticRegressionModel.weights.size) + assert(pmml.getDataDictionary.getNumberOfFields === logisticRegressionModel.weights.size + 1) + // This verify that there is a model attached to the pmml object and the model is a regression + // one. It also verifies that the pmml model has a regression table (for target category 1) + // with the same number of predictors of the model weights. + val pmmlRegressionModel = pmml.getModels.get(0).asInstanceOf[RegressionModel] + assert(pmmlRegressionModel.getRegressionTables.get(0).getTargetCategory === "1") + assert(pmmlRegressionModel.getRegressionTables.get(0).getNumericPredictors.size + === logisticRegressionModel.weights.size) // verify if there is a second table with target category 0 and no predictors - assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(1).getTargetCategory() === "0") - assert(pmml.getModels().get(0).asInstanceOf[RegressionModel] - .getRegressionTables().get(1).getNumericPredictors().size() === 0) - - // manual checking - // logisticRegressionModel.toPMML("/tmp/logisticregression.xml") - + assert(pmmlRegressionModel.getRegressionTables.get(1).getTargetCategory === "0") + assert(pmmlRegressionModel.getRegressionTables.get(1).getNumericPredictors.size === 0) } - } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala index a54cd247a120f..b466e08d09e6d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.pmml.export import org.scalatest.FunSuite + import org.apache.spark.mllib.classification.LogisticRegressionModel import org.apache.spark.mllib.classification.SVMModel import org.apache.spark.mllib.clustering.KMeansModel @@ -27,90 +28,62 @@ import org.apache.spark.mllib.regression.LinearRegressionModel import org.apache.spark.mllib.regression.RidgeRegressionModel import org.apache.spark.mllib.util.LinearDataGenerator -class PMMLModelExportFactorySuite extends FunSuite{ +class PMMLModelExportFactorySuite extends FunSuite { - test("PMMLModelExportFactory create KMeansPMMLModelExport when passing a KMeansModel") { - - // arrange + test("PMMLModelExportFactory create KMeansPMMLModelExport when passing a KMeansModel") { val clusterCenters = Array( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), - Vectors.dense(1.0, 4.0, 6.0) - ) - val kmeansModel = new KMeansModel(clusterCenters); - - // act + Vectors.dense(1.0, 4.0, 6.0)) + val kmeansModel = new KMeansModel(clusterCenters) + val modelExport = PMMLModelExportFactory.createPMMLModelExport(kmeansModel) - // assert assert(modelExport.isInstanceOf[KMeansPMMLModelExport]) - } test("PMMLModelExportFactory create GeneralizedLinearPMMLModelExport when passing a " + "LinearRegressionModel, RidgeRegressionModel, LassoModel or SVMModel") { - - // arrange - val linearInput = LinearDataGenerator.generateLinearInput( - 3.0, Array(10.0, 10.0), 1, 17) - val linearRegressionModel = new LinearRegressionModel( - linearInput(0).features, linearInput(0).label) - val ridgeRegressionModel = new RidgeRegressionModel( - linearInput(0).features, linearInput(0).label) - val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label) - val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) - - // act - val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) - // assert - assert(linearModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) - // act - val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) - // assert - assert(ridgeModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) - - // act - val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) - // assert - assert(lassoModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) - - // act - val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) - // assert - assert(svmModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) - + val linearRegressionModel = + new LinearRegressionModel(linearInput(0).features, linearInput(0).label) + val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) + assert(linearModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + + val ridgeRegressionModel = + new RidgeRegressionModel(linearInput(0).features, linearInput(0).label) + val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) + assert(ridgeModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + + + val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label) + val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) + assert(lassoModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + + val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) + val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) + assert(svmModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) } - - test("PMMLModelExportFactory create LogisticRegressionPMMLModelExport " - + "when passing a LogisticRegressionModel") { - - // arrange - val linearInput = LinearDataGenerator.generateLinearInput( - 3.0, Array(10.0, 10.0), 1, 17) - val logisticRegressionModel = new LogisticRegressionModel( - linearInput(0).features, linearInput(0).label); - // act - val logisticRegressionModelExport = PMMLModelExportFactory - .createPMMLModelExport(logisticRegressionModel) - // assert + test("PMMLModelExportFactory create LogisticRegressionPMMLModelExport " + + "when passing a LogisticRegressionModel") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val logisticRegressionModel = + new LogisticRegressionModel(linearInput(0).features, linearInput(0).label) + + val logisticRegressionModelExport = + PMMLModelExportFactory.createPMMLModelExport(logisticRegressionModel) + assert(logisticRegressionModelExport.isInstanceOf[LogisticRegressionPMMLModelExport]) - } test("PMMLModelExportFactory throw IllegalArgumentException " + "when passing an unsupported model") { + val invalidModel = new Object - // arrange - val invalidModel = new Object; - - // assert intercept[IllegalArgumentException] { - // act PMMLModelExportFactory.createPMMLModelExport(invalidModel) } - } - } From 3c22f798aaa5a4e17ca0f404bff856cfbad115b5 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 20 Apr 2015 23:53:37 -0700 Subject: [PATCH 33/41] more code style --- .../spark/mllib/pmml/PMMLExportable.scala | 16 +++-- .../GeneralizedLinearPMMLModelExport.scala | 62 +++++++++---------- .../LogisticRegressionPMMLModelExport.scala | 2 +- ...eneralizedLinearPMMLModelExportSuite.scala | 6 +- .../export/KMeansPMMLModelExportSuite.scala | 5 +- ...gisticRegressionPMMLModelExportSuite.scala | 6 +- .../export/PMMLModelExportFactorySuite.scala | 59 ++++++++---------- 7 files changed, 73 insertions(+), 83 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala index 938a7998cdf5f..354e90f3eeaa6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala @@ -17,9 +17,7 @@ package org.apache.spark.mllib.pmml -import java.io.File -import java.io.OutputStream -import java.io.StringWriter +import java.io.{File, OutputStream, StringWriter} import javax.xml.transform.stream.StreamResult import org.jpmml.model.JAXBUtil @@ -33,7 +31,7 @@ import org.apache.spark.mllib.pmml.export.PMMLModelExportFactory * developed by the Data Mining Group (www.dmg.org). */ trait PMMLExportable { - + /** * Export the model to the stream result in PMML format */ @@ -41,14 +39,14 @@ trait PMMLExportable { val pmmlModelExport = PMMLModelExportFactory.createPMMLModelExport(this) JAXBUtil.marshalPMML(pmmlModelExport.getPmml, streamResult) } - + /** * Export the model to a local file in PMML format */ def toPMML(localPath: String): Unit = { toPMML(new StreamResult(new File(localPath))) } - + /** * Export the model to a directory on a distributed file system in PMML format */ @@ -56,14 +54,14 @@ trait PMMLExportable { val pmml = toPMML() sc.parallelize(Array(pmml), 1).saveAsTextFile(path) } - + /** * Export the model to the OutputStream in PMML format */ def toPMML(outputStream: OutputStream): Unit = { toPMML(new StreamResult(outputStream)) } - + /** * Export the model to a String in PMML format */ @@ -72,5 +70,5 @@ trait PMMLExportable { toPMML(new StreamResult(writer)) writer.toString } - + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala index baab1a2dbf963..8c079d5aec42c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala @@ -27,9 +27,9 @@ import org.apache.spark.mllib.regression.GeneralizedLinearModel * PMML Model Export for GeneralizedLinearModel abstract class */ private[mllib] class GeneralizedLinearPMMLModelExport( - model : GeneralizedLinearModel, - description : String) - extends PMMLModelExport{ + model: GeneralizedLinearModel, + description: String) + extends PMMLModelExport { populateGeneralizedLinearPMML(model) @@ -37,37 +37,37 @@ private[mllib] class GeneralizedLinearPMMLModelExport( * Export the input GeneralizedLinearModel model to PMML format. */ private def populateGeneralizedLinearPMML(model: GeneralizedLinearModel): Unit = { - pmml.getHeader.setDescription(description) + pmml.getHeader.setDescription(description) - if(model.weights.size > 0){ - val fields = new SArray[FieldName](model.weights.size) - val dataDictionary = new DataDictionary - val miningSchema = new MiningSchema - val regressionTable = new RegressionTable(model.intercept) - val regressionModel = new RegressionModel(miningSchema,MiningFunctionType.REGRESSION) - .withModelName(description) - .withRegressionTables(regressionTable) + if (model.weights.size > 0) { + val fields = new SArray[FieldName](model.weights.size) + val dataDictionary = new DataDictionary + val miningSchema = new MiningSchema + val regressionTable = new RegressionTable(model.intercept) + val regressionModel = new RegressionModel(miningSchema, MiningFunctionType.REGRESSION) + .withModelName(description) + .withRegressionTables(regressionTable) - for (i <- 0 until model.weights.size) { - fields(i) = FieldName.create("field_" + i) - dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) - miningSchema - .withMiningFields(new MiningField(fields(i)) - .withUsageType(FieldUsageType.ACTIVE)) - regressionTable.withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) - } - - // for completeness add target field - val targetField = FieldName.create("target") - dataDictionary.withDataFields(new DataField(targetField, OpType.CONTINUOUS, DataType.DOUBLE)) - miningSchema - .withMiningFields(new MiningField(targetField) - .withUsageType(FieldUsageType.TARGET)) + for (i <- 0 until model.weights.size) { + fields(i) = FieldName.create("field_" + i) + dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema + .withMiningFields(new MiningField(fields(i)) + .withUsageType(FieldUsageType.ACTIVE)) + regressionTable.withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) + } - dataDictionary.withNumberOfFields(dataDictionary.getDataFields.size) + // for completeness add target field + val targetField = FieldName.create("target") + dataDictionary.withDataFields(new DataField(targetField, OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema + .withMiningFields(new MiningField(targetField) + .withUsageType(FieldUsageType.TARGET)) - pmml.setDataDictionary(dataDictionary) - pmml.withModels(regressionModel) - } + dataDictionary.withNumberOfFields(dataDictionary.getDataFields.size) + + pmml.setDataDictionary(dataDictionary) + pmml.withModels(regressionModel) + } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala index 75c28e1c03514..6e818c7709bda 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala @@ -29,7 +29,7 @@ import org.apache.spark.mllib.classification.LogisticRegressionModel private[mllib] class LogisticRegressionPMMLModelExport( model : LogisticRegressionModel, description : String) - extends PMMLModelExport{ + extends PMMLModelExport { populateLogisticRegressionPMML(model) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala index f48d39f889cd3..417ea12ccfa0b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala @@ -21,9 +21,7 @@ import org.dmg.pmml.RegressionModel import org.scalatest.FunSuite import org.apache.spark.mllib.classification.SVMModel -import org.apache.spark.mllib.regression.LassoModel -import org.apache.spark.mllib.regression.LinearRegressionModel -import org.apache.spark.mllib.regression.RidgeRegressionModel +import org.apache.spark.mllib.regression.{LassoModel, LinearRegressionModel, RidgeRegressionModel} import org.apache.spark.mllib.util.LinearDataGenerator class GeneralizedLinearPMMLModelExportSuite extends FunSuite { @@ -87,7 +85,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite { test("svm pmml export") { val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) - val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) + val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) // assert that the PMML format is as expected assert(svmModelExport.isInstanceOf[PMMLModelExport]) val pmml = svmModelExport.getPmml diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala index f34e2a210a9fd..d3c1dd85fa3b1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala @@ -26,15 +26,14 @@ import org.apache.spark.mllib.linalg.Vectors class KMeansPMMLModelExportSuite extends FunSuite { test("KMeansPMMLModelExport generate PMML format") { - // arrange model to test val clusterCenters = Array( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), Vectors.dense(1.0, 4.0, 6.0)) val kmeansModel = new KMeansModel(clusterCenters) - + val modelExport = PMMLModelExportFactory.createPMMLModelExport(kmeansModel) - + // assert that the PMML format is as expected assert(modelExport.isInstanceOf[PMMLModelExport]) val pmml = modelExport.asInstanceOf[PMMLModelExport].getPmml diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala index af642702ed942..696f95ed873bb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala @@ -23,13 +23,13 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.classification.LogisticRegressionModel import org.apache.spark.mllib.util.LinearDataGenerator -class LogisticRegressionPMMLModelExportSuite extends FunSuite{ +class LogisticRegressionPMMLModelExportSuite extends FunSuite { test("LogisticRegressionPMMLModelExport generate PMML format") { val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) val logisticRegressionModel = new LogisticRegressionModel(linearInput(0).features, linearInput(0).label) - + val logisticModelExport = PMMLModelExportFactory.createPMMLModelExport(logisticRegressionModel) // assert that the PMML format is as expected @@ -48,5 +48,5 @@ class LogisticRegressionPMMLModelExportSuite extends FunSuite{ // verify if there is a second table with target category 0 and no predictors assert(pmmlRegressionModel.getRegressionTables.get(1).getTargetCategory === "0") assert(pmmlRegressionModel.getRegressionTables.get(1).getNumericPredictors.size === 0) - } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala index b466e08d09e6d..a94854e4c0f20 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala @@ -19,13 +19,10 @@ package org.apache.spark.mllib.pmml.export import org.scalatest.FunSuite -import org.apache.spark.mllib.classification.LogisticRegressionModel -import org.apache.spark.mllib.classification.SVMModel +import org.apache.spark.mllib.classification.{LogisticRegressionModel, SVMModel} import org.apache.spark.mllib.clustering.KMeansModel import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LassoModel -import org.apache.spark.mllib.regression.LinearRegressionModel -import org.apache.spark.mllib.regression.RidgeRegressionModel +import org.apache.spark.mllib.regression.{LassoModel, LinearRegressionModel, RidgeRegressionModel} import org.apache.spark.mllib.util.LinearDataGenerator class PMMLModelExportFactorySuite extends FunSuite { @@ -38,33 +35,32 @@ class PMMLModelExportFactorySuite extends FunSuite { val kmeansModel = new KMeansModel(clusterCenters) val modelExport = PMMLModelExportFactory.createPMMLModelExport(kmeansModel) - + assert(modelExport.isInstanceOf[KMeansPMMLModelExport]) - } - - test("PMMLModelExportFactory create GeneralizedLinearPMMLModelExport when passing a " - + "LinearRegressionModel, RidgeRegressionModel, LassoModel or SVMModel") { - val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + } - val linearRegressionModel = - new LinearRegressionModel(linearInput(0).features, linearInput(0).label) - val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) - assert(linearModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + test("PMMLModelExportFactory create GeneralizedLinearPMMLModelExport when passing a " + + "LinearRegressionModel, RidgeRegressionModel, LassoModel or SVMModel") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) - val ridgeRegressionModel = - new RidgeRegressionModel(linearInput(0).features, linearInput(0).label) - val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) - assert(ridgeModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + val linearRegressionModel = + new LinearRegressionModel(linearInput(0).features, linearInput(0).label) + val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) + assert(linearModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + val ridgeRegressionModel = + new RidgeRegressionModel(linearInput(0).features, linearInput(0).label) + val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) + assert(ridgeModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) - val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label) - val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) - assert(lassoModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label) + val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) + assert(lassoModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) - val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) - val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) - assert(svmModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) - } + val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) + val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) + assert(svmModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + } test("PMMLModelExportFactory create LogisticRegressionPMMLModelExport " + "when passing a LogisticRegressionModel") { @@ -76,14 +72,13 @@ class PMMLModelExportFactorySuite extends FunSuite { PMMLModelExportFactory.createPMMLModelExport(logisticRegressionModel) assert(logisticRegressionModelExport.isInstanceOf[LogisticRegressionPMMLModelExport]) - } - - test("PMMLModelExportFactory throw IllegalArgumentException " - + "when passing an unsupported model") { + } + + test("PMMLModelExportFactory throw IllegalArgumentException when passing an unsupported model") { val invalidModel = new Object - + intercept[IllegalArgumentException] { PMMLModelExportFactory.createPMMLModelExport(invalidModel) } - } + } } From 66b7c12bdf5a0b28dc7bcfbd2457641df818959f Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Tue, 21 Apr 2015 23:24:18 +0100 Subject: [PATCH 34/41] [SPARK-1406] Updated pmml model lib to 1.1.15, latest Java 6 compatible --- mllib/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index 640a7aea8fa37..fd5106adbfc51 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -112,7 +112,7 @@ org.jpmml pmml-model - 1.1.7 + 1.1.15 From dea98ca416a8d510c9dc8613df1f3b88840a8eb8 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Tue, 21 Apr 2015 23:49:59 +0100 Subject: [PATCH 35/41] [SPARK-1406] Exclude transitive dependency for pmml model --- mllib/pom.xml | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index fd5106adbfc51..a3c57ae26000b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -110,9 +110,19 @@ test - org.jpmml - pmml-model - 1.1.15 + org.jpmml + pmml-model + 1.1.15 + + + com.sun.xml.fastinfoset + FastInfoset + + + com.sun.istack + istack-commons-runtime + + From 25dce336065e39da8492c6d8379a21ab3733cd45 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 22 Apr 2015 00:48:23 +0100 Subject: [PATCH 36/41] [SPARK-1406] Update code to latest pmml model --- .../pmml/export/GeneralizedLinearPMMLModelExport.scala | 4 +++- .../spark/mllib/pmml/export/KMeansPMMLModelExport.scala | 9 ++++++--- .../pmml/export/LogisticRegressionPMMLModelExport.scala | 4 +++- 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala index 8c079d5aec42c..1874786af0002 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala @@ -44,7 +44,9 @@ private[mllib] class GeneralizedLinearPMMLModelExport( val dataDictionary = new DataDictionary val miningSchema = new MiningSchema val regressionTable = new RegressionTable(model.intercept) - val regressionModel = new RegressionModel(miningSchema, MiningFunctionType.REGRESSION) + val regressionModel = new RegressionModel() + .withFunctionName(MiningFunctionType.REGRESSION) + .withMiningSchema(miningSchema) .withModelName(description) .withRegressionTables(regressionTable) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala index c12b275b2185c..069e7afc9fca0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala @@ -44,10 +44,13 @@ private[mllib] class KMeansPMMLModelExport(model : KMeansModel) extends PMMLMode val comparisonMeasure = new ComparisonMeasure() .withKind(ComparisonMeasure.Kind.DISTANCE) .withMeasure(new SquaredEuclidean()) - val clusteringModel = new ClusteringModel(miningSchema, comparisonMeasure, - MiningFunctionType.CLUSTERING, ClusteringModel.ModelClass.CENTER_BASED, - model.clusterCenters.length) + val clusteringModel = new ClusteringModel() .withModelName("k-means") + .withMiningSchema(miningSchema) + .withComparisonMeasure(comparisonMeasure) + .withFunctionName(MiningFunctionType.CLUSTERING) + .withModelClass(ClusteringModel.ModelClass.CENTER_BASED) + .withNumberOfClusters(model.clusterCenters.length) for (i <- 0 until clusterCenter.size) { fields(i) = FieldName.create("field_" + i) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala index 6e818c7709bda..2bf4fa858b09b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala @@ -45,7 +45,9 @@ private[mllib] class LogisticRegressionPMMLModelExport( val miningSchema = new MiningSchema val regressionTableYES = new RegressionTable(model.intercept).withTargetCategory("1") val regressionTableNO = new RegressionTable(0.0).withTargetCategory("0") - val regressionModel = new RegressionModel(miningSchema, MiningFunctionType.CLASSIFICATION) + val regressionModel = new RegressionModel() + .withFunctionName(MiningFunctionType.CLASSIFICATION) + .withMiningSchema(miningSchema) .withModelName(description) .withNormalizationMethod(RegressionNormalizationMethodType.LOGIT) .withRegressionTables(regressionTableYES, regressionTableNO) From cfcb5960cee46c2a84b69c3f3a766d78e38f6c42 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Sat, 25 Apr 2015 08:40:51 +0100 Subject: [PATCH 37/41] [SPARK-1406] Throw IllegalArgumentException when exporting a multinomial logistic regression --- .../mllib/pmml/export/PMMLModelExportFactory.scala | 6 +++++- .../pmml/export/PMMLModelExportFactorySuite.scala | 12 ++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala index 0c374a46fb562..bd8c8f96a6e55 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala @@ -44,7 +44,11 @@ private[mllib] object PMMLModelExportFactory { new GeneralizedLinearPMMLModelExport(svm, "linear SVM: if predicted value > 0, the outcome is positive, or negative otherwise") case logistic: LogisticRegressionModel => - new LogisticRegressionPMMLModelExport(logistic, "logistic regression") + if(logistic.numClasses == 2) + new LogisticRegressionPMMLModelExport(logistic, "logistic regression") + else + throw new IllegalArgumentException( + "PMML Export not supported for Multinomial Logistic Regression") case _ => throw new IllegalArgumentException( "PMML Export not supported for model: " + model.getClass.getName) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala index a94854e4c0f20..b87e96e7032f3 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala @@ -73,6 +73,18 @@ class PMMLModelExportFactorySuite extends FunSuite { assert(logisticRegressionModelExport.isInstanceOf[LogisticRegressionPMMLModelExport]) } + + test("PMMLModelExportFactory throw IllegalArgumentException " + + "when passing a Multinomial Logistic Regression") { + /** 3 classes, 2 features */ + val multiclassLogisticRegressionModel = new LogisticRegressionModel( + weights = Vectors.dense(0.1, 0.2, 0.3, 0.4), intercept = 1.0, + numFeatures = 2, numClasses = 3) + + intercept[IllegalArgumentException] { + PMMLModelExportFactory.createPMMLModelExport(multiclassLogisticRegressionModel) + } + } test("PMMLModelExportFactory throw IllegalArgumentException when passing an unsupported model") { val invalidModel = new Object From 7a5e0ec20176f43477e1d2ee21fb4432937ffc30 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Tue, 28 Apr 2015 06:53:59 +0100 Subject: [PATCH 38/41] [SPARK-1406] Binary classification for SVM and Logistic Regression --- ...BinaryClassificationPMMLModelExport.scala} | 31 ++++++++++------ .../pmml/export/PMMLModelExportFactory.scala | 13 ++++--- ...yClassificationPMMLModelExportSuite.scala} | 36 +++++++++++++++++-- ...eneralizedLinearPMMLModelExportSuite.scala | 25 ++----------- .../export/KMeansPMMLModelExportSuite.scala | 1 + .../export/PMMLModelExportFactorySuite.scala | 19 +++++----- 6 files changed, 77 insertions(+), 48 deletions(-) rename mllib/src/main/scala/org/apache/spark/mllib/pmml/export/{LogisticRegressionPMMLModelExport.scala => BinaryClassificationPMMLModelExport.scala} (71%) rename mllib/src/test/scala/org/apache/spark/mllib/pmml/export/{LogisticRegressionPMMLModelExportSuite.scala => BinaryClassificationPMMLModelExportSuite.scala} (56%) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala similarity index 71% rename from mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala rename to mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala index 2bf4fa858b09b..58f4b52f1b497 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala @@ -21,22 +21,24 @@ import scala.{Array => SArray} import org.dmg.pmml._ -import org.apache.spark.mllib.classification.LogisticRegressionModel +import org.apache.spark.mllib.regression.GeneralizedLinearModel /** - * PMML Model Export for LogisticRegressionModel class + * PMML Model Export for GeneralizedLinearModel class with binary ClassificationModel */ -private[mllib] class LogisticRegressionPMMLModelExport( - model : LogisticRegressionModel, - description : String) +private[mllib] class BinaryClassificationPMMLModelExport( + model : GeneralizedLinearModel, + description : String, + normalizationMethod : RegressionNormalizationMethodType, + threshold: Double) extends PMMLModelExport { - populateLogisticRegressionPMML(model) + populateBinaryClassificationPMML() /** - * Export the input LogisticRegressionModel model to PMML format + * Export the input LogisticRegressionModel or SVMModel to PMML format. */ - private def populateLogisticRegressionPMML(model : LogisticRegressionModel): Unit = { + private def populateBinaryClassificationPMML(): Unit = { pmml.getHeader.setDescription(description) if (model.weights.size > 0) { @@ -44,12 +46,21 @@ private[mllib] class LogisticRegressionPMMLModelExport( val dataDictionary = new DataDictionary val miningSchema = new MiningSchema val regressionTableYES = new RegressionTable(model.intercept).withTargetCategory("1") - val regressionTableNO = new RegressionTable(0.0).withTargetCategory("0") + var interceptNO = threshold + if (RegressionNormalizationMethodType.LOGIT == normalizationMethod) { + if (threshold <= 0) + interceptNO = 1000 + else if (threshold >= 1) + interceptNO = -1000 + else + interceptNO = -math.log(1/threshold -1) + } + val regressionTableNO = new RegressionTable(interceptNO).withTargetCategory("0") val regressionModel = new RegressionModel() .withFunctionName(MiningFunctionType.CLASSIFICATION) .withMiningSchema(miningSchema) .withModelName(description) - .withNormalizationMethod(RegressionNormalizationMethodType.LOGIT) + .withNormalizationMethod(normalizationMethod) .withRegressionTables(regressionTableYES, regressionTableNO) for (i <- 0 until model.weights.size) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala index bd8c8f96a6e55..965e2785c3acc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala @@ -17,6 +17,8 @@ package org.apache.spark.mllib.pmml.export +import org.dmg.pmml.RegressionNormalizationMethodType + import org.apache.spark.mllib.classification.LogisticRegressionModel import org.apache.spark.mllib.classification.SVMModel import org.apache.spark.mllib.clustering.KMeansModel @@ -41,11 +43,14 @@ private[mllib] object PMMLModelExportFactory { case lasso: LassoModel => new GeneralizedLinearPMMLModelExport(lasso, "lasso regression") case svm: SVMModel => - new GeneralizedLinearPMMLModelExport(svm, - "linear SVM: if predicted value > 0, the outcome is positive, or negative otherwise") + new BinaryClassificationPMMLModelExport( + svm, "linear SVM", RegressionNormalizationMethodType.NONE, + svm.getThreshold.getOrElse(0.0)) case logistic: LogisticRegressionModel => - if(logistic.numClasses == 2) - new LogisticRegressionPMMLModelExport(logistic, "logistic regression") + if (logistic.numClasses == 2) + new BinaryClassificationPMMLModelExport( + logistic, "logistic regression", RegressionNormalizationMethodType.LOGIT, + logistic.getThreshold.getOrElse(0.5)) else throw new IllegalArgumentException( "PMML Export not supported for Multinomial Logistic Regression") diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala similarity index 56% rename from mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala index 696f95ed873bb..0b646cf1ce6c4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/LogisticRegressionPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala @@ -18,14 +18,16 @@ package org.apache.spark.mllib.pmml.export import org.dmg.pmml.RegressionModel +import org.dmg.pmml.RegressionNormalizationMethodType import org.scalatest.FunSuite import org.apache.spark.mllib.classification.LogisticRegressionModel +import org.apache.spark.mllib.classification.SVMModel import org.apache.spark.mllib.util.LinearDataGenerator -class LogisticRegressionPMMLModelExportSuite extends FunSuite { +class BinaryClassificationPMMLModelExportSuite extends FunSuite { - test("LogisticRegressionPMMLModelExport generate PMML format") { + test("logistic regression PMML export") { val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) val logisticRegressionModel = new LogisticRegressionModel(linearInput(0).features, linearInput(0).label) @@ -48,5 +50,35 @@ class LogisticRegressionPMMLModelExportSuite extends FunSuite { // verify if there is a second table with target category 0 and no predictors assert(pmmlRegressionModel.getRegressionTables.get(1).getTargetCategory === "0") assert(pmmlRegressionModel.getRegressionTables.get(1).getNumericPredictors.size === 0) + // ensure logistic regression has normalization method set to LOGIT + assert(pmmlRegressionModel.getNormalizationMethod() == RegressionNormalizationMethodType.LOGIT) } + + test("linear SVM PMML export") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) + + val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) + + // assert that the PMML format is as expected + assert(svmModelExport.isInstanceOf[PMMLModelExport]) + val pmml = svmModelExport.getPmml + assert(pmml.getHeader.getDescription + === "linear SVM") + // check that the number of fields match the weights size + assert(pmml.getDataDictionary.getNumberOfFields === svmModel.weights.size + 1) + // This verify that there is a model attached to the pmml object and the model is a regression + // one. It also verifies that the pmml model has a regression table (for target category 1) + // with the same number of predictors of the model weights. + val pmmlRegressionModel = pmml.getModels.get(0).asInstanceOf[RegressionModel] + assert(pmmlRegressionModel.getRegressionTables.get(0).getTargetCategory === "1") + assert(pmmlRegressionModel.getRegressionTables.get(0).getNumericPredictors.size + === svmModel.weights.size) + // verify if there is a second table with target category 0 and no predictors + assert(pmmlRegressionModel.getRegressionTables.get(1).getTargetCategory === "0") + assert(pmmlRegressionModel.getRegressionTables.get(1).getNumericPredictors.size === 0) + // ensure linear SVM has normalization method set to NONE + assert(pmmlRegressionModel.getNormalizationMethod() == RegressionNormalizationMethodType.NONE) + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala index 417ea12ccfa0b..f9afbd888dfc5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala @@ -20,13 +20,12 @@ package org.apache.spark.mllib.pmml.export import org.dmg.pmml.RegressionModel import org.scalatest.FunSuite -import org.apache.spark.mllib.classification.SVMModel import org.apache.spark.mllib.regression.{LassoModel, LinearRegressionModel, RidgeRegressionModel} import org.apache.spark.mllib.util.LinearDataGenerator class GeneralizedLinearPMMLModelExportSuite extends FunSuite { - test("linear regression pmml export") { + test("linear regression PMML export") { val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) val linearRegressionModel = new LinearRegressionModel(linearInput(0).features, linearInput(0).label) @@ -45,7 +44,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite { === linearRegressionModel.weights.size) } - test("ridge regression pmml export") { + test("ridge regression PMML export") { val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) val ridgeRegressionModel = new RidgeRegressionModel(linearInput(0).features, linearInput(0).label) @@ -64,7 +63,7 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite { === ridgeRegressionModel.weights.size) } - test("lasso pmml export") { + test("lasso PMML export") { val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label) val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) @@ -82,22 +81,4 @@ class GeneralizedLinearPMMLModelExportSuite extends FunSuite { === lassoModel.weights.size) } - test("svm pmml export") { - val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) - val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) - val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) - // assert that the PMML format is as expected - assert(svmModelExport.isInstanceOf[PMMLModelExport]) - val pmml = svmModelExport.getPmml - assert(pmml.getHeader.getDescription - === "linear SVM: if predicted value > 0, the outcome is positive, or negative otherwise") - // check that the number of fields match the weights size - assert(pmml.getDataDictionary.getNumberOfFields === svmModel.weights.size + 1) - // This verify that there is a model attached to the pmml object and the model is a regression - // one. It also verifies that the pmml model has a regression table with the same number of - // predictors of the model weights. - val pmmlRegressionModel = pmml.getModels.get(0).asInstanceOf[RegressionModel] - assert(pmmlRegressionModel.getRegressionTables.get(0).getNumericPredictors.size - === svmModel.weights.size) - } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala index d3c1dd85fa3b1..b985d0446d7b0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala @@ -45,4 +45,5 @@ class KMeansPMMLModelExportSuite extends FunSuite { val pmmlClusteringModel = pmml.getModels.get(0).asInstanceOf[ClusteringModel] assert(pmmlClusteringModel.getNumberOfClusters === clusterCenters.length) } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala index b87e96e7032f3..f28a4ac8ad01f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala @@ -40,7 +40,7 @@ class PMMLModelExportFactorySuite extends FunSuite { } test("PMMLModelExportFactory create GeneralizedLinearPMMLModelExport when passing a " - + "LinearRegressionModel, RidgeRegressionModel, LassoModel or SVMModel") { + + "LinearRegressionModel, RidgeRegressionModel or LassoModel") { val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) val linearRegressionModel = @@ -56,22 +56,21 @@ class PMMLModelExportFactorySuite extends FunSuite { val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label) val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) assert(lassoModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) - - val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) - val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) - assert(svmModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) } - test("PMMLModelExportFactory create LogisticRegressionPMMLModelExport " - + "when passing a LogisticRegressionModel") { + test("PMMLModelExportFactory create BinaryClassificationPMMLModelExport " + + "when passing a LogisticRegressionModel or SVMModel") { val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val logisticRegressionModel = new LogisticRegressionModel(linearInput(0).features, linearInput(0).label) - val logisticRegressionModelExport = PMMLModelExportFactory.createPMMLModelExport(logisticRegressionModel) - - assert(logisticRegressionModelExport.isInstanceOf[LogisticRegressionPMMLModelExport]) + assert(logisticRegressionModelExport.isInstanceOf[BinaryClassificationPMMLModelExport]) + + val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) + val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) + assert(svmModelExport.isInstanceOf[BinaryClassificationPMMLModelExport]) } test("PMMLModelExportFactory throw IllegalArgumentException " From 30165c4caaea77b5b91ce2bef0ca94cb0454f527 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Tue, 28 Apr 2015 07:43:57 +0100 Subject: [PATCH 39/41] [SPARK-1406] Fixed extreme cases for logit --- .../pmml/export/BinaryClassificationPMMLModelExport.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala index 58f4b52f1b497..e469a7ac9bc85 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala @@ -49,9 +49,9 @@ private[mllib] class BinaryClassificationPMMLModelExport( var interceptNO = threshold if (RegressionNormalizationMethodType.LOGIT == normalizationMethod) { if (threshold <= 0) - interceptNO = 1000 - else if (threshold >= 1) interceptNO = -1000 + else if (threshold >= 1) + interceptNO = 1000 else interceptNO = -math.log(1/threshold -1) } From 085cf42a01da4b6f8b3881a79d75f39c8353e690 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 29 Apr 2015 07:57:17 +0100 Subject: [PATCH 40/41] [SPARK-1406] Added Double Min and Max Fixed scala style --- .../BinaryClassificationPMMLModelExport.scala | 13 +++++++------ .../mllib/pmml/export/PMMLModelExportFactory.scala | 5 +++-- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala index e469a7ac9bc85..34b447584e521 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala @@ -48,12 +48,13 @@ private[mllib] class BinaryClassificationPMMLModelExport( val regressionTableYES = new RegressionTable(model.intercept).withTargetCategory("1") var interceptNO = threshold if (RegressionNormalizationMethodType.LOGIT == normalizationMethod) { - if (threshold <= 0) - interceptNO = -1000 - else if (threshold >= 1) - interceptNO = 1000 - else - interceptNO = -math.log(1/threshold -1) + if (threshold <= 0) { + interceptNO = Double.MinValue + } else if (threshold >= 1) { + interceptNO = Double.MaxValue + } else { + interceptNO = -math.log(1 / threshold - 1) + } } val regressionTableNO = new RegressionTable(interceptNO).withTargetCategory("0") val regressionModel = new RegressionModel() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala index 965e2785c3acc..c16e83d6a067d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala @@ -47,13 +47,14 @@ private[mllib] object PMMLModelExportFactory { svm, "linear SVM", RegressionNormalizationMethodType.NONE, svm.getThreshold.getOrElse(0.0)) case logistic: LogisticRegressionModel => - if (logistic.numClasses == 2) + if (logistic.numClasses == 2) { new BinaryClassificationPMMLModelExport( logistic, "logistic regression", RegressionNormalizationMethodType.LOGIT, logistic.getThreshold.getOrElse(0.5)) - else + } else { throw new IllegalArgumentException( "PMML Export not supported for Multinomial Logistic Regression") + } case _ => throw new IllegalArgumentException( "PMML Export not supported for model: " + model.getClass.getName) From 852aac6149d4d0c932ff34e959d148419b369983 Mon Sep 17 00:00:00 2001 From: Vincenzo Selvaggio Date: Wed, 29 Apr 2015 08:05:51 +0100 Subject: [PATCH 41/41] [SPARK-1406] Update JPMML version to 1.1.15 in LICENSE file --- LICENSE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/LICENSE b/LICENSE index defdb17029a5f..21c42e9a20fa3 100644 --- a/LICENSE +++ b/LICENSE @@ -814,7 +814,7 @@ BSD-style licenses The following components are provided under a BSD-style license. See project link for details. (BSD 3 Clause) core (com.github.fommil.netlib:core:1.1.2 - https://github.com/fommil/netlib-java/core) - (BSD 3 Clause) JPMML-Model (org.jpmml:pmml-model:1.1.7 - https://github.com/jpmml/jpmml-model) + (BSD 3 Clause) JPMML-Model (org.jpmml:pmml-model:1.1.15 - https://github.com/jpmml/jpmml-model) (BSD 3-clause style license) jblas (org.jblas:jblas:1.2.3 - http://jblas.org/) (BSD License) AntLR Parser Generator (antlr:antlr:2.7.7 - http://www.antlr.org/) (BSD License) Javolution (javolution:javolution:5.5.1 - http://javolution.org)