You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by hhbyyh <gi...@git.apache.org> on 2016/03/09 09:01:06 UTC

[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

GitHub user hhbyyh opened a pull request:

    https://github.com/apache/spark/pull/11601

    [SPARK-13568] [ML] Create feature transformer to impute missing values

    ## What changes were proposed in this pull request?
    
    It is quite common to encounter missing values in data sets. It would be useful to implement a Transformer that can impute missing data points, similar to e.g. Imputer in scikit-learn.
    Initially, options for imputation could include mean, median and most frequent, but we could add various other approaches. Where possible existing DataFrame code can be used (e.g. for approximate quantiles etc).
    
    Currently this PR supports imputation for Double and Vector (null and NaN in Vector).
    
    
    ## How was this patch tested?
    
    new unit tests and manual test
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/hhbyyh/spark imputer

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/11601.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #11601
    
----
commit 2999b268192e244bd7a520d62a0914e4742ee45d
Author: Yuhao Yang <hh...@gmail.com>
Date:   2016-02-29T17:46:04Z

    initial commit for Imputer

commit 8335cf21ebde164a22f3447000a1c468a69f39fc
Author: Yuhao Yang <hh...@gmail.com>
Date:   2016-02-29T18:27:40Z

    adjust mean and most

commit 7be5e9bcb2c9cd7671d128b01f5090ee737d207a
Author: Yuhao Yang <hh...@gmail.com>
Date:   2016-03-02T17:44:50Z

    Merge remote-tracking branch 'upstream/master' into imputer

commit 131f7d5b061a75242e7c305ba14c8c759d09c532
Author: Yuhao Yang <hh...@gmail.com>
Date:   2016-03-03T03:07:21Z

    Merge remote-tracking branch 'upstream/master' into imputer

commit a72a3ea81f6f76439068650cf47e4f784e0c4b7c
Author: Yuhao Yang <hh...@gmail.com>
Date:   2016-03-05T19:00:37Z

    Merge remote-tracking branch 'upstream/master' into imputer

commit 78df589e488bbec963b3969012cf9266fe4895cb
Author: Yuhao Yang <hh...@gmail.com>
Date:   2016-03-07T20:26:00Z

    Merge remote-tracking branch 'upstream/master' into imputer

commit b949be5746608ca3861df672ccd76d9af4257ae2
Author: Yuhao Yang <hh...@gmail.com>
Date:   2016-03-09T02:19:32Z

    refine code and add ut

commit 79b1c62b644aa05f07a33f13cc78f47a99d7e861
Author: Yuhao Yang <hh...@gmail.com>
Date:   2016-03-09T02:19:39Z

    Merge remote-tracking branch 'upstream/master' into imputer

commit c3d5d554f5ee90a18d96ff043f03f51f49d2ca7f
Author: Yuhao Yang <hh...@gmail.com>
Date:   2016-03-09T03:52:04Z

    minor change

commit 1b3966800982fa980307d1b6ded6e28e5f5985e8
Author: Yuhao Yang <hh...@gmail.com>
Date:   2016-03-09T07:57:38Z

    add object Imputer and ut refine

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-216472078
  
    @MLnick Yes, I've been preparing the doc and examples and they are almost ready. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-214780361
  
    @hhbyyh sorry for the delay. I've made some comments and suggestions.
    
    In particular, as per one of my inline comments, I'd propose we don't support the "mode" option in this first version. Let's see whether any users actually need/request it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61467752
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("exp_" + strategy, "out").collect().foreach {
    +       case Row(exp: Double, out: Double) =>
    +          assert(exp ~== out absTol 1e-5, s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer for Double with missing Value -1.0 and contains NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1.0)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("exp_" + strategy, "out").collect().foreach {
    +        case Row(exp: Double, out: Double) =>
    +          assert((exp.isNaN && out.isNaN) || (exp ~== out absTol 1e-5),
    +            s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer for Float with missing Value -1.0") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0F, 1.0F, 1.0F),
    +      (1, 3.0F, 3.0F, 3.0F),
    +      (2, 10.0F, 10.0F, 10.0F),
    +      (3, 10.0F, 10.0F, 10.0F),
    +      (4, -1.0F, 6.0F, 3.0F)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1)
    +      val model = imputer.fit(df)
    +      val result = model.transform(df)
    +      model.transform(df).select("exp_" + strategy, "out").collect().foreach {
    +        case Row(exp: Float, out: Float) =>
    +          assert(exp == out, s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer should impute null") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 4.0, 4.0, 4.0),
    +      (1, 10.0, 10.0, 10.0),
    +      (2, 10.0, 10.0, 10.0),
    +      (3, Double.NaN, 8.0, 10.0),
    +      (4, -1.0, 8.0, 10.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val df2 = df.selectExpr("*", "IF(value=-1.0, null, value) as nullable_value")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("nullable_value").setOutputCol("out")
    +        .setStrategy(strategy)
    +      val model = imputer.fit(df2)
    +      model.transform(df2).select("exp_" + strategy, "out").collect().foreach {
    --- End diff --
    
    As Jenkins has already pointed out, you'll need to change this to `.select("expected_" + ...` here and elsewhere.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104889417
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols contains" +
    +      s" duplicates: (${$(inputCols).mkString(", ")})")
    +    require($(outputCols).length == $(outputCols).distinct.length, s"outputCols contains" +
    +      s" duplicates: (${$(outputCols).mkString(", ")})")
    +    require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" +
    +      s" and outputCols(${$(outputCols).length}) should have the same length")
    +    val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) =>
    +      val inputField = schema(inputCol)
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      StructField(outputCol, inputField.dataType, inputField.nullable)
    +    }
    +    StructType(schema ++ outputFields)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed. For
    + * computing median, DataFrameStatFunctions.approxQuantile is used with a relative error of 0.001.
    + */
    +@Experimental
    +class Imputer @Since("2.2.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.2.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.2.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> Imputer.mean, missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val spark = dataset.sparkSession
    +    import spark.implicits._
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue) && !ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    --- End diff --
    
    I think we can do `filtered.take(1).size > 0` which should be more efficent


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-200830887
  
    I've optimized the "mean" for vectors. Yet for "median" and "most", still we need to iterate through the features in the vector column. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104404137
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the
    +   * feature (relative error less than 0.001).
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols duplicates:" +
    +      s" (${$(inputCols).mkString(", ")})")
    +    require($(outputCols).length == $(outputCols).distinct.length, s"outputCols duplicates:" +
    +      s" (${$(outputCols).mkString(", ")})")
    +    require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" +
    +      s" and outputCols(${$(outputCols).length}) should have the same length")
    +    val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) =>
    +      val inputField = schema(inputCol)
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      StructField(outputCol, inputField.dataType, inputField.nullable)
    +    }
    +    StructType(schema ++ outputFields)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    --- End diff --
    
    As mentioned above at https://github.com/apache/spark/pull/11601/files#r104403880, you can add the note about relative error here.
    
    Something like "For computing median, `approxQuantile` is used with a relative error of X" (provide a ScalaDoc link to approxQuantile).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r59323032
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    --- End diff --
    
    Hi Joseph, Do you mean to change "strategy" to "mode"?  "strategy" is from http://scikit-learn.org/stable/modules/generated/sklearn.preprocessing.Imputer.html


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61099521
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +      case "mode" => filtered.rdd.map(r => r.getDouble(0)).map(d => (d, 1)).reduceByKey(_ + _)
    +        .sortBy(-_._2).first()._1
    +    }
    +    copyValues(new ImputerModel(uid, surrogate).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("2.0.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median", "mode")
    +
    +  @Since("2.0.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogate statistics value for each feature during fitting
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogate: Double)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    val inputType = dataset.select($(inputCol)).schema.fields(0).dataType
    +    inputType match {
    --- End diff --
    
    is this necessary since we call `transformSchema` above?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-194184660
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/52734/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #74216 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74216/testReport)** for PR 11601 at commit [`c67afc1`](https://github.com/apache/spark/commit/c67afc11e1fee58b65da67bf3e25e5245f72280d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74216/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61152055
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    --- End diff --
    
    The way these tests are written was pretty confusing to me. It seems like the "mean" column should be the mean of some _values_ but really it is the expected output when "mean" strategy is used. This is minor since it just affects readability and might be more of a personal preference. I think the following is clearer:
    
    ```scala
    val df = sqlContext.createDataFrame( Seq(
          (0, 1.0),
          (1, 1.0),
          (2, 3.0),
          (3, 4.0),
          (4, Double.NaN)
        )).toDF("id", "value")
        val expectedOutput = Map(
          "mean"->Array(1.0, 1.0, 3.0, 4.0, 2.25),
          "median" -> Array(1.0, 1.0, 3.0, 4.0, 1.0),
          "mode" -> Array(1.0, 1.0, 3.0, 4.0, 1.0))
        Seq("mean", "median", "mode").foreach { strategy =>
          val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
          val model = imputer.fit(df)
          val result = model.transform(df).select("out").collect().map(_.getDouble(0))
          result.zip(expectedOutput(strategy)).foreach { case (actual, expected) =>
            assert(actual ~== expected absTol 1e-5)
          }
    ```
    Really, just any way of indicating that the extra columns are _expected outputs_ would be clearer to me.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r59977329
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,230 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && !ic.isNaN && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +      case "mode" => filtered.rdd.map(r => r.getDouble(0)).map(d => (d, 1)).reduceByKey(_ + _)
    +        .sortBy(-_._2).first()._1
    +    }
    +    copyValues(new ImputerModel(uid, surrogate).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("2.0.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median", "mode")
    +
    +  @Since("2.0.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogate statistics value for each feature during fitting
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogate: Double)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val inputType = dataset.select($(inputCol)).schema.fields(0).dataType
    +    inputType match {
    +      case _: NumericType =>
    +        val ic = col($(inputCol)).cast(DoubleType)
    +        dataset.withColumn($(outputCol), when(ic.isNull, surrogate)
    +          .when(ic.isNaN, surrogate)
    +          .when(ic === $(missingValue), surrogate)
    +          .otherwise(ic)
    +          .cast(inputType))
    --- End diff --
    
    I'm neutral on this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r80484748
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("expected_" + strategy, "out").collect().foreach {
    +       case Row(exp: Double, out: Double) =>
    +          assert(exp ~== out absTol 1e-5, s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    --- End diff --
    
    We need to add tests for the case where the entire column is `null` or `NaN`. I just checked the `NaN` case and it will throw a NPE in the fit method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r80518709
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("expected_" + strategy, "out").collect().foreach {
    +       case Row(exp: Double, out: Double) =>
    +          assert(exp ~== out absTol 1e-5, s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer should handle NaNs when computing surrogate value, if missingValue is not NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1.0)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("expected_" + strategy, "out").collect().foreach {
    +        case Row(exp: Double, out: Double) =>
    +          assert((exp.isNaN && out.isNaN) || (exp ~== out absTol 1e-5),
    +            s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer for Float with missing Value -1.0") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0F, 1.0F, 1.0F),
    +      (1, 3.0F, 3.0F, 3.0F),
    +      (2, 10.0F, 10.0F, 10.0F),
    +      (3, 10.0F, 10.0F, 10.0F),
    +      (4, -1.0F, 6.0F, 3.0F)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1)
    +      val model = imputer.fit(df)
    +      val result = model.transform(df)
    --- End diff --
    
    this is never used.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r80592752
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    --- End diff --
    
    This is already checked in appendColumn


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61848348
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +    }
    +    copyValues(new ImputerModel(uid, surrogate).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("2.0.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median")
    +
    +  @Since("2.0.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogate statistics value for each feature during fitting
    --- End diff --
    
    This comment could be clarified - something like "value by which missing values in the input column will be replaced".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    jenkins retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r58922737
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    --- End diff --
    
    What about null values?  Should we treat all null values as missing?  I could imagine cases in which people want to handle both NaN and null.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r80593061
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    --- End diff --
    
    Say here that this does not support categorical features yet and will transform them, possibly creating incorrect values for a categorical feature.  Also add JIRA number for supporting them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104404523
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the
    +   * feature (relative error less than 0.001).
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols duplicates:" +
    +      s" (${$(inputCols).mkString(", ")})")
    +    require($(outputCols).length == $(outputCols).distinct.length, s"outputCols duplicates:" +
    +      s" (${$(outputCols).mkString(", ")})")
    +    require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" +
    +      s" and outputCols(${$(outputCols).length}) should have the same length")
    +    val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) =>
    +      val inputField = schema(inputCol)
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      StructField(outputCol, inputField.dataType, inputField.nullable)
    +    }
    +    StructType(schema ++ outputFields)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.2.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.2.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.2.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  import org.apache.spark.ml.feature.Imputer._
    +  setDefault(strategy -> mean, missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val spark = dataset.sparkSession
    +    import spark.implicits._
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue) && !ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in $inputCol are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case Imputer.mean => filtered.select(avg(inputCol)).as[Double].first()
    +        case Imputer.median => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001).head
    --- End diff --
    
    Not really sure about the relative error here - perhaps `0.01` is sufficient?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104890651
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols contains" +
    +      s" duplicates: (${$(inputCols).mkString(", ")})")
    +    require($(outputCols).length == $(outputCols).distinct.length, s"outputCols contains" +
    +      s" duplicates: (${$(outputCols).mkString(", ")})")
    +    require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" +
    +      s" and outputCols(${$(outputCols).length}) should have the same length")
    +    val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) =>
    +      val inputField = schema(inputCol)
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      StructField(outputCol, inputField.dataType, inputField.nullable)
    +    }
    +    StructType(schema ++ outputFields)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed. For
    + * computing median, DataFrameStatFunctions.approxQuantile is used with a relative error of 0.001.
    + */
    +@Experimental
    +class Imputer @Since("2.2.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.2.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.2.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> Imputer.mean, missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val spark = dataset.sparkSession
    +    import spark.implicits._
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue) && !ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in $inputCol are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case Imputer.mean => filtered.select(avg(inputCol)).as[Double].first()
    +        case Imputer.median => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001).head
    +      }
    +      surrogate
    +    }
    +
    +    val rows = spark.sparkContext.parallelize(Seq(Row.fromSeq(surrogates)))
    +    val schema = StructType($(inputCols).map(col => StructField(col, DoubleType, nullable = false)))
    +    val surrogateDF = spark.createDataFrame(rows, schema)
    +    copyValues(new ImputerModel(uid, surrogateDF).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = defaultCopy(extra)
    +}
    +
    +@Since("2.2.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** strategy names that Imputer currently supports. */
    +  private[ml] val mean = "mean"
    +  private[ml] val median = "median"
    +
    +  @Since("2.2.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogateDF a DataFrame contains inputCols and their corresponding surrogates, which are
    + *                    used to replace the missing values in the input DataFrame.
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogateDF: DataFrame)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    var outputDF = dataset
    +    val surrogates = surrogateDF.select($(inputCols).map(col): _*).head().toSeq
    +
    +    $(inputCols).zip($(outputCols)).zip(surrogates).foreach {
    +      case ((inputCol, outputCol), surrogate) =>
    +        val inputType = dataset.schema(inputCol).dataType
    --- End diff --
    
    maybe just make a `val schema = dataset.schema`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104257741
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    --- End diff --
    
    Sure, however I didn't get your first comment. Do you mean we should remove the import?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/65908/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-216008178
  
    **[Test build #57459 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57459/consoleFull)** for PR 11601 at commit [`cca8dd4`](https://github.com/apache/spark/commit/cca8dd41714d79476c2bf23f706012a282c53bcb).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103874503
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,136 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.{DataFrame, Row}
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should handle NaNs when computing surrogate value, if missingValue is not NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1.0)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer for Float with missing Value -1.0") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0F, 1.0F, 1.0F),
    +      (1, 3.0F, 3.0F, 3.0F),
    +      (2, 10.0F, 10.0F, 10.0F),
    +      (3, 10.0F, 10.0F, 10.0F),
    +      (4, -1.0F, 6.0F, 3.0F)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should impute null as well as 'missingValue'") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 4.0, 4.0, 4.0),
    +      (1, 10.0, 10.0, 10.0),
    +      (2, 10.0, 10.0, 10.0),
    +      (3, Double.NaN, 8.0, 10.0),
    +      (4, -1.0, 8.0, 10.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val df2 = df.selectExpr("*", "IF(value=-1.0, null, value) as nullable_value")
    +    val imputer = new Imputer().setInputCols(Array("nullable_value")).setOutputCols(Array("out"))
    +    ImputerSuite.iterateStrategyTest(imputer, df2)
    +  }
    +
    +
    +  test("Imputer throws exception when surrogate cannot be computed") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, Double.NaN, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +        .setStrategy(strategy)
    +      intercept[SparkException] {
    +        val model = imputer.fit(df)
    +      }
    +    }
    +  }
    +
    +  test("Imputer read/write") {
    +    val t = new Imputer()
    +      .setInputCols(Array("myInputCol"))
    +      .setOutputCols(Array("myOutputCol"))
    +      .setMissingValue(-1.0)
    +    testDefaultReadWrite(t)
    +  }
    +
    +  test("ImputerModel read/write") {
    +    val spark = this.spark
    +    import spark.implicits._
    +    val surrogateDF = Seq(1.234).toDF("myInputCol")
    --- End diff --
    
    This should be "surrogate" col name - though I see we don't actually use it in load or transform


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-195008735
  
    @sethah @MLnick Thanks for helping with review. I made a pass according to the comments and add some more comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-200333487
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-200274744
  
    **[Test build #53923 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53923/consoleFull)** for PR 11601 at commit [`1b36deb`](https://github.com/apache/spark/commit/1b36deb3eb0391ec7080bafebd2dfb662d09a6e4).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by BenFradet <gi...@git.apache.org>.
Github user BenFradet commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61111326
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    --- End diff --
    
    I think the word "values" is missing in this one.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r58922828
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  private[feature] def isMissingValue(value: Double): Boolean = {
    +    val miss = $(missingValue)
    +    value == miss || (value.isNaN && miss.isNaN)
    +  }
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    --- End diff --
    
    +1 for using existing implementations in DataFrame or spark.mllib.stats.  I don't think we need to implement anything new here, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61157610
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    --- End diff --
    
    Make all Param vals final


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61286255
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0, 1.0 )
    +    )).toDF("id", "value", "mean", "median", "mode")
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Double, d2: Double) =>
    +          assert(d1 ~== d2 absTol 1e-5, s"Imputer ut error: $d2 should be $d1")
    +        }
    +    }
    +  }
    +
    +  test("Imputer for Double with missing Value -1.0") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0, 4.0),
    +      (4, -1.0, 2.25, 1.0, 1.0 )
    +    )).toDF("id", "value", "mean", "median", "mode")
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1.0)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Double, d2: Double) =>
    +          assert(d1 ~== d2 absTol 1e-5, s"Imputer ut error: $d2 should be $d1")
    +        }
    +    }
    +  }
    +
    +  test("Imputer for Int with missing Value -1") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1, 1, 1, 1),
    +      (1, 3, 3, 3, 3),
    +      (2, 10, 10, 10, 10),
    +      (3, 10, 10, 10, 10),
    +      (4, -1, 6, 3, 10)
    +    )).toDF("id", "value", "mean", "median", "mode")
    +
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Int, d2: Int) =>
    +          assert(d1 === d2, s"Imputer ut error: $d2 should be $d1")
    +        }
    +    }
    +  }
    +
    --- End diff --
    
    added


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74038/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-212503267
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/56372/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r58922889
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  private[feature] def isMissingValue(value: Double): Boolean = {
    +    val miss = $(missingValue)
    +    value == miss || (value.isNaN && miss.isNaN)
    +  }
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        val doubleRDD = dataset.select($(inputCol)).rdd.map(_.getDouble(0))
    +        Vectors.dense(getColStatistics(doubleRDD))
    +      case _: VectorUDT =>
    +        val filteredDF = dataset.where(s"${$(inputCol)} IS NOT NULL").select($(inputCol))
    +        val vectorRDD = filteredDF.rdd.map(_.getAs[Vector](0)).cache()
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        $(strategy) match {
    +          case "mean" =>
    +            val summary = vectorRDD.treeAggregate((new Array[Double](vl), new Array[Int](vl)))(
    +              (prev, data) => (prev, data) match { case ((mean, count), data) =>
    +                  var i = 0
    +                  while (i < mean.length) {
    +                    if (data(i) != 0 && !data(i).isNaN) {
    +                      count(i) += 1
    +                      mean(i) = mean(i) + (data(i) - mean(i)) / count(i)
    +                    }
    +                    i += 1
    +                  }
    +                  (mean, count)
    +              }, (aggregator1, aggregator2) => (aggregator1, aggregator2) match {
    +                case ((mean1, c1), (mean2, c2)) =>
    +                  (0 until mean1.length).foreach{ i =>
    +                    mean1(i) = mean1(i) + (mean2(i) - mean1(i)) * c2(i) / (c1(i) + c2(i))
    +                    c1(i) += c2(i)
    +                  }
    +                  (mean1, c1)
    +              })
    +            Vectors.dense(summary._1)
    +          case _ =>
    +            val statisticsArray = new Array[Double](vl)
    +            (0 until vl).foreach(i => {
    +              statisticsArray(i) = getColStatistics(vectorRDD.map(v => v(i)))
    +            })
    +            Vectors.dense(statisticsArray)
    +        }
    +    }
    +    copyValues(new ImputerModel(uid, alternate).setParent(this))
    +  }
    +
    +  /** Extract the statistics info from a Double column according to the strategy */
    +  private def getColStatistics(data: RDD[Double]): Double = {
    +    val filteredRDD = data.filter(!isMissingValue(_))
    +    val colStatistics = $(strategy) match {
    +      case "mean" => filteredRDD.mean()
    +      case "median" => filteredRDD.sortBy(d => d).zipWithIndex()
    --- End diff --
    
    We did not open up the multiple columns API because approxQuantile was still experimental.  Originally, we wanted to provide approx quantiles via dataframe.describe, but decided it would make describe() too slow.  It's provided as a stat function for now, until we either speed it up or decide to keep it as a stat function.  I'm OK with opening up the multiple-column API.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-200324485
  
    **[Test build #53931 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53931/consoleFull)** for PR 11601 at commit [`72d104d`](https://github.com/apache/spark/commit/72d104d92a96ba03d60a72a5fa0b06e583a28bdc).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r55550039
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, Params}
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean along the axis.
    +    * If "median", then replace missing values using the median along the axis.
    +    * If "most", then replace missing using the most frequent value along the axis.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean along the axis." +
    +    "If median, then replace missing values using the median along the axis." +
    +    "If most, then replace missing using the most frequent value along the axis.")
    --- End diff --
    
    Could you add a param validation function since there are a limited number of valid strategies? You can add an attribute like `val supportedMissingValueStrategies = Set("mean", "median", "most")` to the `Imputer` companion object like is done [here](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala#L50)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61136387
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +      case "mode" => filtered.rdd.map(r => r.getDouble(0)).map(d => (d, 1)).reduceByKey(_ + _)
    --- End diff --
    
    I think leaving it out for now seems reasonable, but I have seen "mode" suggested for imputing categorical features. Is there no similar "approxMode" type algorithm we can take advantage of for data frames? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104410545
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the
    +   * feature (relative error less than 0.001).
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols duplicates:" +
    +      s" (${$(inputCols).mkString(", ")})")
    +    require($(outputCols).length == $(outputCols).distinct.length, s"outputCols duplicates:" +
    +      s" (${$(outputCols).mkString(", ")})")
    +    require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" +
    +      s" and outputCols(${$(outputCols).length}) should have the same length")
    +    val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) =>
    +      val inputField = schema(inputCol)
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      StructField(outputCol, inputField.dataType, inputField.nullable)
    +    }
    +    StructType(schema ++ outputFields)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.2.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.2.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.2.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  import org.apache.spark.ml.feature.Imputer._
    +  setDefault(strategy -> mean, missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val spark = dataset.sparkSession
    +    import spark.implicits._
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue) && !ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in $inputCol are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case Imputer.mean => filtered.select(avg(inputCol)).as[Double].first()
    +        case Imputer.median => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001).head
    --- End diff --
    
    Later perhaps we can even expose it as an expert param (but not for now)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-216011080
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/57459/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #73868 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73868/testReport)** for PR 11601 at commit [`41d91b9`](https://github.com/apache/spark/commit/41d91b9ef855a611016c9a9613942e578ff599dd).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103870475
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(outputCols).length, "inputCols and outputCols should have" +
    +      "the same length")
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputSchema = schema
    +
    +    $(inputCols).indices.foreach { i =>
    +      val inputCol = localInputCols(i)
    +      val outputCol = localOutputCols(i)
    +      val inputType = schema(inputCol).dataType
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      outputSchema = SchemaUtils.appendColumn(outputSchema, outputCol, inputType)
    +    }
    +    outputSchema
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.1.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue))
    --- End diff --
    
    Is it possible to just consolidate this into one `filter` (include the `!ic.isNaN`)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103869621
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(outputCols).length, "inputCols and outputCols should have" +
    +      "the same length")
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputSchema = schema
    +
    +    $(inputCols).indices.foreach { i =>
    +      val inputCol = localInputCols(i)
    +      val outputCol = localOutputCols(i)
    +      val inputType = schema(inputCol).dataType
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      outputSchema = SchemaUtils.appendColumn(outputSchema, outputCol, inputType)
    +    }
    +    outputSchema
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.1.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue))
    +        .filter(!ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in ${inputCol} are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case "mean" => filtered.select(avg(inputCol)).first().getDouble(0)
    +        case "median" => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001)(0)
    +      }
    +      surrogate.asInstanceOf[Double]
    +    }
    +
    +    import dataset.sparkSession.implicits._
    +    val surrogateDF = Seq(surrogates).toDF("surrogates")
    +    copyValues(new ImputerModel(uid, surrogateDF).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = defaultCopy(extra)
    +}
    +
    +@Since("2.1.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median")
    --- End diff --
    
    Could we factor out the `mean` and `median` names in to `private[ml] val` so to be used instead of the raw strings throughout?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103888244
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(outputCols).length, "inputCols and outputCols should have" +
    +      "the same length")
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputSchema = schema
    +
    +    $(inputCols).indices.foreach { i =>
    +      val inputCol = localInputCols(i)
    +      val outputCol = localOutputCols(i)
    +      val inputType = schema(inputCol).dataType
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      outputSchema = SchemaUtils.appendColumn(outputSchema, outputCol, inputType)
    +    }
    +    outputSchema
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.1.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue))
    +        .filter(!ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in ${inputCol} are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case "mean" => filtered.select(avg(inputCol)).first().getDouble(0)
    +        case "median" => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001)(0)
    +      }
    +      surrogate.asInstanceOf[Double]
    +    }
    +
    +    import dataset.sparkSession.implicits._
    +    val surrogateDF = Seq(surrogates).toDF("surrogates")
    +    copyValues(new ImputerModel(uid, surrogateDF).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = defaultCopy(extra)
    +}
    +
    +@Since("2.1.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median")
    +
    +  @Since("2.1.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogateDF Value by which missing values in the input columns will be replaced. This
    + *    is stored using DataFrame with input column names and the corresponding surrogates.
    --- End diff --
    
    This is misleading - you're just storing the array of `surrogates`... did you mean something different? Otherwise the comment must be changed,


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104403880
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the
    +   * feature (relative error less than 0.001).
    --- End diff --
    
    I think remove the part `(relative error less than 0.001)`.
    
    This can be moved to the overall ScalaDoc for `Imputer` at L95.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104257857
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(outputCols).length, "inputCols and outputCols should have" +
    +      "the same length")
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputSchema = schema
    +
    +    $(inputCols).indices.foreach { i =>
    +      val inputCol = localInputCols(i)
    +      val outputCol = localOutputCols(i)
    +      val inputType = schema(inputCol).dataType
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      outputSchema = SchemaUtils.appendColumn(outputSchema, outputCol, inputType)
    +    }
    +    outputSchema
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.1.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue))
    +        .filter(!ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in ${inputCol} are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case "mean" => filtered.select(avg(inputCol)).first().getDouble(0)
    +        case "median" => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001)(0)
    +      }
    +      surrogate.asInstanceOf[Double]
    --- End diff --
    
    no


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #74038 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74038/testReport)** for PR 11601 at commit [`e378db5`](https://github.com/apache/spark/commit/e378db5944d7d8bed0ebadc0573a3ea03fe387f0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215155942
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61286231
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0, 1.0 )
    +    )).toDF("id", "value", "mean", "median", "mode")
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Double, d2: Double) =>
    +          assert(d1 ~== d2 absTol 1e-5, s"Imputer ut error: $d2 should be $d1")
    +        }
    +    }
    +  }
    +
    +  test("Imputer for Double with missing Value -1.0") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0, 4.0),
    +      (4, -1.0, 2.25, 1.0, 1.0 )
    +    )).toDF("id", "value", "mean", "median", "mode")
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1.0)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Double, d2: Double) =>
    +          assert(d1 ~== d2 absTol 1e-5, s"Imputer ut error: $d2 should be $d1")
    +        }
    +    }
    +  }
    +
    +  test("Imputer for Int with missing Value -1") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1, 1, 1, 1),
    +      (1, 3, 3, 3, 3),
    +      (2, 10, 10, 10, 10),
    +      (3, 10, 10, 10, 10),
    +      (4, -1, 6, 3, 10)
    +    )).toDF("id", "value", "mean", "median", "mode")
    +
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Int, d2: Int) =>
    +          assert(d1 === d2, s"Imputer ut error: $d2 should be $d1")
    +        }
    +    }
    +  }
    +
    +  test("Imputer should impute null") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1, 1, 1, 1),
    +      (1, 3, 3, 3, 3),
    +      (2, 10, 10, 10, 10),
    +      (3, 10, 10, 10, 10),
    +      (4, -1, 6, 3, 10)
    --- End diff --
    
    added


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Sent an update to add multi-column support. Let me know if this is not what you have in mind.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61576752
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("expected_" + strategy, "out").collect().foreach {
    +       case Row(exp: Double, out: Double) =>
    +          assert(exp ~== out absTol 1e-5, s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer for Double with missing Value -1.0 and contains NaN") {
    --- End diff --
    
    Can we call this `Imputer should handle NaNs in data when 'missingValue' is not NaN` for clarity


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104408351
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,166 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.{DataFrame, Row}
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 4.0, 1.0, 1.0, 4.0, 4.0),
    +      (1, 11.0, 12.0, 11.0, 11.0, 12.0, 12.0),
    +      (2, 3.0, Double.NaN, 3.0, 3.0, 10.0, 12.0),
    +      (3, Double.NaN, 14.0, 5.0, 3.0, 14.0, 14.0)
    +    )).toDF("id", "value1", "value2", "expected_mean_value1", "expected_median_value1",
    +      "expected_mean_value2", "expected_median_value2")
    +    val imputer = new Imputer()
    +      .setInputCols(Array("value1", "value2"))
    +      .setOutputCols(Array("out1", "out2"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should handle NaNs when computing surrogate value, if missingValue is not NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1.0)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer for Float with missing Value -1.0") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0F, 1.0F, 1.0F),
    +      (1, 3.0F, 3.0F, 3.0F),
    +      (2, 10.0F, 10.0F, 10.0F),
    +      (3, 10.0F, 10.0F, 10.0F),
    +      (4, -1.0F, 6.0F, 3.0F)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should impute null as well as 'missingValue'") {
    +    val rawDf = spark.createDataFrame( Seq(
    +      (0, 4.0, 4.0, 4.0),
    +      (1, 10.0, 10.0, 10.0),
    +      (2, 10.0, 10.0, 10.0),
    +      (3, Double.NaN, 8.0, 10.0),
    +      (4, -1.0, 8.0, 10.0)
    +    )).toDF("id", "rawValue", "expected_mean_value", "expected_median_value")
    +    val df = rawDf.selectExpr("*", "IF(rawValue=-1.0, null, rawValue) as value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer throws exception when surrogate cannot be computed") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, Double.NaN, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +        .setStrategy(strategy)
    +      intercept[SparkException] {
    +        val model = imputer.fit(df)
    +      }
    +    }
    +  }
    +
    +  test("Imputer throws exception when inputCols does not match outputCols") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value1", "value2", "value3")
    +    Seq("mean", "median").foreach { strategy =>
    +      // inputCols and outCols length different
    +      val imputer = new Imputer()
    +        .setInputCols(Array("value1", "value2"))
    +        .setOutputCols(Array("out1"))
    +        .setStrategy(strategy)
    +      intercept[IllegalArgumentException] {
    +        val model = imputer.fit(df)
    +      }
    +      // duplicate name in inputCols
    +      imputer.setInputCols(Array("value1", "value1")).setOutputCols(Array("out1, out2"))
    +      intercept[IllegalArgumentException] {
    +        val model = imputer.fit(df)
    +      }
    +
    +    }
    +  }
    +
    +  test("Imputer read/write") {
    +    val t = new Imputer()
    +      .setInputCols(Array("myInputCol"))
    +      .setOutputCols(Array("myOutputCol"))
    +      .setMissingValue(-1.0)
    +    testDefaultReadWrite(t)
    +  }
    +
    +  test("ImputerModel read/write") {
    +    val spark = this.spark
    +    import spark.implicits._
    +    val surrogateDF = Seq(1.234).toDF("myInputCol")
    +
    +    val instance = new ImputerModel(
    +      "myImputer", surrogateDF)
    +      .setInputCols(Array("myInputCol"))
    +      .setOutputCols(Array("myOutputCol"))
    +    val newInstance = testDefaultReadWrite(instance)
    +    assert(newInstance.surrogateDF.collect() === instance.surrogateDF.collect())
    +  }
    +
    +}
    +
    +object ImputerSuite{
    --- End diff --
    
    space before `{`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-220254380
  
    @hhbyyh really sorry this slipped in the rush around 2.0 QA. I'm afraid it will have to be revisited after 2.0 release! But I think it should be in good shape to merge soon after that happens.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-216011079
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104408310
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,136 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.{DataFrame, Row}
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should handle NaNs when computing surrogate value, if missingValue is not NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1.0)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer for Float with missing Value -1.0") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0F, 1.0F, 1.0F),
    +      (1, 3.0F, 3.0F, 3.0F),
    +      (2, 10.0F, 10.0F, 10.0F),
    +      (3, 10.0F, 10.0F, 10.0F),
    +      (4, -1.0F, 6.0F, 3.0F)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should impute null as well as 'missingValue'") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 4.0, 4.0, 4.0),
    +      (1, 10.0, 10.0, 10.0),
    +      (2, 10.0, 10.0, 10.0),
    +      (3, Double.NaN, 8.0, 10.0),
    +      (4, -1.0, 8.0, 10.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val df2 = df.selectExpr("*", "IF(value=-1.0, null, value) as nullable_value")
    +    val imputer = new Imputer().setInputCols(Array("nullable_value")).setOutputCols(Array("out"))
    +    ImputerSuite.iterateStrategyTest(imputer, df2)
    +  }
    +
    +
    +  test("Imputer throws exception when surrogate cannot be computed") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, Double.NaN, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +        .setStrategy(strategy)
    +      intercept[SparkException] {
    +        val model = imputer.fit(df)
    +      }
    +    }
    +  }
    +
    +  test("Imputer read/write") {
    +    val t = new Imputer()
    +      .setInputCols(Array("myInputCol"))
    +      .setOutputCols(Array("myOutputCol"))
    +      .setMissingValue(-1.0)
    +    testDefaultReadWrite(t)
    +  }
    +
    +  test("ImputerModel read/write") {
    +    val spark = this.spark
    +    import spark.implicits._
    +    val surrogateDF = Seq(1.234).toDF("myInputCol")
    --- End diff --
    
    Ok - we should add a test here to check the column names of `instance` and `newInstance` match up? (The below check is just for the actual values of the surrogate, correct?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103872555
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(outputCols).length, "inputCols and outputCols should have" +
    +      "the same length")
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputSchema = schema
    +
    +    $(inputCols).indices.foreach { i =>
    +      val inputCol = localInputCols(i)
    +      val outputCol = localOutputCols(i)
    +      val inputType = schema(inputCol).dataType
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      outputSchema = SchemaUtils.appendColumn(outputSchema, outputCol, inputType)
    +    }
    +    outputSchema
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.1.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue))
    +        .filter(!ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in ${inputCol} are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case "mean" => filtered.select(avg(inputCol)).first().getDouble(0)
    +        case "median" => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001)(0)
    +      }
    +      surrogate.asInstanceOf[Double]
    +    }
    +
    +    import dataset.sparkSession.implicits._
    +    val surrogateDF = Seq(surrogates).toDF("surrogates")
    +    copyValues(new ImputerModel(uid, surrogateDF).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = defaultCopy(extra)
    +}
    +
    +@Since("2.1.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median")
    +
    +  @Since("2.1.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogateDF Value by which missing values in the input columns will be replaced. This
    + *    is stored using DataFrame with input column names and the corresponding surrogates.
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogateDF: DataFrame)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputDF = dataset
    +    val surrogates = surrogateDF.head().getSeq[Double](0)
    --- End diff --
    
    `.as[Seq[Double]].head()`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215656462
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/57321/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-208731666
  
    **[Test build #55591 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/55591/consoleFull)** for PR 11601 at commit [`fdd6f94`](https://github.com/apache/spark/commit/fdd6f943da2123aebaca4fe9d48ce6b6356bfa42).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r58666904
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  private[feature] def isMissingValue(value: Double): Boolean = {
    +    val miss = $(missingValue)
    +    value == miss || (value.isNaN && miss.isNaN)
    +  }
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        val doubleRDD = dataset.select($(inputCol)).rdd.map(_.getDouble(0))
    +        Vectors.dense(getColStatistics(doubleRDD))
    +      case _: VectorUDT =>
    +        val filteredDF = dataset.where(s"${$(inputCol)} IS NOT NULL").select($(inputCol))
    +        val vectorRDD = filteredDF.rdd.map(_.getAs[Vector](0)).cache()
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        $(strategy) match {
    +          case "mean" =>
    +            val summary = vectorRDD.treeAggregate((new Array[Double](vl), new Array[Int](vl)))(
    +              (prev, data) => (prev, data) match { case ((mean, count), data) =>
    +                  var i = 0
    +                  while (i < mean.length) {
    +                    if (data(i) != 0 && !data(i).isNaN) {
    +                      count(i) += 1
    +                      mean(i) = mean(i) + (data(i) - mean(i)) / count(i)
    +                    }
    +                    i += 1
    +                  }
    +                  (mean, count)
    +              }, (aggregator1, aggregator2) => (aggregator1, aggregator2) match {
    +                case ((mean1, c1), (mean2, c2)) =>
    +                  (0 until mean1.length).foreach{ i =>
    +                    mean1(i) = mean1(i) + (mean2(i) - mean1(i)) * c2(i) / (c1(i) + c2(i))
    +                    c1(i) += c2(i)
    +                  }
    +                  (mean1, c1)
    +              })
    +            Vectors.dense(summary._1)
    +          case _ =>
    +            val statisticsArray = new Array[Double](vl)
    +            (0 until vl).foreach(i => {
    +              statisticsArray(i) = getColStatistics(vectorRDD.map(v => v(i)))
    +            })
    +            Vectors.dense(statisticsArray)
    +        }
    +    }
    +    copyValues(new ImputerModel(uid, alternate).setParent(this))
    +  }
    +
    +  /** Extract the statistics info from a Double column according to the strategy */
    +  private def getColStatistics(data: RDD[Double]): Double = {
    +    val filteredRDD = data.filter(!isMissingValue(_))
    +    val colStatistics = $(strategy) match {
    +      case "mean" => filteredRDD.mean()
    +      case "median" => filteredRDD.sortBy(d => d).zipWithIndex()
    --- End diff --
    
    Thanks @MLnick.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-216477659
  
    **[Test build #57620 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57620/consoleFull)** for PR 11601 at commit [`4e07431`](https://github.com/apache/spark/commit/4e0743139796ac53df2554cfa53736b8035bae15).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #65908 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65908/consoleFull)** for PR 11601 at commit [`544a65c`](https://github.com/apache/spark/commit/544a65c82a7d921bdff73998e8b350e11b51dcbe).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103868184
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(outputCols).length, "inputCols and outputCols should have" +
    +      "the same length")
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputSchema = schema
    +
    +    $(inputCols).indices.foreach { i =>
    +      val inputCol = localInputCols(i)
    +      val outputCol = localOutputCols(i)
    +      val inputType = schema(inputCol).dataType
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      outputSchema = SchemaUtils.appendColumn(outputSchema, outputCol, inputType)
    +    }
    +    outputSchema
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    --- End diff --
    
    All `@Since` annotations -> `2.2.0`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104409770
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,166 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.{DataFrame, Row}
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 4.0, 1.0, 1.0, 4.0, 4.0),
    +      (1, 11.0, 12.0, 11.0, 11.0, 12.0, 12.0),
    +      (2, 3.0, Double.NaN, 3.0, 3.0, 10.0, 12.0),
    +      (3, Double.NaN, 14.0, 5.0, 3.0, 14.0, 14.0)
    +    )).toDF("id", "value1", "value2", "expected_mean_value1", "expected_median_value1",
    +      "expected_mean_value2", "expected_median_value2")
    +    val imputer = new Imputer()
    +      .setInputCols(Array("value1", "value2"))
    +      .setOutputCols(Array("out1", "out2"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should handle NaNs when computing surrogate value, if missingValue is not NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1.0)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer for Float with missing Value -1.0") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0F, 1.0F, 1.0F),
    +      (1, 3.0F, 3.0F, 3.0F),
    +      (2, 10.0F, 10.0F, 10.0F),
    +      (3, 10.0F, 10.0F, 10.0F),
    +      (4, -1.0F, 6.0F, 3.0F)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should impute null as well as 'missingValue'") {
    +    val rawDf = spark.createDataFrame( Seq(
    +      (0, 4.0, 4.0, 4.0),
    +      (1, 10.0, 10.0, 10.0),
    +      (2, 10.0, 10.0, 10.0),
    +      (3, Double.NaN, 8.0, 10.0),
    +      (4, -1.0, 8.0, 10.0)
    +    )).toDF("id", "rawValue", "expected_mean_value", "expected_median_value")
    +    val df = rawDf.selectExpr("*", "IF(rawValue=-1.0, null, rawValue) as value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer throws exception when surrogate cannot be computed") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, Double.NaN, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +        .setStrategy(strategy)
    +      intercept[SparkException] {
    +        val model = imputer.fit(df)
    +      }
    +    }
    +  }
    +
    +  test("Imputer throws exception when inputCols does not match outputCols") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value1", "value2", "value3")
    +    Seq("mean", "median").foreach { strategy =>
    +      // inputCols and outCols length different
    +      val imputer = new Imputer()
    --- End diff --
    
    You can also perhaps use `withClue` to put a message for the subtest / exception assertion (e.g. `withClue("Imputer should fail if inputCols and outputCols are different length")`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-212489019
  
    **[Test build #56372 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/56372/consoleFull)** for PR 11601 at commit [`594c501`](https://github.com/apache/spark/commit/594c501f85cad2a278caee5f08b85deb61272e5d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61306820
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    --- End diff --
    
    Sorry to nitpick, but "expected_mean" seems significantly more clear than "exp_mean" at the expense of only a few extra characters. Only change it if you push more commits since its minor :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104258573
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(outputCols).length, "inputCols and outputCols should have" +
    +      "the same length")
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputSchema = schema
    +
    +    $(inputCols).indices.foreach { i =>
    +      val inputCol = localInputCols(i)
    +      val outputCol = localOutputCols(i)
    +      val inputType = schema(inputCol).dataType
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      outputSchema = SchemaUtils.appendColumn(outputSchema, outputCol, inputType)
    +    }
    +    outputSchema
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.1.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue))
    +        .filter(!ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in ${inputCol} are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case "mean" => filtered.select(avg(inputCol)).first().getDouble(0)
    +        case "median" => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001)(0)
    +      }
    +      surrogate.asInstanceOf[Double]
    +    }
    +
    +    import dataset.sparkSession.implicits._
    +    val surrogateDF = Seq(surrogates).toDF("surrogates")
    +    copyValues(new ImputerModel(uid, surrogateDF).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = defaultCopy(extra)
    +}
    +
    +@Since("2.1.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median")
    +
    +  @Since("2.1.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogateDF Value by which missing values in the input columns will be replaced. This
    + *    is stored using DataFrame with input column names and the corresponding surrogates.
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogateDF: DataFrame)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputDF = dataset
    +    val surrogates = surrogateDF.head().getSeq[Double](0)
    --- End diff --
    
    I change it to
        val surrogates = surrogateDF.select($(inputCols).head, $(inputCols).tail: _*).head().toSeq
    
    which can actually handle different datatypes. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    I'll make a review pass now


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #65908 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65908/consoleFull)** for PR 11601 at commit [`544a65c`](https://github.com/apache/spark/commit/544a65c82a7d921bdff73998e8b350e11b51dcbe).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class Imputer @Since(\"2.1.0\")(override val uid: String)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215719811
  
    @hhbyyh if you have time, could you create 2 follow up JIRAs for:
    * PySpark impl
    * adding mode at a later date for categorical features
        * investigate efficiency of approaches using DataFrame/Dataset and/or approx approaches such as `frequentItems` or Count-Min Sketch (will require an update to CMS to return "heavy-hitters").
        * investigate if we can use metadata to only allow mode for categorical features (or perhaps as an easier alternative, allow mode for only Int/Long columns)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215330198
  
    Documents updated and remove an unit test.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Merged to master. Thanks @hhbyyh and also everyone for reviews.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-212723400
  
    @jkbradley Appreciate if you can take another look, especially for the question from @MLnick in last comment.
    @MLnick I'm not sure if we need to wait for Joseph to make the final decision. Can I know your preference on output column type?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215155656
  
    **[Test build #57140 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57140/consoleFull)** for PR 11601 at commit [`053d489`](https://github.com/apache/spark/commit/053d489a70a28674029ee51a69f529e851261c96).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/66476/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104404339
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the
    +   * feature (relative error less than 0.001).
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols duplicates:" +
    +      s" (${$(inputCols).mkString(", ")})")
    +    require($(outputCols).length == $(outputCols).distinct.length, s"outputCols duplicates:" +
    +      s" (${$(outputCols).mkString(", ")})")
    +    require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" +
    +      s" and outputCols(${$(outputCols).length}) should have the same length")
    +    val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) =>
    +      val inputField = schema(inputCol)
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      StructField(outputCol, inputField.dataType, inputField.nullable)
    +    }
    +    StructType(schema ++ outputFields)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.2.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.2.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.2.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  import org.apache.spark.ml.feature.Imputer._
    --- End diff --
    
    This import should probably be above with the others (or within `fit`)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61162298
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +      case "mode" => filtered.rdd.map(r => r.getDouble(0)).map(d => (d, 1)).reduceByKey(_ + _)
    +        .sortBy(-_._2).first()._1
    +    }
    +    copyValues(new ImputerModel(uid, surrogate).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("2.0.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median", "mode")
    +
    +  @Since("2.0.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogate statistics value for each feature during fitting
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogate: Double)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    val inputType = dataset.select($(inputCol)).schema.fields(0).dataType
    +    inputType match {
    +      case _: NumericType =>
    +        val ic = col($(inputCol)).cast(DoubleType)
    +        dataset.withColumn($(outputCol), when(ic.isNull, surrogate)
    +          .when(ic === $(missingValue), surrogate)
    +          .otherwise(ic)
    +          .cast(inputType))
    --- End diff --
    
    But with option (b), we would not cast back to IntegerType since we'd only accept FloatType or DoubleType inputs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-194184658
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215057260
  
    Thanks for helping review and the helpful discussion. I've read through the comments. Two major changes are:
    1. This initial PR does not include support for "mode" strategy. The preference is to implement "mode" for categorical features (metadata) and use DataFrame API.
    2. We'll support only FloatType and DoubleType for type safety.
    
    Preparing update now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61156296
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +      case "mode" => filtered.rdd.map(r => r.getDouble(0)).map(d => (d, 1)).reduceByKey(_ + _)
    +        .sortBy(-_._2).first()._1
    +    }
    +    copyValues(new ImputerModel(uid, surrogate).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("2.0.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median", "mode")
    +
    +  @Since("2.0.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogate statistics value for each feature during fitting
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogate: Double)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    val inputType = dataset.select($(inputCol)).schema.fields(0).dataType
    +    inputType match {
    +      case _: NumericType =>
    +        val ic = col($(inputCol)).cast(DoubleType)
    +        dataset.withColumn($(outputCol), when(ic.isNull, surrogate)
    +          .when(ic === $(missingValue), surrogate)
    +          .otherwise(ic)
    +          .cast(inputType))
    --- End diff --
    
    Just catching up now... I like the idea of maintaining the input type.  I'm imagining using an Imputer to fill in continuous features with the mean and categoricals with the mode.  Later on, we could even check to see if a column is categorical (in the metadata) and throw an exception for mean.
    
    I'd prefer your option (b) to be safe.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r59977297
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,230 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && !ic.isNaN && ic =!= $(missingValue))
    --- End diff --
    
    Hi @MLnick . The reason I filtered NaN is that the surrogate will always be NaN if missingValue != NaN and the feature contains a NaN value. Now I think it's more reasonable if I just move the filter to the mean case below.
    
    scikit-learn will just discard the column contains NaN when missingValue != NaN
    ```
    >>> a
    matrix([[  1.,   2.],
            [  3.,  nan],
            [  5.,   0.]])
    >>> imputer
    Imputer(axis=0, copy=True, missing_values=0, strategy='mean', verbose=0)
    >>> imputer.fit_transform(a)
    array([[ 1.],
           [ 3.],
           [ 5.]])
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-210944565
  
    **[Test build #56035 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/56035/consoleFull)** for PR 11601 at commit [`1718422`](https://github.com/apache/spark/commit/171842210d3ea2e3c97fe803f0a8bb3831063f3f).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r58830007
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  private[feature] def isMissingValue(value: Double): Boolean = {
    +    val miss = $(missingValue)
    +    value == miss || (value.isNaN && miss.isNaN)
    +  }
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        val doubleRDD = dataset.select($(inputCol)).rdd.map(_.getDouble(0))
    +        Vectors.dense(getColStatistics(doubleRDD))
    +      case _: VectorUDT =>
    +        val filteredDF = dataset.where(s"${$(inputCol)} IS NOT NULL").select($(inputCol))
    +        val vectorRDD = filteredDF.rdd.map(_.getAs[Vector](0)).cache()
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        $(strategy) match {
    +          case "mean" =>
    +            val summary = vectorRDD.treeAggregate((new Array[Double](vl), new Array[Int](vl)))(
    +              (prev, data) => (prev, data) match { case ((mean, count), data) =>
    +                  var i = 0
    +                  while (i < mean.length) {
    +                    if (data(i) != 0 && !data(i).isNaN) {
    +                      count(i) += 1
    +                      mean(i) = mean(i) + (data(i) - mean(i)) / count(i)
    +                    }
    +                    i += 1
    +                  }
    +                  (mean, count)
    +              }, (aggregator1, aggregator2) => (aggregator1, aggregator2) match {
    +                case ((mean1, c1), (mean2, c2)) =>
    +                  (0 until mean1.length).foreach{ i =>
    +                    mean1(i) = mean1(i) + (mean2(i) - mean1(i)) * c2(i) / (c1(i) + c2(i))
    +                    c1(i) += c2(i)
    +                  }
    +                  (mean1, c1)
    +              })
    +            Vectors.dense(summary._1)
    +          case _ =>
    +            val statisticsArray = new Array[Double](vl)
    +            (0 until vl).foreach(i => {
    +              statisticsArray(i) = getColStatistics(vectorRDD.map(v => v(i)))
    +            })
    +            Vectors.dense(statisticsArray)
    +        }
    +    }
    +    copyValues(new ImputerModel(uid, alternate).setParent(this))
    +  }
    +
    +  /** Extract the statistics info from a Double column according to the strategy */
    +  private def getColStatistics(data: RDD[Double]): Double = {
    +    val filteredRDD = data.filter(!isMissingValue(_))
    +    val colStatistics = $(strategy) match {
    +      case "mean" => filteredRDD.mean()
    +      case "median" => filteredRDD.sortBy(d => d).zipWithIndex()
    --- End diff --
    
    @viirya thanks for that!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103870352
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(outputCols).length, "inputCols and outputCols should have" +
    +      "the same length")
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputSchema = schema
    +
    +    $(inputCols).indices.foreach { i =>
    +      val inputCol = localInputCols(i)
    +      val outputCol = localOutputCols(i)
    +      val inputType = schema(inputCol).dataType
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      outputSchema = SchemaUtils.appendColumn(outputSchema, outputCol, inputType)
    +    }
    +    outputSchema
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.1.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue))
    +        .filter(!ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in ${inputCol} are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case "mean" => filtered.select(avg(inputCol)).first().getDouble(0)
    +        case "median" => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001)(0)
    +      }
    +      surrogate.asInstanceOf[Double]
    --- End diff --
    
    is the `asInstanceOf[Double]` necessary here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r55807303
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,290 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    validateParams()
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        Vectors.dense(getColStatistics(dataset, $(inputCol)))
    +      case _: VectorUDT =>
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        val statisticsArray = new Array[Double](vl)
    +        (0 until vl).foreach(i => {
    +          val getI = udf((v: Vector) => v(i))
    +          val tempColName = $(inputCol) + i
    +          val tempData = dataset.where(s"${$(inputCol)} IS NOT NULL")
    +            .select($(inputCol)).withColumn(tempColName, getI(col($(inputCol))))
    +          statisticsArray(i) = getColStatistics(tempData, tempColName)
    +        })
    +        Vectors.dense(statisticsArray)
    +    }
    +    copyValues(new ImputerModel(uid, alternate).setParent(this))
    +  }
    +
    +  /** Extract the statistics info from a Double column according to the strategy */
    +  private def getColStatistics(dataset: DataFrame, colName: String): Double = {
    +    val missValue = $(missingValue) match {
    +      case Double.NaN => "NaN"
    +      case _ => $(missingValue).toString
    +    }
    +    val filteredDF = dataset.select(colName).where(s"$colName != '$missValue'")
    +    val colStatistics = $(strategy) match {
    +      case "mean" =>
    +        filteredDF.selectExpr(s"avg($colName)").first().getDouble(0)
    +      case "median" =>
    --- End diff --
    
    I think we should favour using the new `approxQuantile` sql stat function [here](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala#L66) rather than computing exactly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r57705251
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  private[feature] def isMissingValue(value: Double): Boolean = {
    +    val miss = $(missingValue)
    +    value == miss || (value.isNaN && miss.isNaN)
    +  }
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        val doubleRDD = dataset.select($(inputCol)).rdd.map(_.getDouble(0))
    +        Vectors.dense(getColStatistics(doubleRDD))
    +      case _: VectorUDT =>
    +        val filteredDF = dataset.where(s"${$(inputCol)} IS NOT NULL").select($(inputCol))
    +        val vectorRDD = filteredDF.rdd.map(_.getAs[Vector](0)).cache()
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        $(strategy) match {
    +          case "mean" =>
    +            val summary = vectorRDD.treeAggregate((new Array[Double](vl), new Array[Int](vl)))(
    +              (prev, data) => (prev, data) match { case ((mean, count), data) =>
    +                  var i = 0
    +                  while (i < mean.length) {
    +                    if (data(i) != 0 && !data(i).isNaN) {
    +                      count(i) += 1
    +                      mean(i) = mean(i) + (data(i) - mean(i)) / count(i)
    +                    }
    +                    i += 1
    +                  }
    +                  (mean, count)
    +              }, (aggregator1, aggregator2) => (aggregator1, aggregator2) match {
    +                case ((mean1, c1), (mean2, c2)) =>
    +                  (0 until mean1.length).foreach{ i =>
    +                    mean1(i) = mean1(i) + (mean2(i) - mean1(i)) * c2(i) / (c1(i) + c2(i))
    +                    c1(i) += c2(i)
    +                  }
    +                  (mean1, c1)
    +              })
    +            Vectors.dense(summary._1)
    +          case _ =>
    +            val statisticsArray = new Array[Double](vl)
    +            (0 until vl).foreach(i => {
    +              statisticsArray(i) = getColStatistics(vectorRDD.map(v => v(i)))
    +            })
    +            Vectors.dense(statisticsArray)
    +        }
    +    }
    +    copyValues(new ImputerModel(uid, alternate).setParent(this))
    +  }
    +
    +  /** Extract the statistics info from a Double column according to the strategy */
    +  private def getColStatistics(data: RDD[Double]): Double = {
    +    val filteredRDD = data.filter(!isMissingValue(_))
    +    val colStatistics = $(strategy) match {
    +      case "mean" => filteredRDD.mean()
    +      case "median" => filteredRDD.sortBy(d => d).zipWithIndex()
    --- End diff --
    
    OK. I didn't know it's supporting multiple cols. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61105066
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0, 1.0 )
    +    )).toDF("id", "value", "mean", "median", "mode")
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Double, d2: Double) =>
    +          assert(d1 ~== d2 absTol 1e-5, s"Imputer ut error: $d2 should be $d1")
    +        }
    +    }
    +  }
    +
    +  test("Imputer for Double with missing Value -1.0") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0, 4.0),
    +      (4, -1.0, 2.25, 1.0, 1.0 )
    +    )).toDF("id", "value", "mean", "median", "mode")
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1.0)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Double, d2: Double) =>
    +          assert(d1 ~== d2 absTol 1e-5, s"Imputer ut error: $d2 should be $d1")
    +        }
    +    }
    +  }
    +
    +  test("Imputer for Int with missing Value -1") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1, 1, 1, 1),
    +      (1, 3, 3, 3, 3),
    +      (2, 10, 10, 10, 10),
    +      (3, 10, 10, 10, 10),
    +      (4, -1, 6, 3, 10)
    +    )).toDF("id", "value", "mean", "median", "mode")
    +
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Int, d2: Int) =>
    +          assert(d1 === d2, s"Imputer ut error: $d2 should be $d1")
    +        }
    +    }
    +  }
    +
    +  test("Imputer should impute null") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1, 1, 1, 1),
    +      (1, 3, 3, 3, 3),
    +      (2, 10, 10, 10, 10),
    +      (3, 10, 10, 10, 10),
    +      (4, -1, 6, 3, 10)
    --- End diff --
    
    can we test here a "missing value" as well as null, to see it does the right thing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61158015
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +      case "mode" => filtered.rdd.map(r => r.getDouble(0)).map(d => (d, 1)).reduceByKey(_ + _)
    +        .sortBy(-_._2).first()._1
    +    }
    +    copyValues(new ImputerModel(uid, surrogate).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("2.0.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median", "mode")
    +
    +  @Since("2.0.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogate statistics value for each feature during fitting
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogate: Double)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    val inputType = dataset.select($(inputCol)).schema.fields(0).dataType
    +    inputType match {
    +      case _: NumericType =>
    +        val ic = col($(inputCol)).cast(DoubleType)
    +        dataset.withColumn($(outputCol), when(ic.isNull, surrogate)
    +          .when(ic === $(missingValue), surrogate)
    +          .otherwise(ic)
    +          .cast(inputType))
    --- End diff --
    
    For reference, I checked scikit-learn and the Imputer class returns floats regardless of inputs. I also checked R package "mlr" and it appears to do the same. One concern with b.) would be if the true median was something like 5.0, but approxQuantile returned 4.999999999. Then, we cast back to `IntegerType` and return 4. I wasn't able to produce this situation when I briefly experimented with it, and also the median is already approximate, so I'm not sure if this is really a problem. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Hi @MLnick I changed the surrogateDF format  for better extensibility in the last update and added unit tests for multi-column support. Let me know if I miss anything.
    
    inputCol1|inputCol2
    -------------|------------
    surrogate1 | surrogate2



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Thanks for the comments @MLnick @jkbradley @sethah 
    I have sent update according to the comments and change `ImputerModel.surrogate` and persistence format into DataFrame. 
    
    As for the multiple columns support, do we need to assemble the imputed column together into one output column or should we support multiple output columns?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r58715151
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  private[feature] def isMissingValue(value: Double): Boolean = {
    +    val miss = $(missingValue)
    +    value == miss || (value.isNaN && miss.isNaN)
    +  }
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        val doubleRDD = dataset.select($(inputCol)).rdd.map(_.getDouble(0))
    +        Vectors.dense(getColStatistics(doubleRDD))
    +      case _: VectorUDT =>
    +        val filteredDF = dataset.where(s"${$(inputCol)} IS NOT NULL").select($(inputCol))
    +        val vectorRDD = filteredDF.rdd.map(_.getAs[Vector](0)).cache()
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        $(strategy) match {
    +          case "mean" =>
    +            val summary = vectorRDD.treeAggregate((new Array[Double](vl), new Array[Int](vl)))(
    +              (prev, data) => (prev, data) match { case ((mean, count), data) =>
    +                  var i = 0
    +                  while (i < mean.length) {
    +                    if (data(i) != 0 && !data(i).isNaN) {
    +                      count(i) += 1
    +                      mean(i) = mean(i) + (data(i) - mean(i)) / count(i)
    +                    }
    +                    i += 1
    +                  }
    +                  (mean, count)
    +              }, (aggregator1, aggregator2) => (aggregator1, aggregator2) match {
    +                case ((mean1, c1), (mean2, c2)) =>
    +                  (0 until mean1.length).foreach{ i =>
    +                    mean1(i) = mean1(i) + (mean2(i) - mean1(i)) * c2(i) / (c1(i) + c2(i))
    +                    c1(i) += c2(i)
    +                  }
    +                  (mean1, c1)
    +              })
    +            Vectors.dense(summary._1)
    +          case _ =>
    +            val statisticsArray = new Array[Double](vl)
    +            (0 until vl).foreach(i => {
    +              statisticsArray(i) = getColStatistics(vectorRDD.map(v => v(i)))
    +            })
    +            Vectors.dense(statisticsArray)
    +        }
    +    }
    +    copyValues(new ImputerModel(uid, alternate).setParent(this))
    +  }
    +
    +  /** Extract the statistics info from a Double column according to the strategy */
    +  private def getColStatistics(data: RDD[Double]): Double = {
    +    val filteredRDD = data.filter(!isMissingValue(_))
    +    val colStatistics = $(strategy) match {
    +      case "mean" => filteredRDD.mean()
    +      case "median" => filteredRDD.sortBy(d => d).zipWithIndex()
    --- End diff --
    
    @MLnick I submitted a PR to open the API at #12207.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-216462512
  
    @hhbyyh made a small doc comment too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-208928621
  
    **[Test build #55612 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/55612/consoleFull)** for PR 11601 at commit [`4bdf595`](https://github.com/apache/spark/commit/4bdf595f576fae76f710bfb21e1e3f71571c55c8).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-216485692
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104407859
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,166 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.{DataFrame, Row}
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 4.0, 1.0, 1.0, 4.0, 4.0),
    +      (1, 11.0, 12.0, 11.0, 11.0, 12.0, 12.0),
    +      (2, 3.0, Double.NaN, 3.0, 3.0, 10.0, 12.0),
    +      (3, Double.NaN, 14.0, 5.0, 3.0, 14.0, 14.0)
    +    )).toDF("id", "value1", "value2", "expected_mean_value1", "expected_median_value1",
    +      "expected_mean_value2", "expected_median_value2")
    +    val imputer = new Imputer()
    +      .setInputCols(Array("value1", "value2"))
    +      .setOutputCols(Array("out1", "out2"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should handle NaNs when computing surrogate value, if missingValue is not NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1.0)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer for Float with missing Value -1.0") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0F, 1.0F, 1.0F),
    +      (1, 3.0F, 3.0F, 3.0F),
    +      (2, 10.0F, 10.0F, 10.0F),
    +      (3, 10.0F, 10.0F, 10.0F),
    +      (4, -1.0F, 6.0F, 3.0F)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should impute null as well as 'missingValue'") {
    +    val rawDf = spark.createDataFrame( Seq(
    +      (0, 4.0, 4.0, 4.0),
    +      (1, 10.0, 10.0, 10.0),
    +      (2, 10.0, 10.0, 10.0),
    +      (3, Double.NaN, 8.0, 10.0),
    +      (4, -1.0, 8.0, 10.0)
    +    )).toDF("id", "rawValue", "expected_mean_value", "expected_median_value")
    +    val df = rawDf.selectExpr("*", "IF(rawValue=-1.0, null, rawValue) as value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer throws exception when surrogate cannot be computed") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, Double.NaN, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +        .setStrategy(strategy)
    +      intercept[SparkException] {
    +        val model = imputer.fit(df)
    +      }
    +    }
    +  }
    +
    +  test("Imputer throws exception when inputCols does not match outputCols") {
    --- End diff --
    
    Maybe call the test "Imputer input & output column validation" as it covers more than testing matching lengths.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104889721
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,259 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols contains" +
    +      s" duplicates: (${$(inputCols).mkString(", ")})")
    +    require($(outputCols).length == $(outputCols).distinct.length, s"outputCols contains" +
    +      s" duplicates: (${$(outputCols).mkString(", ")})")
    +    require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" +
    +      s" and outputCols(${$(outputCols).length}) should have the same length")
    +    val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) =>
    +      val inputField = schema(inputCol)
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      StructField(outputCol, inputField.dataType, inputField.nullable)
    +    }
    +    StructType(schema ++ outputFields)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed. For
    + * computing median, DataFrameStatFunctions.approxQuantile is used with a relative error of 0.001.
    + */
    +@Experimental
    +class Imputer @Since("2.2.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.2.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.2.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> Imputer.mean, missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val spark = dataset.sparkSession
    +    import spark.implicits._
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue) && !ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in $inputCol are Null, Nan or missingValue ($missingValue)")
    --- End diff --
    
    `($missingValue)` -> `${$(missingValue)}`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Created [SPARK-19969](https://issues.apache.org/jira/browse/SPARK-19969) to track doc and examples to be done for 2.2 release. I can help with this if you're tied up.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    By the way out of curiosity, I tested things out on a cluster (4x workers, 192 cores & 480GB RAM total), with 100 columns of 100 million doubles each, 1% `NaN` occurrence. Reading from a Parquet file.
    
    _not cached_
    `fit` takes about 1.5 seconds per column (150 secs), while `transform` takes 50 secs.
    
    _cached_
    `fit`: 15 sec; `transform`: 16 sec.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r55555741
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, Params}
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean along the axis.
    +    * If "median", then replace missing values using the median along the axis.
    +    * If "most", then replace missing using the most frequent value along the axis.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean along the axis." +
    +    "If median, then replace missing values using the median along the axis." +
    +    "If most, then replace missing using the most frequent value along the axis.")
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingvalues will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingvalues will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    validateParams()
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +  override def validateParams(): Unit = {
    +    require(Seq("mean", "median", "most").contains($(strategy)),
    +      s"${$(strategy)} is not supported. Options are mean, median and most")
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values encodings.
    + *
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /** @group setParam */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        val colStatistics = getColStatistics(dataset, $(inputCol))
    +        Vectors.dense(Array(colStatistics))
    --- End diff --
    
    Perhaps `Vectors.dense(colStatistics)` is cleaner?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-195020256
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/52842/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-194432841
  
    Looking at the Jiras, it is unclear if any concrete decisions were made regarding handling Vectors and how NaN values should be handled in colStats. Is there any update?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r80543997
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("expected_" + strategy, "out").collect().foreach {
    +       case Row(exp: Double, out: Double) =>
    +          assert(exp ~== out absTol 1e-5, s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    --- End diff --
    
    Yeah, actually this also fails if the entire input column is the missing value as well. We need to beef up the test suite :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    jenkins retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73753/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61315937
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +      case "mode" => filtered.rdd.map(r => r.getDouble(0)).map(d => (d, 1)).reduceByKey(_ + _)
    --- End diff --
    
    We also have Count-min Sketch in Spark as a SQL function, but it doesn't return the "heavy-hitters" (only the approx frequency for a given item). I think it could be made to support "heavy-hitteres" aka top-k so one could use it for approx mode at some point.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61104774
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +      case "mode" => filtered.rdd.map(r => r.getDouble(0)).map(d => (d, 1)).reduceByKey(_ + _)
    --- End diff --
    
    I'm still a little concerned about performance implications here on large datasets... and I can't think of a real use case for "mode". I would argue that almost always users would choose to fill in the "mean" or "median" value. And if we add more functionality later, it is likely to be more complex versions of the mean/median (e.g. there are [R packages](http://www.analyticsvidhya.com/blog/2016/03/tutorial-powerful-packages-imputing-missing-values/) that have some interesting methods).
    
    I propose we remove "mode" from this initial implementation. Let's see how users use this Imputer in practice and if anyone actually wants the "mode" option. @jkbradley @sethah @holdenk thoughts?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215503334
  
    Other than a couple minor comments, this LGTM pending tests.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #74651 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74651/testReport)** for PR 11601 at commit [`c67afc1`](https://github.com/apache/spark/commit/c67afc11e1fee58b65da67bf3e25e5245f72280d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61105305
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0, 1.0 )
    +    )).toDF("id", "value", "mean", "median", "mode")
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Double, d2: Double) =>
    +          assert(d1 ~== d2 absTol 1e-5, s"Imputer ut error: $d2 should be $d1")
    +        }
    +    }
    +  }
    +
    +  test("Imputer for Double with missing Value -1.0") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0, 4.0),
    +      (4, -1.0, 2.25, 1.0, 1.0 )
    +    )).toDF("id", "value", "mean", "median", "mode")
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1.0)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Double, d2: Double) =>
    +          assert(d1 ~== d2 absTol 1e-5, s"Imputer ut error: $d2 should be $d1")
    +        }
    +    }
    +  }
    +
    +  test("Imputer for Int with missing Value -1") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1, 1, 1, 1),
    +      (1, 3, 3, 3, 3),
    +      (2, 10, 10, 10, 10),
    +      (3, 10, 10, 10, 10),
    +      (4, -1, 6, 3, 10)
    +    )).toDF("id", "value", "mean", "median", "mode")
    +
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Int, d2: Int) =>
    +          assert(d1 === d2, s"Imputer ut error: $d2 should be $d1")
    +        }
    +    }
    +  }
    +
    --- End diff --
    
    we should also have a test for a `non-NaN` missing value, but with `NaN` in the dataset, to check that "mean" and "median" behave as we expect.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61373605
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean") or the
    + * median("median") of the column in which the missing values are located.
    + *
    + * Note that all the null values will be imputed as well.
    --- End diff --
    
    Yes. That's better.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61308682
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean") or the
    + * median("median") of the column in which the missing values are located.
    + *
    --- End diff --
    
    Can we document that we only support "Float" and "Double" types for now?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104407603
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the
    +   * feature (relative error less than 0.001).
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols duplicates:" +
    +      s" (${$(inputCols).mkString(", ")})")
    +    require($(outputCols).length == $(outputCols).distinct.length, s"outputCols duplicates:" +
    --- End diff --
    
    perhaps `... outputCols contains duplicates ...`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-200333349
  
    **[Test build #53931 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53931/consoleFull)** for PR 11601 at commit [`72d104d`](https://github.com/apache/spark/commit/72d104d92a96ba03d60a72a5fa0b06e583a28bdc).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61157367
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +      case "mode" => filtered.rdd.map(r => r.getDouble(0)).map(d => (d, 1)).reduceByKey(_ + _)
    --- End diff --
    
    I agree that mode is important mainly for categorical features.  I'd like to support it.  I'd recommend doing this computation in the DataFrame API.  You could do it in a straightforward manner with a count, or you could do something fancy using an approximate algorithm like freqItems.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61305806
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,141 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0)
    +    )).toDF("id", "value", "exp_mean", "exp_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("exp_" + strategy, "out").collect().foreach {
    +       case Row(exp: Double, out: Double) =>
    +          assert(exp ~== out absTol 1e-5, s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer for Double with missing Value -1.0") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, -1.0, 2.25, 1.0)
    +    )).toDF("id", "value", "exp_mean", "exp_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1.0)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("exp_" + strategy, "out").collect().foreach {
    +        case Row(exp: Double, out: Double) =>
    +          assert(exp ~== out absTol 1e-5, s"Impute($strategy) error. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer for Double with missing Value -1.0 and contains NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    --- End diff --
    
    Does approx quantile always choose the greater of the two middle values as the median? If so, can we add a comment noting that? NumPy computes the median of `[1.0, 3.0]` exactly as `2.0`. Future developers might think it's a mistake.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r59918234
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,230 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && !ic.isNaN && ic =!= $(missingValue))
    --- End diff --
    
    I'm not sure that we want to handle `NaN` regardless of the value of `missingValue`. At least, I think the scikit-learn impl only handles `NaN` if the missing value is set to `NaN` (@hhbyyh can you double check that? Also what do R packages do?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-194599332
  
    I prefer to keep Statistics.colStats(rdd) unchanged for now. As ut in this PR suggests, we can cover Double and Vector for now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r80644244
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    --- End diff --
    
    I've heard an argument that everything in the class is implicitly since 2.1.0 since the class itself is - unless otherwise stated. Which does make sense. But I do slightly favour being explicit about it (even if it is a bit pedantic) so yeah let's add the annotation to all the setters.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r58681145
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  private[feature] def isMissingValue(value: Double): Boolean = {
    +    val miss = $(missingValue)
    +    value == miss || (value.isNaN && miss.isNaN)
    +  }
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        val doubleRDD = dataset.select($(inputCol)).rdd.map(_.getDouble(0))
    +        Vectors.dense(getColStatistics(doubleRDD))
    +      case _: VectorUDT =>
    +        val filteredDF = dataset.where(s"${$(inputCol)} IS NOT NULL").select($(inputCol))
    +        val vectorRDD = filteredDF.rdd.map(_.getAs[Vector](0)).cache()
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        $(strategy) match {
    +          case "mean" =>
    +            val summary = vectorRDD.treeAggregate((new Array[Double](vl), new Array[Int](vl)))(
    +              (prev, data) => (prev, data) match { case ((mean, count), data) =>
    +                  var i = 0
    +                  while (i < mean.length) {
    +                    if (data(i) != 0 && !data(i).isNaN) {
    +                      count(i) += 1
    +                      mean(i) = mean(i) + (data(i) - mean(i)) / count(i)
    +                    }
    +                    i += 1
    +                  }
    +                  (mean, count)
    +              }, (aggregator1, aggregator2) => (aggregator1, aggregator2) match {
    +                case ((mean1, c1), (mean2, c2)) =>
    +                  (0 until mean1.length).foreach{ i =>
    +                    mean1(i) = mean1(i) + (mean2(i) - mean1(i)) * c2(i) / (c1(i) + c2(i))
    +                    c1(i) += c2(i)
    +                  }
    +                  (mean1, c1)
    +              })
    +            Vectors.dense(summary._1)
    +          case _ =>
    +            val statisticsArray = new Array[Double](vl)
    +            (0 until vl).foreach(i => {
    +              statisticsArray(i) = getColStatistics(vectorRDD.map(v => v(i)))
    +            })
    +            Vectors.dense(statisticsArray)
    +        }
    +    }
    +    copyValues(new ImputerModel(uid, alternate).setParent(this))
    +  }
    +
    +  /** Extract the statistics info from a Double column according to the strategy */
    +  private def getColStatistics(data: RDD[Double]): Double = {
    +    val filteredRDD = data.filter(!isMissingValue(_))
    +    val colStatistics = $(strategy) match {
    +      case "mean" => filteredRDD.mean()
    +      case "median" => filteredRDD.sortBy(d => d).zipWithIndex()
    --- End diff --
    
    @MLnick I think we just don't consider multiple cols use case? We should add the API. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-200333490
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53931/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-210947568
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/56035/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    @hhbyyh seems behavior of approx quantiles may have changed somewhere? Can you take a look into it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61575765
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,122 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("expected_" + strategy, "out").collect().foreach {
    +       case Row(exp: Double, out: Double) =>
    +          assert(exp ~== out absTol 1e-5, s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer for Double with missing Value -1.0 and contains NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1.0)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("expected_" + strategy, "out").collect().foreach {
    +        case Row(exp: Double, out: Double) =>
    +          assert((exp.isNaN && out.isNaN) || (exp ~== out absTol 1e-5),
    +            s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer for Float with missing Value -1.0") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0F, 1.0F, 1.0F),
    +      (1, 3.0F, 3.0F, 3.0F),
    +      (2, 10.0F, 10.0F, 10.0F),
    +      (3, 10.0F, 10.0F, 10.0F),
    +      (4, -1.0F, 6.0F, 3.0F)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1)
    +      val model = imputer.fit(df)
    +      val result = model.transform(df)
    +      model.transform(df).select("expected_" + strategy, "out").collect().foreach {
    +        case Row(exp: Float, out: Float) =>
    +          assert(exp == out, s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer should impute null") {
    --- End diff --
    
    Could we call this `Imputer should impute null as well as 'missingValue'` to clarify


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #65053 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65053/consoleFull)** for PR 11601 at commit [`544a65c`](https://github.com/apache/spark/commit/544a65c82a7d921bdff73998e8b350e11b51dcbe).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    So, I am trying to refresh my memory on this PR. I see we settled on not supporting vector type and not supporting mode. Did we ever settle on supporting multiple input columns? I am not sure I see the practicality of this if we don't. I wonder how many use cases fit the mold of imputing a single column?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #73868 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73868/testReport)** for PR 11601 at commit [`41d91b9`](https://github.com/apache/spark/commit/41d91b9ef855a611016c9a9613942e578ff599dd).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class Imputer @Since(\"2.2.0\")(override val uid: String)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/66516/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215155949
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/57140/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73868/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61318084
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean") or the
    + * median("median") of the column in which the missing values are located.
    --- End diff --
    
    can we add something saying how we impute, i.e. "either using the mean or median of the input column, computed after filtering out missing values"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215615467
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/57298/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-216435795
  
    @MLnick @jkbradley Is it possible to make this into 2.0? Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61286179
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    --- End diff --
    
    I get the point, yet data in the format of 
    ```
        val df = sqlContext.createDataFrame( Seq(
          (0, 1.0, 1.0, 1.0),
          (1, 1.0, 1.0, 1.0),
          (2, 3.0, 3.0, 3.0),
          (3, 4.0, 4.0, 4.0),
          (4, Double.NaN, 2.25, 1.0)
        )).toDF("id", "value", "exp_mean", "exp_median")
    ```
    provides direct correspondence.
    
    I've updated the columns names.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-208932083
  
    **[Test build #55612 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/55612/consoleFull)** for PR 11601 at commit [`4bdf595`](https://github.com/apache/spark/commit/4bdf595f576fae76f710bfb21e1e3f71571c55c8).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-200275414
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53923/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74651/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-194184517
  
    **[Test build #52734 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52734/consoleFull)** for PR 11601 at commit [`1b39668`](https://github.com/apache/spark/commit/1b3966800982fa980307d1b6ded6e28e5f5985e8).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104280679
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,136 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.{DataFrame, Row}
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should handle NaNs when computing surrogate value, if missingValue is not NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1.0)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer for Float with missing Value -1.0") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0F, 1.0F, 1.0F),
    +      (1, 3.0F, 3.0F, 3.0F),
    +      (2, 10.0F, 10.0F, 10.0F),
    +      (3, 10.0F, 10.0F, 10.0F),
    +      (4, -1.0F, 6.0F, 3.0F)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should impute null as well as 'missingValue'") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 4.0, 4.0, 4.0),
    +      (1, 10.0, 10.0, 10.0),
    +      (2, 10.0, 10.0, 10.0),
    +      (3, Double.NaN, 8.0, 10.0),
    +      (4, -1.0, 8.0, 10.0)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    val df2 = df.selectExpr("*", "IF(value=-1.0, null, value) as nullable_value")
    +    val imputer = new Imputer().setInputCols(Array("nullable_value")).setOutputCols(Array("out"))
    +    ImputerSuite.iterateStrategyTest(imputer, df2)
    +  }
    +
    +
    +  test("Imputer throws exception when surrogate cannot be computed") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, Double.NaN, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value", "expected_mean", "expected_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +        .setStrategy(strategy)
    +      intercept[SparkException] {
    +        val model = imputer.fit(df)
    +      }
    +    }
    +  }
    +
    +  test("Imputer read/write") {
    +    val t = new Imputer()
    +      .setInputCols(Array("myInputCol"))
    +      .setOutputCols(Array("myOutputCol"))
    +      .setMissingValue(-1.0)
    +    testDefaultReadWrite(t)
    +  }
    +
    +  test("ImputerModel read/write") {
    +    val spark = this.spark
    +    import spark.implicits._
    +    val surrogateDF = Seq(1.234).toDF("myInputCol")
    --- End diff --
    
    this happens to be the correct column name for now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-195019793
  
    **[Test build #52842 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52842/consoleFull)** for PR 11601 at commit [`4e45f81`](https://github.com/apache/spark/commit/4e45f81f89f0b1ad13add524b3dd89fe52126bc0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215615452
  
    **[Test build #57298 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57298/consoleFull)** for PR 11601 at commit [`aef094b`](https://github.com/apache/spark/commit/aef094bc7b7a00c0ded1b2998b7f98d2bc42c666).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-210947567
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61318222
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean") or the
    --- End diff --
    
    I don't think it's necessary to say mean ("mean") and median ("median") here. Just "mean or the median" will suffice.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215975928
  
    Thanks for the review @MLnick. I'll create the jira now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r80593819
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    --- End diff --
    
    Shall we add Since annotations for the setters?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-216011066
  
    **[Test build #57459 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57459/consoleFull)** for PR 11601 at commit [`cca8dd4`](https://github.com/apache/spark/commit/cca8dd41714d79476c2bf23f706012a282c53bcb).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104407571
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the
    +   * feature (relative error less than 0.001).
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols duplicates:" +
    --- End diff --
    
    perhaps `... inputCols contains duplicates ...`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-207904347
  
    +1 @MLnick Given the performance concern on "median" and "most" for vector type, I agree with your suggestion. I'll prepare the update.
    @jkbradley Let me know if you have any concern.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215335486
  
    **[Test build #57229 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57229/consoleFull)** for PR 11601 at commit [`4e1c34a`](https://github.com/apache/spark/commit/4e1c34a77b8e4382c00a0438e00e34f544b591a3).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-208932139
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/55612/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61365096
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,141 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0)
    +    )).toDF("id", "value", "exp_mean", "exp_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("exp_" + strategy, "out").collect().foreach {
    +       case Row(exp: Double, out: Double) =>
    +          assert(exp ~== out absTol 1e-5, s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer for Double with missing Value -1.0") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, -1.0, 2.25, 1.0)
    +    )).toDF("id", "value", "exp_mean", "exp_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +        .setMissingValue(-1.0)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("exp_" + strategy, "out").collect().foreach {
    +        case Row(exp: Double, out: Double) =>
    +          assert(exp ~== out absTol 1e-5, s"Impute($strategy) error. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer for Double with missing Value -1.0 and contains NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    --- End diff --
    
    Actually, it always choose the smaller one of the two middle values as I saw in some tests.
    In this test case, the median is computed from [1, 3, Double.NaN]. And Double.NaN is treated as it's greater than Double.MaValue.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61308835
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    --- End diff --
    
    unused import


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215330151
  
    **[Test build #57229 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57229/consoleFull)** for PR 11601 at commit [`4e1c34a`](https://github.com/apache/spark/commit/4e1c34a77b8e4382c00a0438e00e34f544b591a3).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103867980
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    --- End diff --
    
    Fix comment indentation here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    I agree we should plan to support multiple columns and Vector columns in the future.  The 2 places I noticed may cause problems in the future are:
    * ```ImputerModel.surrogate```: This is nice to expose, but it could be a problem in the future if we need to return a Vector of surrogates (for a Vector input), or even an array of surrogates (for a list of input columns).  I'm Ok with leaving it as-is for now.  When we generalize it, we can throw a runtime exception when the surrogate is a Vector or other type.
    * Persistence format


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r59917508
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,230 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && !ic.isNaN && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +      case "mode" => filtered.rdd.map(r => r.getDouble(0)).map(d => (d, 1)).reduceByKey(_ + _)
    +        .sortBy(-_._2).first()._1
    +    }
    +    copyValues(new ImputerModel(uid, surrogate).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("2.0.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median", "mode")
    +
    +  @Since("2.0.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogate statistics value for each feature during fitting
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogate: Double)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    val inputType = dataset.select($(inputCol)).schema.fields(0).dataType
    +    inputType match {
    +      case _: NumericType =>
    +        val ic = col($(inputCol)).cast(DoubleType)
    +        dataset.withColumn($(outputCol), when(ic.isNull, surrogate)
    +          .when(ic.isNaN, surrogate)
    +          .when(ic === $(missingValue), surrogate)
    +          .otherwise(ic)
    +          .cast(inputType))
    --- End diff --
    
    @jkbradley what do you think about handling different numeric types in input/output columns? If the input is say `IntType`, then strategy `mode` and`median` is ok but `mean` is somewhat problematic - or are we ok with rounding to and `Int`? The alternative is the Imputer always appends a `Double` output column.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104407441
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,166 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.{DataFrame, Row}
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 4.0, 1.0, 1.0, 4.0, 4.0),
    +      (1, 11.0, 12.0, 11.0, 11.0, 12.0, 12.0),
    +      (2, 3.0, Double.NaN, 3.0, 3.0, 10.0, 12.0),
    +      (3, Double.NaN, 14.0, 5.0, 3.0, 14.0, 14.0)
    +    )).toDF("id", "value1", "value2", "expected_mean_value1", "expected_median_value1",
    +      "expected_mean_value2", "expected_median_value2")
    +    val imputer = new Imputer()
    +      .setInputCols(Array("value1", "value2"))
    +      .setOutputCols(Array("out1", "out2"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should handle NaNs when computing surrogate value, if missingValue is not NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1.0)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer for Float with missing Value -1.0") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0F, 1.0F, 1.0F),
    +      (1, 3.0F, 3.0F, 3.0F),
    +      (2, 10.0F, 10.0F, 10.0F),
    +      (3, 10.0F, 10.0F, 10.0F),
    +      (4, -1.0F, 6.0F, 3.0F)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should impute null as well as 'missingValue'") {
    +    val rawDf = spark.createDataFrame( Seq(
    +      (0, 4.0, 4.0, 4.0),
    +      (1, 10.0, 10.0, 10.0),
    +      (2, 10.0, 10.0, 10.0),
    +      (3, Double.NaN, 8.0, 10.0),
    +      (4, -1.0, 8.0, 10.0)
    +    )).toDF("id", "rawValue", "expected_mean_value", "expected_median_value")
    +    val df = rawDf.selectExpr("*", "IF(rawValue=-1.0, null, rawValue) as value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer throws exception when surrogate cannot be computed") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, Double.NaN, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +        .setStrategy(strategy)
    +      intercept[SparkException] {
    +        val model = imputer.fit(df)
    +      }
    +    }
    +  }
    +
    +  test("Imputer throws exception when inputCols does not match outputCols") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value1", "value2", "value3")
    +    Seq("mean", "median").foreach { strategy =>
    +      // inputCols and outCols length different
    +      val imputer = new Imputer()
    +        .setInputCols(Array("value1", "value2"))
    +        .setOutputCols(Array("out1"))
    +        .setStrategy(strategy)
    +      intercept[IllegalArgumentException] {
    +        val model = imputer.fit(df)
    +      }
    +      // duplicate name in inputCols
    --- End diff --
    
    Should we test for the exception for duplicate out cols too? We should probably check the thrown message with an `assert`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    @hhbyyh could you update the since annotations to target `2.1.0`? @jkbradley if you have a chance to review would be great. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Sorry for the delay - been a bit tied up! Overall looks good. Will leave open a day or two for @sethah or @jkbradley make any final comment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r80644112
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +    }
    +    copyValues(new ImputerModel(uid, surrogate).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("2.1.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median")
    +
    +  @Since("2.1.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogate Value by which missing values in the input column will be replaced.
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogate: Double)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    val inputType = dataset.select($(inputCol)).schema.fields(0).dataType
    +    val ic = col($(inputCol))
    +    dataset.withColumn($(outputCol), when(ic.isNull, surrogate)
    +      .when(ic === $(missingValue), surrogate)
    +      .otherwise(ic)
    +      .cast(inputType))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): ImputerModel = {
    +    val copied = new ImputerModel(uid, surrogate)
    +    copyValues(copied, extra).setParent(parent)
    +  }
    +
    +  @Since("2.1.0")
    +  override def write: MLWriter = new ImputerModelWriter(this)
    +}
    +
    +
    +@Since("2.1.0")
    +object ImputerModel extends MLReadable[ImputerModel] {
    +
    +  private[ImputerModel] class ImputerModelWriter(instance: ImputerModel) extends MLWriter {
    +
    +    private case class Data(surrogate: Double)
    --- End diff --
    
    I would think that if we support multiple columns, we need to match up the column name to the surrogate, correct? So I'd think we would want to save a DF with the same columns as `inputCol(s)` and then yes either double or vector type. Is this what you mean here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104638806
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the
    +   * feature (relative error less than 0.001).
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols duplicates:" +
    +      s" (${$(inputCols).mkString(", ")})")
    +    require($(outputCols).length == $(outputCols).distinct.length, s"outputCols duplicates:" +
    +      s" (${$(outputCols).mkString(", ")})")
    +    require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" +
    +      s" and outputCols(${$(outputCols).length}) should have the same length")
    +    val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) =>
    +      val inputField = schema(inputCol)
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      StructField(outputCol, inputField.dataType, inputField.nullable)
    +    }
    +    StructType(schema ++ outputFields)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    --- End diff --
    
    Ah right - perhaps just mention using approxQuantile?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-216485606
  
    **[Test build #57620 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57620/consoleFull)** for PR 11601 at commit [`4e07431`](https://github.com/apache/spark/commit/4e0743139796ac53df2554cfa53736b8035bae15).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r55619789
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, Params}
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean along the axis.
    +    * If "median", then replace missing values using the median along the axis.
    +    * If "most", then replace missing using the most frequent value along the axis.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean along the axis." +
    +    "If median, then replace missing values using the median along the axis." +
    +    "If most, then replace missing using the most frequent value along the axis.")
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingvalues will be imputed.
    --- End diff --
    
    This should be "missingValue". I'll change it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r80592740
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    --- End diff --
    
    Doc: Note that null values are always treated as missing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #73753 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73753/testReport)** for PR 11601 at commit [`e86d919`](https://github.com/apache/spark/commit/e86d9198c65c3b289b091150b52708deda37f090).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-208732599
  
    @MLnick @jkbradley updates:
    1. limit the input type to numeric only.
    2. support imputing for null, with ut.
    3. use approxQuantile for median.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215143141
  
    **[Test build #57140 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57140/consoleFull)** for PR 11601 at commit [`053d489`](https://github.com/apache/spark/commit/053d489a70a28674029ee51a69f529e851261c96).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103888377
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(outputCols).length, "inputCols and outputCols should have" +
    +      "the same length")
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputSchema = schema
    +
    +    $(inputCols).indices.foreach { i =>
    +      val inputCol = localInputCols(i)
    +      val outputCol = localOutputCols(i)
    +      val inputType = schema(inputCol).dataType
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      outputSchema = SchemaUtils.appendColumn(outputSchema, outputCol, inputType)
    +    }
    +    outputSchema
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.1.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue))
    +        .filter(!ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in ${inputCol} are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case "mean" => filtered.select(avg(inputCol)).first().getDouble(0)
    +        case "median" => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001)(0)
    +      }
    +      surrogate.asInstanceOf[Double]
    +    }
    +
    +    import dataset.sparkSession.implicits._
    +    val surrogateDF = Seq(surrogates).toDF("surrogates")
    +    copyValues(new ImputerModel(uid, surrogateDF).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = defaultCopy(extra)
    +}
    +
    +@Since("2.1.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median")
    +
    +  @Since("2.1.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogateDF Value by which missing values in the input columns will be replaced. This
    + *    is stored using DataFrame with input column names and the corresponding surrogates.
    --- End diff --
    
    It _sounds_ like you had the idea of storing the surrogates something like:
    
    ```
    +------+---------+
    |column|surrogate|
    +------+---------+
    |  col1|      1.2|
    |  col2|      3.4|
    |  col3|      5.4|
    +------+---------+
    ```
    
    ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61099076
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    --- End diff --
    
    likewise, we can use `SchemaUtils.appendColumn` here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r55560834
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, Params}
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean along the axis.
    +    * If "median", then replace missing values using the median along the axis.
    +    * If "most", then replace missing using the most frequent value along the axis.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean along the axis." +
    +    "If median, then replace missing values using the median along the axis." +
    +    "If most, then replace missing using the most frequent value along the axis.")
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingvalues will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingvalues will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    validateParams()
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +  override def validateParams(): Unit = {
    +    require(Seq("mean", "median", "most").contains($(strategy)),
    +      s"${$(strategy)} is not supported. Options are mean, median and most")
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values encodings.
    + *
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /** @group setParam */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        val colStatistics = getColStatistics(dataset, $(inputCol))
    +        Vectors.dense(Array(colStatistics))
    +      case _: VectorUDT =>
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        val statisticsArray = new Array[Double](vl)
    +        (0 until vl).foreach(i => {
    +          val getI = udf((v: Vector) => v(i))
    +          val tempColName = $(inputCol) + i
    +          val tempData = dataset.where(s"${$(inputCol)} is not null")
    +            .select($(inputCol)).withColumn(tempColName, getI(col($(inputCol))))
    +          statisticsArray(i) = getColStatistics(tempData, tempColName)
    +        })
    +        Vectors.dense(statisticsArray)
    +    }
    +    copyValues(new ImputerModel(uid, alternate).setParent(this))
    +  }
    +
    +  private def getColStatistics(dataset: DataFrame, colName: String): Double = {
    +    val missValue = $(missingValue) match {
    +      case Double.NaN => "NaN"
    +      case _ => $(missingValue).toString
    +    }
    +    val filteredDF = dataset.select(colName).where(s"$colName != '$missValue'")
    +    val colStatistics = $(strategy) match {
    +      case "mean" =>
    +        filteredDF.selectExpr(s"avg($colName)").first().getDouble(0)
    +      case "median" =>
    +        // TODO: optimize the sort with quick-select or Percentile(Hive) if required
    +        val rddDouble = filteredDF.rdd.map(_.getDouble(0))
    +        rddDouble.sortBy(d => d).zipWithIndex().map {
    +          case (v, idx) => (idx, v)
    +        }.lookup(rddDouble.count()/2).head
    +      case "most" =>
    +        val input = filteredDF.rdd.map(_.getDouble(0))
    +        val most = input.map(d => (d, 1)).reduceByKey(_ + _).sortBy(-_._2).first()._1
    +        most
    +    }
    +    colStatistics
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("1.6.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  @Since("1.6.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    +  * :: Experimental ::
    +  * Model fitted by [[Imputer]].
    +  *
    +  * @param alternate statistics value for each original column during fitting
    +  */
    +@Experimental
    +class ImputerModel private[ml] (
    +    override val uid: String,
    +    val alternate: Vector)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  private def matchMissingValue(value: Double): Boolean = {
    --- End diff --
    
    IMO `isMissingValue` makes the code easier to understand.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215143854
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/57139/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215715233
  
    A few small final comments, otherwise LGTM. @jkbradley will leave open for a day or two in case you want to take a final pass.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r58922597
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    --- End diff --
    
    indentation errors here and elsewhere


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-212188252
  
    Updated according to the comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61577109
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the
    + * median of the column in which the missing values are located. InputCol should be
    --- End diff --
    
    I prefer "The input column" rather than "InputCol".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-209714739
  
    Hope it helps @hash-X .
    @MLnick @jkbradley Appreciate if you can take a look.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61286207
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,121 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0, 1.0 )
    +    )).toDF("id", "value", "mean", "median", "mode")
    +    Seq("mean", "median", "mode").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select(strategy, "out").collect()
    +        .foreach { case Row(d1: Double, d2: Double) =>
    +          assert(d1 ~== d2 absTol 1e-5, s"Imputer ut error: $d2 should be $d1")
    --- End diff --
    
    sure, changed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r80594424
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +    }
    +    copyValues(new ImputerModel(uid, surrogate).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("2.1.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median")
    +
    +  @Since("2.1.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogate Value by which missing values in the input column will be replaced.
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogate: Double)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    val inputType = dataset.select($(inputCol)).schema.fields(0).dataType
    --- End diff --
    
    simplify: ```dataset.schema($(inputCol)).dataType```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61373554
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean") or the
    + * median("median") of the column in which the missing values are located.
    --- End diff --
    
    @sethah, I tried yet I'm afraid it draws more confusion than the help. After all, the current behavior is not out of expectation and works for most ( if not all) users. I'd prefer to skip the detailed explanation in API document. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #73268 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73268/testReport)** for PR 11601 at commit [`e86d919`](https://github.com/apache/spark/commit/e86d9198c65c3b289b091150b52708deda37f090).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104524697
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the
    +   * feature (relative error less than 0.001).
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols duplicates:" +
    +      s" (${$(inputCols).mkString(", ")})")
    +    require($(outputCols).length == $(outputCols).distinct.length, s"outputCols duplicates:" +
    +      s" (${$(outputCols).mkString(", ")})")
    +    require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" +
    +      s" and outputCols(${$(outputCols).length}) should have the same length")
    +    val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) =>
    +      val inputField = schema(inputCol)
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      StructField(outputCol, inputField.dataType, inputField.nullable)
    +    }
    +    StructType(schema ++ outputFields)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    --- End diff --
    
    I didn't add the link as it may break java doc generation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #74651 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74651/testReport)** for PR 11601 at commit [`c67afc1`](https://github.com/apache/spark/commit/c67afc11e1fee58b65da67bf3e25e5245f72280d).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61373571
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean") or the
    + * median("median") of the column in which the missing values are located.
    + *
    --- End diff --
    
    Cool. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #66516 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/66516/consoleFull)** for PR 11601 at commit [`8744524`](https://github.com/apache/spark/commit/8744524e8da174316207cb4c33b425cbbd78f68e).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103868010
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    --- End diff --
    
    Not applicable anymore as it's used below now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    jenkins retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61103526
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +      case "mode" => filtered.rdd.map(r => r.getDouble(0)).map(d => (d, 1)).reduceByKey(_ + _)
    +        .sortBy(-_._2).first()._1
    +    }
    +    copyValues(new ImputerModel(uid, surrogate).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("2.0.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median", "mode")
    +
    +  @Since("2.0.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogate statistics value for each feature during fitting
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogate: Double)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    val inputType = dataset.select($(inputCol)).schema.fields(0).dataType
    +    inputType match {
    +      case _: NumericType =>
    +        val ic = col($(inputCol)).cast(DoubleType)
    +        dataset.withColumn($(outputCol), when(ic.isNull, surrogate)
    +          .when(ic === $(missingValue), surrogate)
    +          .otherwise(ic)
    +          .cast(inputType))
    --- End diff --
    
    I propose we either (a) do the cast back to input type, but if the user selected "mean" and the input type is not Float or Double, log a warning; or (b) only support Float and Double type for this initial version of the Imputer.
    
    Thoughts @jkbradley @sethah @holdenk?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61305479
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,141 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.ml.util.{DefaultReadWriteTest}
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.Row
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = sqlContext.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 1.0, 1.0, 1.0),
    +      (2, 3.0, 3.0, 3.0),
    +      (3, 4.0, 4.0, 4.0),
    +      (4, Double.NaN, 2.25, 1.0)
    +    )).toDF("id", "value", "exp_mean", "exp_median")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCol("value").setOutputCol("out").setStrategy(strategy)
    +      val model = imputer.fit(df)
    +      model.transform(df).select("exp_" + strategy, "out").collect().foreach {
    +       case Row(exp: Double, out: Double) =>
    +          assert(exp ~== out absTol 1e-5, s"Imputed values differ. Expected: $exp, actual: $out")
    +      }
    +    }
    +  }
    +
    +  test("Imputer for Double with missing Value -1.0") {
    --- End diff --
    
    I'd argue this test is not necessary because of the test right below, where we test that it works for integer missing values and leaves `NaN`s alone.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-195003941
  
    **[Test build #52842 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52842/consoleFull)** for PR 11601 at commit [`4e45f81`](https://github.com/apache/spark/commit/4e45f81f89f0b1ad13add524b3dd89fe52126bc0).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215335519
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/57229/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103871091
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(outputCols).length, "inputCols and outputCols should have" +
    +      "the same length")
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputSchema = schema
    +
    +    $(inputCols).indices.foreach { i =>
    +      val inputCol = localInputCols(i)
    +      val outputCol = localOutputCols(i)
    +      val inputType = schema(inputCol).dataType
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      outputSchema = SchemaUtils.appendColumn(outputSchema, outputCol, inputType)
    +    }
    +    outputSchema
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.1.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue))
    +        .filter(!ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in ${inputCol} are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case "mean" => filtered.select(avg(inputCol)).first().getDouble(0)
    --- End diff --
    
    slightly prefer `filtered.select(avg(inputCol)).as[Double].first()`  (or `... head`)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by ChristopheDuong <gi...@git.apache.org>.
Github user ChristopheDuong commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r101032949
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,225 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.1.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +      .filter(!ic.isNaN)
    +    if(filtered.count() == 0) {
    --- End diff --
    
    Since we don't actually need the exact total count here, wouldn't it be better to use here?
    `if (filtered.rdd.isEmpty()) {`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r59331281
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    --- End diff --
    
    I see. Thanks. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-208731746
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-208932133
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-208731747
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/55591/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r59323059
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    --- End diff --
    
    Yes. added support for null values.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r58989389
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  private[feature] def isMissingValue(value: Double): Boolean = {
    +    val miss = $(missingValue)
    +    value == miss || (value.isNaN && miss.isNaN)
    +  }
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    --- End diff --
    
    For single numerical column, we can indeed use existing stat functions for computing mean and median (after filtering the missing values), i.e. `mean` and `approxQuantile`.
    
    If we decide to support vector columns, then we need:
    1. `Statistics.colStats` to handle NaN / missing values (null?) [SPARK-13639](https://issues.apache.org/jira/browse/SPARK-13639)
    2. to check whether `approxQuantiles` function can be used for this. My sense is it can't directly, we would need a version that can operate on Array or Vector columns


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104638951
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -99,7 +98,8 @@ private[feature] trait ImputerParams extends Params with HasInputCols {
      * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
      *
      * Note that the mean/median value is computed after filtering out missing values.
    - * All Null values in the input column are treated as missing, and so are also imputed.
    + * All Null values in the input column are treated as missing, and so are also imputed. For
    + * computing median, DataFrameStatFunctions.approxQuantile is used with a relative error of 0.001.
    --- End diff --
    
    Ah I see it is here - nevermind


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104409545
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,166 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.{DataFrame, Row}
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 4.0, 1.0, 1.0, 4.0, 4.0),
    +      (1, 11.0, 12.0, 11.0, 11.0, 12.0, 12.0),
    +      (2, 3.0, Double.NaN, 3.0, 3.0, 10.0, 12.0),
    +      (3, Double.NaN, 14.0, 5.0, 3.0, 14.0, 14.0)
    +    )).toDF("id", "value1", "value2", "expected_mean_value1", "expected_median_value1",
    +      "expected_mean_value2", "expected_median_value2")
    +    val imputer = new Imputer()
    +      .setInputCols(Array("value1", "value2"))
    +      .setOutputCols(Array("out1", "out2"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should handle NaNs when computing surrogate value, if missingValue is not NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1.0)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer for Float with missing Value -1.0") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0F, 1.0F, 1.0F),
    +      (1, 3.0F, 3.0F, 3.0F),
    +      (2, 10.0F, 10.0F, 10.0F),
    +      (3, 10.0F, 10.0F, 10.0F),
    +      (4, -1.0F, 6.0F, 3.0F)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should impute null as well as 'missingValue'") {
    +    val rawDf = spark.createDataFrame( Seq(
    +      (0, 4.0, 4.0, 4.0),
    +      (1, 10.0, 10.0, 10.0),
    +      (2, 10.0, 10.0, 10.0),
    +      (3, Double.NaN, 8.0, 10.0),
    +      (4, -1.0, 8.0, 10.0)
    +    )).toDF("id", "rawValue", "expected_mean_value", "expected_median_value")
    +    val df = rawDf.selectExpr("*", "IF(rawValue=-1.0, null, rawValue) as value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer throws exception when surrogate cannot be computed") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, Double.NaN, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +        .setStrategy(strategy)
    +      intercept[SparkException] {
    --- End diff --
    
    Check message here also.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104406006
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the
    +   * feature (relative error less than 0.001).
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols duplicates:" +
    +      s" (${$(inputCols).mkString(", ")})")
    +    require($(outputCols).length == $(outputCols).distinct.length, s"outputCols duplicates:" +
    +      s" (${$(outputCols).mkString(", ")})")
    +    require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" +
    +      s" and outputCols(${$(outputCols).length}) should have the same length")
    +    val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) =>
    +      val inputField = schema(inputCol)
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      StructField(outputCol, inputField.dataType, inputField.nullable)
    +    }
    +    StructType(schema ++ outputFields)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.2.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.2.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.2.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  import org.apache.spark.ml.feature.Imputer._
    +  setDefault(strategy -> mean, missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val spark = dataset.sparkSession
    +    import spark.implicits._
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue) && !ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in $inputCol are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case Imputer.mean => filtered.select(avg(inputCol)).as[Double].first()
    +        case Imputer.median => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001).head
    +      }
    +      surrogate
    +    }
    +
    +    val rows = spark.sparkContext.parallelize(Seq(Row.fromSeq(surrogates)))
    +    val schema = StructType($(inputCols).map(col => StructField(col, DoubleType, nullable = false)))
    +    val surrogateDF = spark.createDataFrame(rows, schema)
    +    copyValues(new ImputerModel(uid, surrogateDF).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = defaultCopy(extra)
    +}
    +
    +@Since("2.2.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** strategy names that Imputer currently supports. */
    +  private[ml] val mean = "mean"
    +  private[ml] val median = "median"
    +
    +  @Since("2.2.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogateDF a DataFrame contains inputCols and their corresponding surrogates, which are
    + *                    used to replace the missing values in the input DataFrame.
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogateDF: DataFrame)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    var outputDF = dataset
    +    val surrogates = surrogateDF.select($(inputCols).head, $(inputCols).tail: _*).head().toSeq
    --- End diff --
    
    Maybe this is slightly cleaner: `surrogateDF.select($(inputCols).map(col): _*)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Thanks a lot for making a pass @MLnick. The last update mainly focus on the interface and behavior change. I'll make a pass and also address your comments. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61098866
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    --- End diff --
    
    I think we can use `SchemaUtils.checkColumnType` here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r58922750
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  private[feature] def isMissingValue(value: Double): Boolean = {
    +    val miss = $(missingValue)
    +    value == miss || (value.isNaN && miss.isNaN)
    +  }
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    --- End diff --
    
    Document the accepted input schema


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by BenFradet <gi...@git.apache.org>.
Github user BenFradet commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61110533
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    --- End diff --
    
    `checkNumericType` even


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-194169449
  
    **[Test build #52734 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52734/consoleFull)** for PR 11601 at commit [`1b39668`](https://github.com/apache/spark/commit/1b3966800982fa980307d1b6ded6e28e5f5985e8).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #74038 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74038/testReport)** for PR 11601 at commit [`e378db5`](https://github.com/apache/spark/commit/e378db5944d7d8bed0ebadc0573a3ea03fe387f0).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Made a few last comments. LGTM.
    
    cc @sethah @jkbradley I am going to merge this for 2.2. Let me know if you have any final comments.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r55810108
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,290 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    validateParams()
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        Vectors.dense(getColStatistics(dataset, $(inputCol)))
    +      case _: VectorUDT =>
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        val statisticsArray = new Array[Double](vl)
    +        (0 until vl).foreach(i => {
    +          val getI = udf((v: Vector) => v(i))
    +          val tempColName = $(inputCol) + i
    +          val tempData = dataset.where(s"${$(inputCol)} IS NOT NULL")
    +            .select($(inputCol)).withColumn(tempColName, getI(col($(inputCol))))
    +          statisticsArray(i) = getColStatistics(tempData, tempColName)
    +        })
    +        Vectors.dense(statisticsArray)
    +    }
    +    copyValues(new ImputerModel(uid, alternate).setParent(this))
    +  }
    +
    +  /** Extract the statistics info from a Double column according to the strategy */
    +  private def getColStatistics(dataset: DataFrame, colName: String): Double = {
    +    val missValue = $(missingValue) match {
    +      case Double.NaN => "NaN"
    +      case _ => $(missingValue).toString
    +    }
    +    val filteredDF = dataset.select(colName).where(s"$colName != '$missValue'")
    +    val colStatistics = $(strategy) match {
    +      case "mean" =>
    +        filteredDF.selectExpr(s"avg($colName)").first().getDouble(0)
    +      case "median" =>
    +        // TODO: optimize the sort with quick-select or Percentile(Hive) if required
    +        val rddDouble = filteredDF.rdd.map(_.getDouble(0))
    +        rddDouble.sortBy(d => d).zipWithIndex().map {
    +          case (v, idx) => (idx, v)
    +        }.lookup(rddDouble.count() / 2).head
    +      case "most" =>
    --- End diff --
    
    I'm a little worried about performance here - on huge columns of `Double` this could be a problem. I struggle to actually see a use case for `most`... perhaps on categorical or ordinal columns it can make sense. 
    
    Will think about it a bit more.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r59330944
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    --- End diff --
    
    Rename "most" to "mode" (mode is the most frequent value in the dataset)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215143820
  
    **[Test build #57139 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57139/consoleFull)** for PR 11601 at commit [`b3633e8`](https://github.com/apache/spark/commit/b3633e8dd0edf47a684aa344ba6a3c43ac0d91fe).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103878058
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(outputCols).length, "inputCols and outputCols should have" +
    +      "the same length")
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputSchema = schema
    +
    +    $(inputCols).indices.foreach { i =>
    +      val inputCol = localInputCols(i)
    +      val outputCol = localOutputCols(i)
    +      val inputType = schema(inputCol).dataType
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      outputSchema = SchemaUtils.appendColumn(outputSchema, outputCol, inputType)
    +    }
    +    outputSchema
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.1.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.1.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue))
    +        .filter(!ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in ${inputCol} are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case "mean" => filtered.select(avg(inputCol)).first().getDouble(0)
    +        case "median" => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001)(0)
    +      }
    +      surrogate.asInstanceOf[Double]
    +    }
    +
    +    import dataset.sparkSession.implicits._
    +    val surrogateDF = Seq(surrogates).toDF("surrogates")
    +    copyValues(new ImputerModel(uid, surrogateDF).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = defaultCopy(extra)
    +}
    +
    +@Since("2.1.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median")
    +
    +  @Since("2.1.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogateDF Value by which missing values in the input columns will be replaced. This
    + *    is stored using DataFrame with input column names and the corresponding surrogates.
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogateDF: DataFrame)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputDF = dataset
    +    val surrogates = surrogateDF.head().getSeq[Double](0)
    +
    +    $(inputCols).indices.foreach { i =>
    --- End diff --
    
    You could do `$(inputCols).zip($(outputCols)).zip(surrogates).map { case ((inputCol, outputCol), icSurrogate) => ...`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104407942
  
    --- Diff: mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala ---
    @@ -0,0 +1,166 @@
    +/*
    + * 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.feature
    +
    +import org.apache.spark.{SparkException, SparkFunSuite}
    +import org.apache.spark.ml.util.DefaultReadWriteTest
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +import org.apache.spark.sql.{DataFrame, Row}
    +
    +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest {
    +
    +  test("Imputer for Double with default missing Value NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 4.0, 1.0, 1.0, 4.0, 4.0),
    +      (1, 11.0, 12.0, 11.0, 11.0, 12.0, 12.0),
    +      (2, 3.0, Double.NaN, 3.0, 3.0, 10.0, 12.0),
    +      (3, Double.NaN, 14.0, 5.0, 3.0, 14.0, 14.0)
    +    )).toDF("id", "value1", "value2", "expected_mean_value1", "expected_median_value1",
    +      "expected_mean_value2", "expected_median_value2")
    +    val imputer = new Imputer()
    +      .setInputCols(Array("value1", "value2"))
    +      .setOutputCols(Array("out1", "out2"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should handle NaNs when computing surrogate value, if missingValue is not NaN") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, 3.0, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN),
    +      (3, -1.0, 2.0, 3.0)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1.0)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer for Float with missing Value -1.0") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0F, 1.0F, 1.0F),
    +      (1, 3.0F, 3.0F, 3.0F),
    +      (2, 10.0F, 10.0F, 10.0F),
    +      (3, 10.0F, 10.0F, 10.0F),
    +      (4, -1.0F, 6.0F, 3.0F)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +      .setMissingValue(-1)
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer should impute null as well as 'missingValue'") {
    +    val rawDf = spark.createDataFrame( Seq(
    +      (0, 4.0, 4.0, 4.0),
    +      (1, 10.0, 10.0, 10.0),
    +      (2, 10.0, 10.0, 10.0),
    +      (3, Double.NaN, 8.0, 10.0),
    +      (4, -1.0, 8.0, 10.0)
    +    )).toDF("id", "rawValue", "expected_mean_value", "expected_median_value")
    +    val df = rawDf.selectExpr("*", "IF(rawValue=-1.0, null, rawValue) as value")
    +    val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +    ImputerSuite.iterateStrategyTest(imputer, df)
    +  }
    +
    +  test("Imputer throws exception when surrogate cannot be computed") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, Double.NaN, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value", "expected_mean_value", "expected_median_value")
    +    Seq("mean", "median").foreach { strategy =>
    +      val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out"))
    +        .setStrategy(strategy)
    +      intercept[SparkException] {
    +        val model = imputer.fit(df)
    +      }
    +    }
    +  }
    +
    +  test("Imputer throws exception when inputCols does not match outputCols") {
    +    val df = spark.createDataFrame( Seq(
    +      (0, 1.0, 1.0, 1.0),
    +      (1, Double.NaN, 3.0, 3.0),
    +      (2, Double.NaN, Double.NaN, Double.NaN)
    +    )).toDF("id", "value1", "value2", "value3")
    +    Seq("mean", "median").foreach { strategy =>
    +      // inputCols and outCols length different
    +      val imputer = new Imputer()
    +        .setInputCols(Array("value1", "value2"))
    +        .setOutputCols(Array("out1"))
    +        .setStrategy(strategy)
    +      intercept[IllegalArgumentException] {
    --- End diff --
    
    Also test for thrown message here and `withClue`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r55617958
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, Params}
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean along the axis.
    +    * If "median", then replace missing values using the median along the axis.
    +    * If "most", then replace missing using the most frequent value along the axis.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean along the axis." +
    +    "If median, then replace missing values using the median along the axis." +
    +    "If most, then replace missing using the most frequent value along the axis.")
    --- End diff --
    
    I added the validation to validateParameter. (which should be moved since it's the deprecated). Thanks for the suggestion. I'll add them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/65053/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r55651413
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, Params}
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean along the axis.
    +    * If "median", then replace missing values using the median along the axis.
    +    * If "most", then replace missing using the most frequent value along the axis.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean along the axis." +
    +    "If median, then replace missing values using the median along the axis." +
    +    "If most, then replace missing using the most frequent value along the axis.")
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingvalues will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingvalues will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    validateParams()
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +  override def validateParams(): Unit = {
    +    require(Seq("mean", "median", "most").contains($(strategy)),
    +      s"${$(strategy)} is not supported. Options are mean, median and most")
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values encodings.
    + *
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /** @group setParam */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        val colStatistics = getColStatistics(dataset, $(inputCol))
    +        Vectors.dense(Array(colStatistics))
    +      case _: VectorUDT =>
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        val statisticsArray = new Array[Double](vl)
    +        (0 until vl).foreach(i => {
    +          val getI = udf((v: Vector) => v(i))
    +          val tempColName = $(inputCol) + i
    +          val tempData = dataset.where(s"${$(inputCol)} is not null")
    +            .select($(inputCol)).withColumn(tempColName, getI(col($(inputCol))))
    +          statisticsArray(i) = getColStatistics(tempData, tempColName)
    +        })
    +        Vectors.dense(statisticsArray)
    +    }
    +    copyValues(new ImputerModel(uid, alternate).setParent(this))
    +  }
    +
    +  private def getColStatistics(dataset: DataFrame, colName: String): Double = {
    +    val missValue = $(missingValue) match {
    +      case Double.NaN => "NaN"
    +      case _ => $(missingValue).toString
    +    }
    +    val filteredDF = dataset.select(colName).where(s"$colName != '$missValue'")
    +    val colStatistics = $(strategy) match {
    +      case "mean" =>
    +        filteredDF.selectExpr(s"avg($colName)").first().getDouble(0)
    +      case "median" =>
    +        // TODO: optimize the sort with quick-select or Percentile(Hive) if required
    +        val rddDouble = filteredDF.rdd.map(_.getDouble(0))
    +        rddDouble.sortBy(d => d).zipWithIndex().map {
    +          case (v, idx) => (idx, v)
    +        }.lookup(rddDouble.count()/2).head
    +      case "most" =>
    +        val input = filteredDF.rdd.map(_.getDouble(0))
    +        val most = input.map(d => (d, 1)).reduceByKey(_ + _).sortBy(-_._2).first()._1
    +        most
    +    }
    +    colStatistics
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("1.6.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  @Since("1.6.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    +  * :: Experimental ::
    +  * Model fitted by [[Imputer]].
    +  *
    +  * @param alternate statistics value for each original column during fitting
    +  */
    +@Experimental
    +class ImputerModel private[ml] (
    +    override val uid: String,
    +    val alternate: Vector)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  private def matchMissingValue(value: Double): Boolean = {
    --- End diff --
    
    Agree


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61163112
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,231 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.types.NumericType
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the median value of the feature.
    +   * If "mode", then replace missing using the most frequent value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If mode, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[NumericType],
    +      s"Input column ${$(inputCol)} must be of NumericType")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+
    +      StructField($(outputCol), inputType, schema($(inputCol)).nullable)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean"), the
    + * median("median") or the most frequent value("mode") of the column in which the missing
    + * values are located.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median" and "mode"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +      case "mode" => filtered.rdd.map(r => r.getDouble(0)).map(d => (d, 1)).reduceByKey(_ + _)
    +        .sortBy(-_._2).first()._1
    +    }
    +    copyValues(new ImputerModel(uid, surrogate).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("2.0.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median", "mode")
    +
    +  @Since("2.0.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogate statistics value for each feature during fitting
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogate: Double)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    val inputType = dataset.select($(inputCol)).schema.fields(0).dataType
    +    inputType match {
    +      case _: NumericType =>
    +        val ic = col($(inputCol)).cast(DoubleType)
    +        dataset.withColumn($(outputCol), when(ic.isNull, surrogate)
    +          .when(ic === $(missingValue), surrogate)
    +          .otherwise(ic)
    +          .cast(inputType))
    --- End diff --
    
    Sorry, that should have said option a.) !


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-208722190
  
    **[Test build #55591 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/55591/consoleFull)** for PR 11601 at commit [`fdd6f94`](https://github.com/apache/spark/commit/fdd6f943da2123aebaca4fe9d48ce6b6356bfa42).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hash-X <gi...@git.apache.org>.
Github user hash-X commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-208934749
  
    @hhbyyh 
    WOW, I have been finding how to impute missing values in spark project lately. LGTM. Thank you.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103868238
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    --- End diff --
    
    We don't use `HasOutputCol` anymore, correct?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #73753 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73753/testReport)** for PR 11601 at commit [`e86d919`](https://github.com/apache/spark/commit/e86d9198c65c3b289b091150b52708deda37f090).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by sethah <gi...@git.apache.org>.
Github user sethah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r55561054
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,288 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, Params}
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean along the axis.
    +    * If "median", then replace missing values using the median along the axis.
    +    * If "most", then replace missing using the most frequent value along the axis.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean along the axis." +
    +    "If median, then replace missing values using the median along the axis." +
    +    "If most, then replace missing using the most frequent value along the axis.")
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingvalues will be imputed.
    --- End diff --
    
    Space missing from "missingvalues"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215141731
  
    **[Test build #57139 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57139/consoleFull)** for PR 11601 at commit [`b3633e8`](https://github.com/apache/spark/commit/b3633e8dd0edf47a684aa344ba6a3c43ac0d91fe).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r58664876
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  private[feature] def isMissingValue(value: Double): Boolean = {
    +    val miss = $(missingValue)
    +    value == miss || (value.isNaN && miss.isNaN)
    +  }
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        val doubleRDD = dataset.select($(inputCol)).rdd.map(_.getDouble(0))
    +        Vectors.dense(getColStatistics(doubleRDD))
    +      case _: VectorUDT =>
    +        val filteredDF = dataset.where(s"${$(inputCol)} IS NOT NULL").select($(inputCol))
    +        val vectorRDD = filteredDF.rdd.map(_.getAs[Vector](0)).cache()
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        $(strategy) match {
    +          case "mean" =>
    +            val summary = vectorRDD.treeAggregate((new Array[Double](vl), new Array[Int](vl)))(
    +              (prev, data) => (prev, data) match { case ((mean, count), data) =>
    +                  var i = 0
    +                  while (i < mean.length) {
    +                    if (data(i) != 0 && !data(i).isNaN) {
    +                      count(i) += 1
    +                      mean(i) = mean(i) + (data(i) - mean(i)) / count(i)
    +                    }
    +                    i += 1
    +                  }
    +                  (mean, count)
    +              }, (aggregator1, aggregator2) => (aggregator1, aggregator2) match {
    +                case ((mean1, c1), (mean2, c2)) =>
    +                  (0 until mean1.length).foreach{ i =>
    +                    mean1(i) = mean1(i) + (mean2(i) - mean1(i)) * c2(i) / (c1(i) + c2(i))
    +                    c1(i) += c2(i)
    +                  }
    +                  (mean1, c1)
    +              })
    +            Vectors.dense(summary._1)
    +          case _ =>
    +            val statisticsArray = new Array[Double](vl)
    +            (0 until vl).foreach(i => {
    +              statisticsArray(i) = getColStatistics(vectorRDD.map(v => v(i)))
    +            })
    +            Vectors.dense(statisticsArray)
    +        }
    +    }
    +    copyValues(new ImputerModel(uid, alternate).setParent(this))
    +  }
    +
    +  /** Extract the statistics info from a Double column according to the strategy */
    +  private def getColStatistics(data: RDD[Double]): Double = {
    +    val filteredRDD = data.filter(!isMissingValue(_))
    +    val colStatistics = $(strategy) match {
    +      case "mean" => filteredRDD.mean()
    +      case "median" => filteredRDD.sortBy(d => d).zipWithIndex()
    --- End diff --
    
    Ah - it's in `StatFunctions.multipleApproxQuantiles` but I see that is `private[sql]`. 
    
    @viirya @jkbradley why doesn't `approxQuantile` support multiple columns (as the underlying impl does)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r104516526
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.HasInputCols
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the
    +   * feature (relative error less than 0.001).
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " +
    +    s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " +
    +    s"If ${Imputer.median}, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median)))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(inputCols).distinct.length, s"inputCols duplicates:" +
    +      s" (${$(inputCols).mkString(", ")})")
    +    require($(outputCols).length == $(outputCols).distinct.length, s"outputCols duplicates:" +
    +      s" (${$(outputCols).mkString(", ")})")
    +    require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" +
    +      s" and outputCols(${$(outputCols).length}) should have the same length")
    +    val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) =>
    +      val inputField = schema(inputCol)
    +      SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType))
    +      StructField(outputCol, inputField.dataType, inputField.nullable)
    +    }
    +    StructType(schema ++ outputFields)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType. Currently Imputer does not support categorical features yet
    + * (SPARK-15041) and possibly creates incorrect values for a categorical feature.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.2.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.2.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setInputCols(value: Array[String]): this.type = set(inputCols, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setOutputCols(value: Array[String]): this.type = set(outputCols, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  @Since("2.2.0")
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  @Since("2.2.0")
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  import org.apache.spark.ml.feature.Imputer._
    +  setDefault(strategy -> mean, missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val spark = dataset.sparkSession
    +    import spark.implicits._
    +    val surrogates = $(inputCols).map { inputCol =>
    +      val ic = col(inputCol)
    +      val filtered = dataset.select(ic.cast(DoubleType))
    +        .filter(ic.isNotNull && ic =!= $(missingValue) && !ic.isNaN)
    +      if(filtered.rdd.isEmpty()) {
    +        throw new SparkException(s"surrogate cannot be computed. " +
    +          s"All the values in $inputCol are Null, Nan or missingValue ($missingValue)")
    +      }
    +      val surrogate = $(strategy) match {
    +        case Imputer.mean => filtered.select(avg(inputCol)).as[Double].first()
    +        case Imputer.median => filtered.stat.approxQuantile(inputCol, Array(0.5), 0.001).head
    --- End diff --
    
    I tried it before. 0.01 and 0.001 actually takes the same time for even a large dataset. Agree we can make it a param later.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    **[Test build #65053 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65053/consoleFull)** for PR 11601 at commit [`544a65c`](https://github.com/apache/spark/commit/544a65c82a7d921bdff73998e8b350e11b51dcbe).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class Imputer @Since(\"2.1.0\")(override val uid: String)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r55810884
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,290 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    validateParams()
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        Vectors.dense(getColStatistics(dataset, $(inputCol)))
    +      case _: VectorUDT =>
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        val statisticsArray = new Array[Double](vl)
    +        (0 until vl).foreach(i => {
    --- End diff --
    
    This could be a big performance issue with large vectors, as we could be running 100s (or millions!) of SQL queries sequentially... For vectors I favour the `colStats` approach of using `MultivariateOnlineSummarizer` for efficiency. I think that if we support vectors here, we should rather enable ignoring the NaNs in `colStats` or a deeper API (even if we do it as `private` or `DevelopApi` for now).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215650485
  
    **[Test build #57321 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57321/consoleFull)** for PR 11601 at commit [`335ded7`](https://github.com/apache/spark/commit/335ded7d92d7ab6f50e2d16348cfde051eb26579).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-195020249
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r80594974
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean or the median
    + * of the column in which the missing values are located. The input column should be of
    + * DoubleType or FloatType.
    + *
    + * Note that the mean/median value is computed after filtering out missing values.
    + * All Null values in the input column are treated as missing, and so are also imputed.
    + */
    +@Experimental
    +class Imputer @Since("2.1.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.1.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are ["mean", "median"].
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: Dataset[_]): ImputerModel = {
    +    transformSchema(dataset.schema, logging = true)
    +    val ic = col($(inputCol))
    +    val filtered = dataset.select(ic.cast(DoubleType))
    +      .filter(ic.isNotNull && ic =!= $(missingValue))
    +    val surrogate = $(strategy) match {
    +      case "mean" => filtered.filter(!ic.isNaN).select(avg($(inputCol))).first().getDouble(0)
    +      case "median" => filtered.stat.approxQuantile($(inputCol), Array(0.5), 0.001)(0)
    +    }
    +    copyValues(new ImputerModel(uid, surrogate).setParent(this))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): Imputer = {
    +    val copied = new Imputer(uid)
    +    copyValues(copied, extra)
    +  }
    +}
    +
    +@Since("2.1.0")
    +object Imputer extends DefaultParamsReadable[Imputer] {
    +
    +  /** Set of strategy names that Imputer currently supports. */
    +  private[ml] val supportedStrategyNames = Set("mean", "median")
    +
    +  @Since("2.1.0")
    +  override def load(path: String): Imputer = super.load(path)
    +}
    +
    +/**
    + * :: Experimental ::
    + * Model fitted by [[Imputer]].
    + *
    + * @param surrogate Value by which missing values in the input column will be replaced.
    + */
    +@Experimental
    +class ImputerModel private[ml](
    +    override val uid: String,
    +    val surrogate: Double)
    +  extends Model[ImputerModel] with ImputerParams with MLWritable {
    +
    +  import ImputerModel._
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  override def transform(dataset: Dataset[_]): DataFrame = {
    +    transformSchema(dataset.schema, logging = true)
    +    val inputType = dataset.select($(inputCol)).schema.fields(0).dataType
    +    val ic = col($(inputCol))
    +    dataset.withColumn($(outputCol), when(ic.isNull, surrogate)
    +      .when(ic === $(missingValue), surrogate)
    +      .otherwise(ic)
    +      .cast(inputType))
    +  }
    +
    +  override def transformSchema(schema: StructType): StructType = {
    +    validateAndTransformSchema(schema)
    +  }
    +
    +  override def copy(extra: ParamMap): ImputerModel = {
    +    val copied = new ImputerModel(uid, surrogate)
    +    copyValues(copied, extra).setParent(parent)
    +  }
    +
    +  @Since("2.1.0")
    +  override def write: MLWriter = new ImputerModelWriter(this)
    +}
    +
    +
    +@Since("2.1.0")
    +object ImputerModel extends MLReadable[ImputerModel] {
    +
    +  private[ImputerModel] class ImputerModelWriter(instance: ImputerModel) extends MLWriter {
    +
    +    private case class Data(surrogate: Double)
    --- End diff --
    
    Should we save an ```ArrayType[VectorUDT]``` or ```ArrayType[ArrayType[Double]]``` here?  That will make this extensible to multiple columns, including mixed NumericType and Vector columns, in the future without us having to change the persistence format.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-200275409
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Looks like CI was interrupted.
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73268/console


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215656460
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-215615465
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r61317809
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    SchemaUtils.checkColumnTypes(schema, $(inputCol), Seq(DoubleType, FloatType))
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    SchemaUtils.appendColumn(schema, $(outputCol), inputType)
    +  }
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean("mean") or the
    + * median("median") of the column in which the missing values are located.
    + *
    + * Note that all the null values will be imputed as well.
    --- End diff --
    
    Can we amend the doc slightly to say something like "Note that null values in the input column are treated as missing, and so are also imputed"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-212503265
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r57700783
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,300 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  private[feature] def isMissingValue(value: Double): Boolean = {
    +    val miss = $(missingValue)
    +    value == miss || (value.isNaN && miss.isNaN)
    +  }
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        val doubleRDD = dataset.select($(inputCol)).rdd.map(_.getDouble(0))
    +        Vectors.dense(getColStatistics(doubleRDD))
    +      case _: VectorUDT =>
    +        val filteredDF = dataset.where(s"${$(inputCol)} IS NOT NULL").select($(inputCol))
    +        val vectorRDD = filteredDF.rdd.map(_.getAs[Vector](0)).cache()
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        $(strategy) match {
    +          case "mean" =>
    +            val summary = vectorRDD.treeAggregate((new Array[Double](vl), new Array[Int](vl)))(
    +              (prev, data) => (prev, data) match { case ((mean, count), data) =>
    +                  var i = 0
    +                  while (i < mean.length) {
    +                    if (data(i) != 0 && !data(i).isNaN) {
    +                      count(i) += 1
    +                      mean(i) = mean(i) + (data(i) - mean(i)) / count(i)
    +                    }
    +                    i += 1
    +                  }
    +                  (mean, count)
    +              }, (aggregator1, aggregator2) => (aggregator1, aggregator2) match {
    +                case ((mean1, c1), (mean2, c2)) =>
    +                  (0 until mean1.length).foreach{ i =>
    +                    mean1(i) = mean1(i) + (mean2(i) - mean1(i)) * c2(i) / (c1(i) + c2(i))
    +                    c1(i) += c2(i)
    +                  }
    +                  (mean1, c1)
    +              })
    +            Vectors.dense(summary._1)
    +          case _ =>
    +            val statisticsArray = new Array[Double](vl)
    +            (0 until vl).foreach(i => {
    +              statisticsArray(i) = getColStatistics(vectorRDD.map(v => v(i)))
    +            })
    +            Vectors.dense(statisticsArray)
    +        }
    +    }
    +    copyValues(new ImputerModel(uid, alternate).setParent(this))
    +  }
    +
    +  /** Extract the statistics info from a Double column according to the strategy */
    +  private def getColStatistics(data: RDD[Double]): Double = {
    +    val filteredRDD = data.filter(!isMissingValue(_))
    +    val colStatistics = $(strategy) match {
    +      case "mean" => filteredRDD.mean()
    +      case "median" => filteredRDD.sortBy(d => d).zipWithIndex()
    --- End diff --
    
    Could you look into using `approxQuantile` from DataFrame stats functions? Passing `Array(0.5)` as arg for that should give median. It can also handle multiple cols I think, and is way more efficient.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by hhbyyh <gi...@git.apache.org>.
Github user hhbyyh commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r55868373
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,290 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.mllib.linalg._
    +import org.apache.spark.sql.{DataFrame, Row}
    +import org.apache.spark.sql.functions.{col, udf}
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
    +
    +/**
    +  * Params for [[Imputer]] and [[ImputerModel]].
    +  */
    +private[feature] trait ImputerParams extends Params with HasInputCol with HasOutputCol {
    +
    +  /**
    +    * The imputation strategy.
    +    * If "mean", then replace missing values using the mean value of the feature.
    +    * If "median", then replace missing values using the median value of the feature.
    +    * If "most", then replace missing using the most frequent value of the feature.
    +    * Default: mean
    +    *
    +    * @group param
    +    */
    +  val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature." +
    +    "If median, then replace missing values using the median value of the feature." +
    +    "If most, then replace missing using the most frequent value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +    * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +    * Default: Double.NaN
    +    *
    +    * @group param
    +    */
    +  val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    validateParams()
    +    val inputType = schema($(inputCol)).dataType
    +    require(inputType.isInstanceOf[VectorUDT] || inputType.isInstanceOf[DoubleType],
    +      s"Input column ${$(inputCol)} must of type Vector or Double")
    +    require(!schema.fieldNames.contains($(outputCol)),
    +      s"Output column ${$(outputCol)} already exists.")
    +    val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false)
    +    StructType(outputFields)
    +  }
    +
    +}
    +
    +/**
    + * :: Experimental ::
    + * Imputation estimator for completing missing values, either using the mean, the median or
    + * the most frequent value of the column in which the missing values are located. This class
    + * also allows for different missing values.
    + */
    +@Experimental
    +class Imputer @Since("2.0.0")(override val uid: String)
    +  extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable {
    +
    +  @Since("2.0.0")
    +  def this() = this(Identifiable.randomUID("imputer"))
    +
    +  /** @group setParam */
    +  def setInputCol(value: String): this.type = set(inputCol, value)
    +
    +  /** @group setParam */
    +  def setOutputCol(value: String): this.type = set(outputCol, value)
    +
    +  /**
    +   * Imputation strategy. Available options are "mean", "median" and "most".
    +   * @group setParam
    +   */
    +  def setStrategy(value: String): this.type = set(strategy, value)
    +
    +  /** @group setParam */
    +  def setMissingValue(value: Double): this.type = set(missingValue, value)
    +
    +  setDefault(strategy -> "mean", missingValue -> Double.NaN)
    +
    +  override def fit(dataset: DataFrame): ImputerModel = {
    +    val alternate = dataset.select($(inputCol)).schema.fields(0).dataType match {
    +      case DoubleType =>
    +        Vectors.dense(getColStatistics(dataset, $(inputCol)))
    +      case _: VectorUDT =>
    +        val vl = dataset.first().getAs[Vector]($(inputCol)).size
    +        val statisticsArray = new Array[Double](vl)
    +        (0 until vl).foreach(i => {
    --- End diff --
    
    I see. I'll run some performance benchmark to compare the solutions and get back to you. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/11601


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #11601: [SPARK-13568] [ML] Create feature transformer to impute ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/11601
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-13568] [ML] Create feature transformer ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11601#issuecomment-210947421
  
    **[Test build #56035 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/56035/consoleFull)** for PR 11601 at commit [`1718422`](https://github.com/apache/spark/commit/171842210d3ea2e3c97fe803f0a8bb3831063f3f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #11601: [SPARK-13568] [ML] Create feature transformer to ...

Posted by MLnick <gi...@git.apache.org>.
Github user MLnick commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11601#discussion_r103878625
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * 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.feature
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.ml.{Estimator, Model}
    +import org.apache.spark.ml.param._
    +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol}
    +import org.apache.spark.ml.util._
    +import org.apache.spark.sql.{DataFrame, Dataset, Row}
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Params for [[Imputer]] and [[ImputerModel]].
    + */
    +private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCol {
    +
    +  /**
    +   * The imputation strategy.
    +   * If "mean", then replace missing values using the mean value of the feature.
    +   * If "median", then replace missing values using the approximate median value of the feature.
    +   * Default: mean
    +   *
    +   * @group param
    +   */
    +  final val strategy: Param[String] = new Param(this, "strategy", "strategy for imputation. " +
    +    "If mean, then replace missing values using the mean value of the feature. " +
    +    "If median, then replace missing values using the median value of the feature.",
    +    ParamValidators.inArray[String](Imputer.supportedStrategyNames.toArray))
    +
    +  /** @group getParam */
    +  def getStrategy: String = $(strategy)
    +
    +  /**
    +   * The placeholder for the missing values. All occurrences of missingValue will be imputed.
    +   * Note that null values are always treated as missing.
    +   * Default: Double.NaN
    +   *
    +   * @group param
    +   */
    +  final val missingValue: DoubleParam = new DoubleParam(this, "missingValue",
    +    "The placeholder for the missing values. All occurrences of missingValue will be imputed")
    +
    +  /** @group getParam */
    +  def getMissingValue: Double = $(missingValue)
    +
    +    /**
    +   * Param for output column names.
    +   * @group param
    +   */
    +  final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols",
    +    "output column names")
    +
    +  /** @group getParam */
    +  final def getOutputCols: Array[String] = $(outputCols)
    +
    +  /** Validates and transforms the input schema. */
    +  protected def validateAndTransformSchema(schema: StructType): StructType = {
    +    require($(inputCols).length == $(outputCols).length, "inputCols and outputCols should have" +
    +      "the same length")
    +    val localInputCols = $(inputCols)
    +    val localOutputCols = $(outputCols)
    +    var outputSchema = schema
    +
    +    $(inputCols).indices.foreach { i =>
    --- End diff --
    
    Can do `$(inputCols).zip($(outputCols)).foreach { case (inputCol, outputCol) => ...`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org