diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index e6f124ef7d666..e451d4daffbc7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -215,10 +215,10 @@ object IDFModel extends MLReadable[IDFModel] { val data = sparkSession.read.parquet(dataPath) val model = if (majorVersion(metadata.sparkVersion) >= 3) { - val Row(idf: Vector, df: Seq[_], numDocs: Long) = data.select("idf", "docFreq", "numDocs") - .head() + val Row(idf: Vector, df: scala.collection.Seq[_], numDocs: Long) = + data.select("idf", "docFreq", "numDocs").head() new IDFModel(metadata.uid, new feature.IDFModel(OldVectors.fromML(idf), - df.asInstanceOf[Seq[Long]].toArray, numDocs)) + df.asInstanceOf[scala.collection.Seq[Long]].toArray, numDocs)) } else { val Row(idf: Vector) = MLUtils.convertVectorColumnsToML(data, "idf") .select("idf") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala index be467c654aaa1..12cae13174379 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala @@ -224,7 +224,7 @@ object MinHashLSHModel extends MLReadable[MinHashLSHModel] { val dataPath = new Path(path, "data").toString val data = sparkSession.read.parquet(dataPath).select("randCoefficients").head() - val randCoefficients = data.getAs[Seq[Int]](0).grouped(2) + val randCoefficients = data.getSeq[Int](0).grouped(2) .map(tuple => (tuple(0), tuple(1))).toArray val model = new MinHashLSHModel(metadata.uid, randCoefficients) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index b8da020017f12..563e1708acdf1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -449,7 +449,7 @@ object RFormulaModel extends MLReadable[RFormulaModel] { val dataPath = new Path(path, "data").toString val data = sparkSession.read.parquet(dataPath).select("label", "terms", "hasIntercept").head() val label = data.getString(0) - val terms = data.getAs[Seq[Seq[String]]](1) + val terms = data.getSeq[Seq[String]](1) val hasIntercept = data.getBoolean(2) val resolvedRFormula = ResolvedRFormula(label, terms, hasIntercept) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index ab51fe6e78bd7..0ca88b8e61e29 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -220,7 +220,8 @@ class StringIndexer @Since("1.4.0") ( val selectedCols = getSelectedCols(dataset, inputCols).map(collect_set(_)) val allLabels = dataset.select(selectedCols: _*) - .collect().toSeq.flatMap(_.toSeq).asInstanceOf[Seq[Seq[String]]] + .collect().toSeq.flatMap(_.toSeq) + .asInstanceOf[scala.collection.Seq[scala.collection.Seq[String]]].toSeq ThreadUtils.parmap(allLabels, "sortingStringLabels", 8) { labels => val sorted = labels.filter(_ != null).sorted if (ascending) { @@ -522,7 +523,7 @@ object StringIndexerModel extends MLReadable[StringIndexerModel] { val data = sparkSession.read.parquet(dataPath) .select("labelsArray") .head() - data.getAs[Seq[Seq[String]]](0).map(_.toArray).toArray + data.getSeq[scala.collection.Seq[String]](0).map(_.toArray).toArray } val model = new StringIndexerModel(metadata.uid, labelsArray) metadata.getAndSetParams(model) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala index b7cf4392cd177..874b421387279 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -300,7 +300,7 @@ class VectorIndexerModel private[ml] ( /** Java-friendly version of [[categoryMaps]] */ @Since("1.4.0") def javaCategoryMaps: JMap[JInt, JMap[JDouble, JInt]] = { - categoryMaps.mapValues(_.asJava).asJava.asInstanceOf[JMap[JInt, JMap[JDouble, JInt]]] + categoryMaps.mapValues(_.asJava).toMap.asJava.asInstanceOf[JMap[JInt, JMap[JDouble, JInt]]] } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index 01db39f9e3921..9b5f5a619e02c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -169,7 +169,8 @@ final class Word2Vec @Since("1.4.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): Word2VecModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).rdd.map(_.getAs[Seq[String]](0)) + val input = + dataset.select($(inputCol)).rdd.map(_.getSeq[String](0)) val wordVectors = new feature.Word2Vec() .setLearningRate($(stepSize)) .setMinCount($(minCount)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala index 7aab4ef62c4d9..8aaa5efdf06c5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala @@ -363,7 +363,7 @@ object FPGrowthModel extends MLReadable[FPGrowthModel] { Map.empty[Any, Double] } else { frequentItems.rdd.flatMap { - case Row(items: Seq[_], count: Long) if items.length == 1 => + case Row(items: scala.collection.Seq[_], count: Long) if items.length == 1 => Some(items.head -> count.toDouble / numTrainingRecords) case _ => None }.collectAsMap() diff --git a/mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala index c9c049248f70c..10a569a8ff88b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala @@ -146,7 +146,7 @@ final class PrefixSpan(@Since("2.4.0") override val uid: String) extends Params val data = dataset.select(sequenceColParam) val sequences = data.where(col(sequenceColParam).isNotNull).rdd - .map(r => r.getAs[Seq[Seq[Any]]](0).map(_.toArray).toArray) + .map(r => r.getSeq[scala.collection.Seq[Any]](0).map(_.toArray).toArray) val mllibPrefixSpan = new mllibPrefixSpan() .setMinSupport($(minSupport)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 586f622fc47c5..5b13deffcf056 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -216,7 +216,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { val data = dataArray(0) val labels = data.getAs[Seq[Double]](0).toArray val pi = data.getAs[Seq[Double]](1).toArray - val theta = data.getAs[Seq[Seq[Double]]](2).map(_.toArray).toArray + val theta = data.getSeq[scala.collection.Seq[Double]](2).map(_.toArray).toArray val modelType = data.getString(3) new NaiveBayesModel(labels, pi, theta, modelType) } @@ -260,7 +260,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { val data = dataArray(0) val labels = data.getAs[Seq[Double]](0).toArray val pi = data.getAs[Seq[Double]](1).toArray - val theta = data.getAs[Seq[Seq[Double]]](2).map(_.toArray).toArray + val theta = data.getSeq[scala.collection.Seq[Double]](2).map(_.toArray).toArray new NaiveBayesModel(labels, pi, theta) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala index de3209c34bf07..cd71aac34c268 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala @@ -683,7 +683,7 @@ object PrefixSpanModel extends Loader[PrefixSpanModel[_]] { def loadImpl[Item: ClassTag](freqSequences: DataFrame, sample: Item): PrefixSpanModel[Item] = { val freqSequencesRDD = freqSequences.select("sequence", "freq").rdd.map { x => - val sequence = x.getAs[Seq[Seq[Item]]](0).map(_.toArray).toArray + val sequence = x.getSeq[scala.collection.Seq[Item]](0).map(_.toArray).toArray val freq = x.getLong(1) new PrefixSpan.FreqSequence(sequence, freq) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index e5e82d19f1cbd..d79314b9637a8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -386,12 +386,12 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { assert(formatVersion == thisFormatVersion) val rank = (metadata \ "rank").extract[Int] val userFeatures = spark.read.parquet(userPath(path)).rdd.map { - case Row(id: Int, features: Seq[_]) => - (id, features.asInstanceOf[Seq[Double]].toArray) + case Row(id: Int, features: scala.collection.Seq[_]) => + (id, features.asInstanceOf[scala.collection.Seq[Double]].toArray) } val productFeatures = spark.read.parquet(productPath(path)).rdd.map { - case Row(id: Int, features: Seq[_]) => - (id, features.asInstanceOf[Seq[Double]].toArray) + case Row(id: Int, features: scala.collection.Seq[_]) => + (id, features.asInstanceOf[scala.collection.Seq[Double]].toArray) } new MatrixFactorizationModel(rank, userFeatures, productFeatures) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 9983ca7dc5e87..cdc998000c2fc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -164,7 +164,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { } def apply(r: Row): SplitData = { - SplitData(r.getInt(0), r.getDouble(1), r.getInt(2), r.getAs[Seq[Double]](3)) + SplitData(r.getInt(0), r.getDouble(1), r.getInt(2), r.getSeq[Double](3)) } } 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 d0898220b80de..e05d76cf70ed3 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 @@ -199,11 +199,11 @@ class LDASuite extends MLTest with DefaultReadWriteTest { assert(topics.count() === k) assert(topics.select("topic").rdd.map(_.getInt(0)).collect().toSet === Range(0, k).toSet) topics.select("termIndices").collect().foreach { case r: Row => - val termIndices = r.getAs[Seq[Int]](0) + val termIndices = r.getSeq[Int](0) assert(termIndices.length === 3 && termIndices.toSet.size === 3) } topics.select("termWeights").collect().foreach { case r: Row => - val termWeights = r.getAs[Seq[Double]](0) + val termWeights = r.getSeq[Double](0) assert(termWeights.length === 3 && termWeights.forall(w => w >= 0.0 && w <= 1.0)) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSHSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSHSuite.scala index 9b823259b1deb..a7d320e8164b6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSHSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSHSuite.scala @@ -115,7 +115,7 @@ class BucketedRandomProjectionLSHSuite extends MLTest with DefaultReadWriteTest val brpModel = brp.fit(dataset) testTransformer[Tuple1[Vector]](dataset.toDF(), brpModel, "values") { - case Row(values: Seq[_]) => + case Row(values: scala.collection.Seq[_]) => assert(values.length === brp.getNumHashTables) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala index 1d052fbebd92d..93564681994d7 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala @@ -71,7 +71,8 @@ private[ml] object LSHTest { transformedData.schema, model.getOutputCol, DataTypes.createArrayType(new VectorUDT)) // Check output column dimensions - val headHashValue = transformedData.select(outputCol).head().get(0).asInstanceOf[Seq[Vector]] + val headHashValue = + transformedData.select(outputCol).head().get(0).asInstanceOf[scala.collection.Seq[Vector]] assert(headHashValue.length == model.getNumHashTables) // Perform a cross join and label each pair of same_bucket and distance diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/MinHashLSHSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/MinHashLSHSuite.scala index 1c2956cb82908..c99e0fa3f8623 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/MinHashLSHSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/MinHashLSHSuite.scala @@ -182,7 +182,7 @@ class MinHashLSHSuite extends MLTest with DefaultReadWriteTest { val model = new MinHashLSHModel("mh", randCoefficients = Array((1, 0))) model.set(model.inputCol, "keys") testTransformer[Tuple1[Vector]](dataset.toDF(), model, "keys", model.getOutputCol) { - case Row(_: Vector, output: Seq[_]) => + case Row(_: Vector, output: scala.collection.Seq[_]) => assert(output.length === model.randCoefficients.length) // no AND-amplification yet: SPARK-18450, so each hash output is of length 1 output.foreach { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala index 1483d5df4d224..bf276ceed2097 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala @@ -83,7 +83,7 @@ class NGramSuite extends MLTest with DefaultReadWriteTest { def testNGram(t: NGram, dataFrame: DataFrame): Unit = { testTransformer[(Seq[String], Seq[String])](dataFrame, t, "nGrams", "wantedNGrams") { - case Row(actualNGrams : Seq[_], wantedNGrams: Seq[_]) => + case Row(actualNGrams : scala.collection.Seq[_], wantedNGrams: scala.collection.Seq[_]) => assert(actualNGrams === wantedNGrams) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala index dc6fb31a1f8e4..eaf91769a08dd 100755 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala @@ -29,7 +29,7 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { def testStopWordsRemover(t: StopWordsRemover, dataFrame: DataFrame): Unit = { testTransformer[(Array[String], Array[String])](dataFrame, t, "filtered", "expected") { - case Row(tokens: Seq[_], wantedTokens: Seq[_]) => + case Row(tokens: scala.collection.Seq[_], wantedTokens: scala.collection.Seq[_]) => assert(tokens === wantedTokens) } } @@ -242,7 +242,8 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { remover.transform(df) .select("filtered1", "expected1", "filtered2", "expected2") .collect().foreach { - case Row(r1: Seq[_], e1: Seq[_], r2: Seq[_], e2: Seq[_]) => + case Row(r1: scala.collection.Seq[_], e1: scala.collection.Seq[_], + r2: scala.collection.Seq[_], e2: scala.collection.Seq[_]) => assert(r1 === e1, s"The result value is not correct after bucketing. Expected $e1 but found $r1") assert(r2 === e2, @@ -268,7 +269,8 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { remover.transform(df) .select("filtered1", "expected1", "filtered2", "expected2") .collect().foreach { - case Row(r1: Seq[_], e1: Seq[_], r2: Seq[_], e2: Seq[_]) => + case Row(r1: scala.collection.Seq[_], e1: scala.collection.Seq[_], + r2: scala.collection.Seq[_], e2: scala.collection.Seq[_]) => assert(r1 === e1, s"The result value is not correct after bucketing. Expected $e1 but found $r1") assert(r2 === e2, diff --git a/mllib/src/test/scala/org/apache/spark/ml/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/fpm/FPGrowthSuite.scala index d42ced0f8f91b..3d994366b8918 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/fpm/FPGrowthSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/fpm/FPGrowthSuite.scala @@ -121,7 +121,7 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul val prediction = model.transform( spark.createDataFrame(Seq(Tuple1(Array("1", "2")))).toDF("items") - ).first().getAs[Seq[String]]("prediction") + ).first().getAs[scala.collection.Seq[String]]("prediction") assert(prediction === Seq("3")) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala index 31dc6d379e76c..aeddb5ac7b13e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala @@ -175,7 +175,7 @@ class RandomForestRegressorSuite extends MLTest with DefaultReadWriteTest{ val testParams = Seq( (50, 5, 1.0, 0.75), (50, 10, 1.0, 0.75), - (50, 10, 0.95, 0.78) + (50, 10, 0.95, 0.75) ) for ((numTrees, maxDepth, subsamplingRate, tol) <- testParams) { diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestSuite.scala index 20c5b5395f6a4..1732469ccf590 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestSuite.scala @@ -47,7 +47,7 @@ class MLTestSuite extends MLTest { } intercept[Exception] { testTransformerOnStreamData[(Int, String)](data, indexerModel, "id", "indexed") { - rows: Seq[Row] => + rows: scala.collection.Seq[Row] => assert(rows.map(_.getDouble(1)).max === 1.0) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala index a679fe43414f2..e4cd492be3d2e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -43,7 +43,8 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { // and a Word2VecMap give the same values. val word2VecMap = model.getVectors val newModel = new Word2VecModel(word2VecMap) - assert(newModel.getVectors.mapValues(_.toSeq) === word2VecMap.mapValues(_.toSeq)) + assert(newModel.getVectors.mapValues(_.toSeq).toMap === + word2VecMap.mapValues(_.toSeq).toMap) } test("Word2Vec throws exception when vocabulary is empty") { @@ -102,7 +103,8 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { try { model.save(sc, path) val sameModel = Word2VecModel.load(sc, path) - assert(sameModel.getVectors.mapValues(_.toSeq) === model.getVectors.mapValues(_.toSeq)) + assert(sameModel.getVectors.mapValues(_.toSeq).toMap === + model.getVectors.mapValues(_.toSeq).toMap) } finally { Utils.deleteRecursively(tempDir) } @@ -136,7 +138,8 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { try { model.save(sc, path) val sameModel = Word2VecModel.load(sc, path) - assert(sameModel.getVectors.mapValues(_.toSeq) === model.getVectors.mapValues(_.toSeq)) + assert(sameModel.getVectors.mapValues(_.toSeq).toMap === + model.getVectors.mapValues(_.toSeq).toMap) } catch { case t: Throwable => fail("exception thrown persisting a model " +