Skip to content

Commit 5682bfc

Browse files
committed
deprecate validateParameters
1 parent 035d3ac commit 5682bfc

32 files changed

+30
-88
lines changed

mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala

Lines changed: 0 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -110,12 +110,6 @@ class Pipeline @Since("1.4.0") (
110110
@Since("1.2.0")
111111
def getStages: Array[PipelineStage] = $(stages).clone()
112112

113-
@Since("1.4.0")
114-
override def validateParams(): Unit = {
115-
super.validateParams()
116-
$(stages).foreach(_.validateParams())
117-
}
118-
119113
/**
120114
* Fits the pipeline to the input dataset with additional parameters. If a stage is an
121115
* [[Estimator]], its [[Estimator#fit]] method will be called on the input dataset to fit a model.
@@ -175,7 +169,6 @@ class Pipeline @Since("1.4.0") (
175169

176170
@Since("1.2.0")
177171
override def transformSchema(schema: StructType): StructType = {
178-
validateParams()
179172
val theStages = $(stages)
180173
require(theStages.toSet.size == theStages.length,
181174
"Cannot have duplicate components in a pipeline.")
@@ -297,12 +290,6 @@ class PipelineModel private[ml] (
297290
this(uid, stages.asScala.toArray)
298291
}
299292

300-
@Since("1.4.0")
301-
override def validateParams(): Unit = {
302-
super.validateParams()
303-
stages.foreach(_.validateParams())
304-
}
305-
306293
@Since("1.2.0")
307294
override def transform(dataset: DataFrame): DataFrame = {
308295
transformSchema(dataset.schema, logging = true)
@@ -311,7 +298,6 @@ class PipelineModel private[ml] (
311298

312299
@Since("1.2.0")
313300
override def transformSchema(schema: StructType): StructType = {
314-
validateParams()
315301
stages.foldLeft(schema)((cur, transformer) => transformer.transformSchema(cur))
316302
}
317303

mllib/src/main/scala/org/apache/spark/ml/Predictor.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,6 @@ private[ml] trait PredictorParams extends Params
4646
schema: StructType,
4747
fitting: Boolean,
4848
featuresDataType: DataType): StructType = {
49-
validateParams()
5049
// TODO: Support casting Array[Double] and Array[Float] to Vector when FeaturesType = Vector
5150
SchemaUtils.checkColumnType(schema, $(featuresCol), featuresDataType)
5251
if (fitting) {

mllib/src/main/scala/org/apache/spark/ml/Transformer.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -103,7 +103,6 @@ abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, OUT, T]]
103103
protected def validateInputType(inputType: DataType): Unit = {}
104104

105105
override def transformSchema(schema: StructType): StructType = {
106-
validateParams()
107106
val inputType = schema($(inputCol)).dataType
108107
validateInputType(inputType)
109108
if (schema.fieldNames.contains($(outputCol))) {

mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -81,7 +81,6 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe
8181
* @return output schema
8282
*/
8383
protected def validateAndTransformSchema(schema: StructType): StructType = {
84-
validateParams()
8584
SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT)
8685
SchemaUtils.appendColumn(schema, $(predictionCol), IntegerType)
8786
}

mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala

Lines changed: 2 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -263,13 +263,6 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM
263263
* @return output schema
264264
*/
265265
protected def validateAndTransformSchema(schema: StructType): StructType = {
266-
validateParams()
267-
SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT)
268-
SchemaUtils.appendColumn(schema, $(topicDistributionCol), new VectorUDT)
269-
}
270-
271-
@Since("1.6.0")
272-
override def validateParams(): Unit = {
273266
if (isSet(docConcentration)) {
274267
if (getDocConcentration.length != 1) {
275268
require(getDocConcentration.length == getK, s"LDA docConcentration was of length" +
@@ -297,6 +290,8 @@ private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasM
297290
s" must be >= 1. Found value: $getTopicConcentration")
298291
}
299292
}
293+
SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT)
294+
SchemaUtils.appendColumn(schema, $(topicDistributionCol), new VectorUDT)
300295
}
301296

302297
private[clustering] def getOldOptimizer: OldLDAOptimizer = getOptimizer match {

mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -86,7 +86,6 @@ final class Bucketizer(override val uid: String)
8686
}
8787

8888
override def transformSchema(schema: StructType): StructType = {
89-
validateParams()
9089
SchemaUtils.checkColumnType(schema, $(inputCol), DoubleType)
9190
SchemaUtils.appendColumn(schema, prepOutputField(schema))
9291
}

mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -88,7 +88,6 @@ final class ChiSqSelector(override val uid: String)
8888
}
8989

9090
override def transformSchema(schema: StructType): StructType = {
91-
validateParams()
9291
SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT)
9392
SchemaUtils.checkColumnType(schema, $(labelCol), DoubleType)
9493
SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT)
@@ -136,7 +135,6 @@ final class ChiSqSelectorModel private[ml] (
136135
}
137136

138137
override def transformSchema(schema: StructType): StructType = {
139-
validateParams()
140138
SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT)
141139
val newField = prepOutputField(schema)
142140
val outputFields = schema.fields :+ newField

mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -70,7 +70,6 @@ private[feature] trait CountVectorizerParams extends Params with HasInputCol wit
7070

7171
/** Validates and transforms the input schema. */
7272
protected def validateAndTransformSchema(schema: StructType): StructType = {
73-
validateParams()
7473
val typeCandidates = List(new ArrayType(StringType, true), new ArrayType(StringType, false))
7574
SchemaUtils.checkColumnTypes(schema, $(inputCol), typeCandidates)
7675
SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT)

mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -69,7 +69,6 @@ class HashingTF(override val uid: String)
6969
}
7070

7171
override def transformSchema(schema: StructType): StructType = {
72-
validateParams()
7372
val inputType = schema($(inputCol)).dataType
7473
require(inputType.isInstanceOf[ArrayType],
7574
s"The input column must be ArrayType, but got $inputType.")

mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -52,7 +52,6 @@ private[feature] trait IDFBase extends Params with HasInputCol with HasOutputCol
5252
* Validate and transform the input schema.
5353
*/
5454
protected def validateAndTransformSchema(schema: StructType): StructType = {
55-
validateParams()
5655
SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT)
5756
SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT)
5857
}

0 commit comments

Comments
 (0)