From 496bc7ba688260317fe5907a8057347a5b184490 Mon Sep 17 00:00:00 2001 From: Sandor Murakozi Date: Thu, 18 Jan 2018 20:03:33 +0100 Subject: [PATCH 1/4] Converted all clustering tests to check streaming --- .../ml/clustering/BisectingKMeansSuite.scala | 39 +++++++++---------- .../apache/spark/ml/clustering/Encoders.scala | 25 ++++++++++++ .../ml/clustering/GaussianMixtureSuite.scala | 21 ++++------ .../spark/ml/clustering/KMeansSuite.scala | 33 ++++++++-------- .../apache/spark/ml/clustering/LDASuite.scala | 24 +++++------- 5 files changed, 78 insertions(+), 64 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/ml/clustering/Encoders.scala diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala index f3ff2afcad2c..18a3b9cc4e5b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -19,17 +19,18 @@ package org.apache.spark.ml.clustering import scala.language.existentials -import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.SparkException import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.clustering.DistanceMeasure -import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.Dataset -class BisectingKMeansSuite - extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + +class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { + + import Encoders._ final val k = 5 @transient var dataset: Dataset[_] = _ @@ -68,10 +69,12 @@ class BisectingKMeansSuite // Verify fit does not fail on very sparse data val model = bkm.fit(sparseDataset) - val result = model.transform(sparseDataset) - val numClusters = result.select("prediction").distinct().collect().length - // Verify we hit the edge case - assert(numClusters < k && numClusters > 1) + + testTransformerByGlobalCheckFunc[Vector](sparseDataset.toDF(), model, "prediction") { rows => + val numClusters = rows.distinct.length + // Verify we hit the edge case + assert(numClusters < k && numClusters > 1) + } } test("setter/getter") { @@ -105,17 +108,13 @@ class BisectingKMeansSuite val model = bkm.fit(dataset) assert(model.clusterCenters.length === k) - val transformed = model.transform(dataset) - val expectedColumns = Array("features", predictionColName) - expectedColumns.foreach { column => - assert(transformed.columns.contains(column)) + testTransformerByGlobalCheckFunc[Vector](dataset.toDF(), model, + "features", predictionColName) { rows => + val clusters = rows.map(_.getAs[Int](predictionColName)).toSet + assert(clusters === Set(0, 1, 2, 3, 4)) + assert(model.computeCost(dataset) < 0.1) + assert(model.hasParent) } - val clusters = - transformed.select(predictionColName).rdd.map(_.getInt(0)).distinct().collect().toSet - assert(clusters.size === k) - assert(clusters === Set(0, 1, 2, 3, 4)) - assert(model.computeCost(dataset) < 0.1) - assert(model.hasParent) // Check validity of model summary val numRows = dataset.count() diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/Encoders.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/Encoders.scala new file mode 100644 index 000000000000..afa72171638e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/Encoders.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.ml.clustering + +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder + +private[clustering] object Encoders { + implicit val vectorEncoder = ExpressionEncoder[Vector]() +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala index d0d461a42711..d692882f2d38 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala @@ -23,15 +23,15 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.ml.linalg.{DenseMatrix, Matrices, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.stat.distribution.MultivariateGaussian -import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Dataset, Row} -class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext - with DefaultReadWriteTest { + +class GaussianMixtureSuite extends MLTest with DefaultReadWriteTest { import testImplicits._ + import Encoders._ import GaussianMixtureSuite._ final val k = 5 @@ -119,15 +119,10 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext assert(model.weights.length === k) assert(model.gaussians.length === k) - val transformed = model.transform(dataset) - val expectedColumns = Array("features", predictionColName, probabilityColName) - expectedColumns.foreach { column => - assert(transformed.columns.contains(column)) - } - // Check prediction matches the highest probability, and probabilities sum to one. - transformed.select(predictionColName, probabilityColName).collect().foreach { - case Row(pred: Int, prob: Vector) => + testTransformer[Vector](dataset.toDF(), model, + "features", predictionColName, probabilityColName) { + case Row(_, pred: Int, prob: Vector) => val probArray = prob.toArray val predFromProb = probArray.zipWithIndex.maxBy(_._1)._2 assert(pred === predFromProb) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index 680a7c203408..d14c9fcb1a16 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -22,20 +22,23 @@ import scala.util.Random import org.dmg.pmml.{ClusteringModel, PMML} -import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.SparkException import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils, PMMLReadWriteTest} import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.mllib.clustering.{DistanceMeasure, KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} +import org.apache.spark.mllib.clustering.{DistanceMeasure, KMeans => MLlibKMeans, + KMeansModel => MLlibKMeansModel} import org.apache.spark.mllib.linalg.{Vectors => MLlibVectors} -import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} + private[clustering] case class TestRow(features: Vector) -class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest - with PMMLReadWriteTest { + +class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTest { + + import testImplicits._ final val k = 5 @transient var dataset: Dataset[_] = _ @@ -109,15 +112,13 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultR val model = kmeans.fit(dataset) assert(model.clusterCenters.length === k) - val transformed = model.transform(dataset) - val expectedColumns = Array("features", predictionColName) - expectedColumns.foreach { column => - assert(transformed.columns.contains(column)) + testTransformerByGlobalCheckFunc[Vector](dataset.toDF(), model, + "features", predictionColName) { rows => + val clusters = rows.map(_.getAs[Int](predictionColName)).toSet + assert(clusters.size === k) + assert(clusters === Set(0, 1, 2, 3, 4)) } - val clusters = - transformed.select(predictionColName).rdd.map(_.getInt(0)).distinct().collect().toSet - assert(clusters.size === k) - assert(clusters === Set(0, 1, 2, 3, 4)) + assert(model.computeCost(dataset) < 0.1) assert(model.hasParent) @@ -149,9 +150,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultR model.setFeaturesCol(featuresColName).setPredictionCol(predictionColName) val transformed = model.transform(dataset.withColumnRenamed("features", featuresColName)) - Seq(featuresColName, predictionColName).foreach { column => - assert(transformed.columns.contains(column)) - } + assert(transformed.schema.fieldNames.toSet === Set(featuresColName, predictionColName)) assert(model.getFeaturesCol == featuresColName) assert(model.getPredictionCol == predictionColName) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala index 4d848205034c..9543a4992c4c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala @@ -21,12 +21,11 @@ import scala.language.existentials import org.apache.hadoop.fs.Path -import org.apache.spark.SparkFunSuite import org.apache.spark.ml.linalg.{Vector, Vectors} -import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder object LDASuite { def generateLDAData( @@ -61,10 +60,12 @@ object LDASuite { } -class LDASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { +class LDASuite extends MLTest with DefaultReadWriteTest { import testImplicits._ + implicit val vectorEncoder = ExpressionEncoder[Vector]() + val k: Int = 5 val vocabSize: Int = 30 @transient var dataset: Dataset[_] = _ @@ -186,16 +187,11 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRead assert(model.topicsMatrix.numCols === k) assert(!model.isDistributed) - // transform() - val transformed = model.transform(dataset) - val expectedColumns = Array("features", lda.getTopicDistributionCol) - expectedColumns.foreach { column => - assert(transformed.columns.contains(column)) - } - transformed.select(lda.getTopicDistributionCol).collect().foreach { r => - val topicDistribution = r.getAs[Vector](0) - assert(topicDistribution.size === k) - assert(topicDistribution.toArray.forall(w => w >= 0.0 && w <= 1.0)) + testTransformer[Vector](dataset.toDF(), model, + "features", lda.getTopicDistributionCol) { + case Row(_, topicDistribution: Vector) => + assert(topicDistribution.size === k) + assert(topicDistribution.toArray.forall(w => w >= 0.0 && w <= 1.0)) } // logLikelihood, logPerplexity From 58239ba3edc0c60900d842caf3d38e30507d885f Mon Sep 17 00:00:00 2001 From: Sandor Murakozi Date: Thu, 18 Jan 2018 21:12:46 +0100 Subject: [PATCH 2/4] formatting, nits --- .../apache/spark/ml/clustering/BisectingKMeansSuite.scala | 1 + .../test/scala/org/apache/spark/ml/clustering/LDASuite.scala | 5 ++--- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala index 18a3b9cc4e5b..e0ac026de175 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -111,6 +111,7 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { testTransformerByGlobalCheckFunc[Vector](dataset.toDF(), model, "features", predictionColName) { rows => val clusters = rows.map(_.getAs[Int](predictionColName)).toSet + assert(clusters.size === k) assert(clusters === Set(0, 1, 2, 3, 4)) assert(model.computeCost(dataset) < 0.1) assert(model.hasParent) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala index 9543a4992c4c..8c13cbe7c0e7 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder + object LDASuite { def generateLDAData( @@ -63,8 +63,7 @@ object LDASuite { class LDASuite extends MLTest with DefaultReadWriteTest { import testImplicits._ - - implicit val vectorEncoder = ExpressionEncoder[Vector]() + import Encoders._ val k: Int = 5 val vocabSize: Int = 30 From eb66659e8b65de9d539584b935899838e1dd9ed8 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 17 May 2018 14:26:17 -0700 Subject: [PATCH 3/4] cleanups --- .../ml/clustering/BisectingKMeansSuite.scala | 17 +++++++------ .../apache/spark/ml/clustering/Encoders.scala | 25 ------------------- .../ml/clustering/GaussianMixtureSuite.scala | 5 ++-- .../spark/ml/clustering/KMeansSuite.scala | 2 +- .../apache/spark/ml/clustering/LDASuite.scala | 4 +-- 5 files changed, 14 insertions(+), 39 deletions(-) delete mode 100644 mllib/src/test/scala/org/apache/spark/ml/clustering/Encoders.scala diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala index e0ac026de175..81842afbddbb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.Dataset class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { - import Encoders._ + import testImplicits._ final val k = 5 @transient var dataset: Dataset[_] = _ @@ -70,10 +70,11 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { // Verify fit does not fail on very sparse data val model = bkm.fit(sparseDataset) - testTransformerByGlobalCheckFunc[Vector](sparseDataset.toDF(), model, "prediction") { rows => - val numClusters = rows.distinct.length - // Verify we hit the edge case - assert(numClusters < k && numClusters > 1) + testTransformerByGlobalCheckFunc[Tuple1[Vector]](sparseDataset.toDF(), model, "prediction") { + rows => + val numClusters = rows.distinct.length + // Verify we hit the edge case + assert(numClusters < k && numClusters > 1) } } @@ -107,14 +108,14 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { val bkm = new BisectingKMeans().setK(k).setPredictionCol(predictionColName).setSeed(1) val model = bkm.fit(dataset) assert(model.clusterCenters.length === k) + assert(model.computeCost(dataset) < 0.1) + assert(model.hasParent) - testTransformerByGlobalCheckFunc[Vector](dataset.toDF(), model, + testTransformerByGlobalCheckFunc[Tuple1[Vector]](dataset.toDF(), model, "features", predictionColName) { rows => val clusters = rows.map(_.getAs[Int](predictionColName)).toSet assert(clusters.size === k) assert(clusters === Set(0, 1, 2, 3, 4)) - assert(model.computeCost(dataset) < 0.1) - assert(model.hasParent) } // Check validity of model summary diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/Encoders.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/Encoders.scala deleted file mode 100644 index afa72171638e..000000000000 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/Encoders.scala +++ /dev/null @@ -1,25 +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.ml.clustering - -import org.apache.spark.ml.linalg.Vector -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder - -private[clustering] object Encoders { - implicit val vectorEncoder = ExpressionEncoder[Vector]() -} diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala index d692882f2d38..0b91f502f615 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala @@ -30,9 +30,8 @@ import org.apache.spark.sql.{Dataset, Row} class GaussianMixtureSuite extends MLTest with DefaultReadWriteTest { - import testImplicits._ - import Encoders._ import GaussianMixtureSuite._ + import testImplicits._ final val k = 5 private val seed = 538009335 @@ -120,7 +119,7 @@ class GaussianMixtureSuite extends MLTest with DefaultReadWriteTest { assert(model.gaussians.length === k) // Check prediction matches the highest probability, and probabilities sum to one. - testTransformer[Vector](dataset.toDF(), model, + testTransformer[Tuple1[Vector]](dataset.toDF(), model, "features", predictionColName, probabilityColName) { case Row(_, pred: Int, prob: Vector) => val probArray = prob.toArray diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index d14c9fcb1a16..344203cfd8f5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -112,7 +112,7 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes val model = kmeans.fit(dataset) assert(model.clusterCenters.length === k) - testTransformerByGlobalCheckFunc[Vector](dataset.toDF(), model, + testTransformerByGlobalCheckFunc[Tuple1[Vector]](dataset.toDF(), model, "features", predictionColName) { rows => val clusters = rows.map(_.getAs[Int](predictionColName)).toSet assert(clusters.size === k) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala index 8c13cbe7c0e7..dfa66a18a118 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql._ object LDASuite { + def generateLDAData( spark: SparkSession, rows: Int, @@ -63,7 +64,6 @@ object LDASuite { class LDASuite extends MLTest with DefaultReadWriteTest { import testImplicits._ - import Encoders._ val k: Int = 5 val vocabSize: Int = 30 @@ -186,7 +186,7 @@ class LDASuite extends MLTest with DefaultReadWriteTest { assert(model.topicsMatrix.numCols === k) assert(!model.isDistributed) - testTransformer[Vector](dataset.toDF(), model, + testTransformer[Tuple1[Vector]](dataset.toDF(), model, "features", lda.getTopicDistributionCol) { case Row(_, topicDistribution: Vector) => assert(topicDistribution.size === k) From 95f6f2919a5ab0853434592000f5b811915ca15f Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 17 May 2018 15:11:20 -0700 Subject: [PATCH 4/4] more cleanups --- .../test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala | 2 -- .../test/scala/org/apache/spark/ml/clustering/LDASuite.scala | 2 -- 2 files changed, 4 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index 344203cfd8f5..2569e7a432ca 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -32,10 +32,8 @@ import org.apache.spark.mllib.clustering.{DistanceMeasure, KMeans => MLlibKMeans import org.apache.spark.mllib.linalg.{Vectors => MLlibVectors} import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} - private[clustering] case class TestRow(features: Vector) - class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTest { import testImplicits._ diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala index dfa66a18a118..096b5416899e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala @@ -26,9 +26,7 @@ import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.sql._ - object LDASuite { - def generateLDAData( spark: SparkSession, rows: Int,